SlideShare ist ein Scribd-Unternehmen logo
1 von 58
Downloaden Sie, um offline zu lesen
Akka Streams & HTTP
Dr. Roland Kuhn
@rolandkuhn — Akka Tech Lead
Streams Occur Naturally
• traditionally:
• file input/output
• network connections
• more modern:
• processing big data with finite memory
• real-time data processing (CEP)
• serving numerous clients simultaneously with bounded
resources (IoT, streaming HTTP APIs)
2
3
What is a Stream?
• ephemeral, time-dependent sequence of elements
• possibly unbounded in length
• therefore focusing on transformations
«You cannot step twice into the same stream.
For as you are stepping in, other waters are ever
flowing on to you.» — Heraclitus
Reactive Traits
4
Reactive means Message Flow
5
Reactive means Message Flow
6
37% discount
with code
kuhnco
Distributed vs. Static Typing
• in general a very tough problem
• independent entities evolving concurrently
• promising progress on Project Gålbma
• in many specific cases no type evolution necessary
• letting uniform messages flow in safe conduits
• making streams even more interesting
7
Possible Solutions
• the Traditional way: blocking calls
8
Possible Solutions
• the Push way: buffering and/or dropping

(optimized for fast consumer)
9
Possible Solutions
• the Pull way: poor performance

(optimized for fast producer)
10
Possible Solutions
• the Reactive way:

non-blocking & non-dropping & bounded
11
Reactive Streams
Origin and motivation
• all participants face the same basic problem
• all are building tools for their community
• a common solution benefits everybody
• interoperability to make best use of efforts
• propose to include in future JDK
13
Goals
• minimal interfaces—essentials only
• rigorous specification of semantics
• TCK for verification of implementation
• complete freedom for many idiomatic APIs
• specification should be efficiently implementable
14
Reactive Streams
• asynchronous & non-blocking
• flow of data
• flow of demand
• minimal coordination and contention
• message passing allows for distribution across
• applications, nodes, CPUs, threads, actors
15
A Data Market using Supply & Demand
• data elements flow downstream
• demand flows upstream
• data elements flow only when there is demand
• data in flight is bounded by signaled demand
• recipient is in control of maximal incoming data rate
16
Publisher Subscriber
data
demand
Reactive Push–Pull
• “push”—when consumer is faster
• “pull”—when producer is faster
• switches automatically between these
• batching demand allows batching data
17
Publisher Subscriber
data
demand
Explicit Demand: One-to-many
18
demand
data
Splitting the data means merging the demand
Explicit Demand: Many-to-one
19
Merging the data means splitting the demand
Back-Pressure is Contagious
• C is slow
• B must slow down
• A must slow down
20
CA B
• TCP for example has it built-in
Back-Pressure can be Propagated
21
CA B
networkhosts
The Meat
22
trait Publisher[T] {
def subscribe(sub: Subscriber[T]): Unit
}
trait Subscription {
def request(n: Long): Unit
def cancel(): Unit
}
trait Subscriber[T] {
def onSubscribe(s: Subscription): Unit
def onNext(e: T): Unit
def onError(t: Throwable): Unit
def onComplete(): Unit
}
Not user-level API
Intended as SPI for interop
Akka Streams
Declaring a Stream Topology
24
Declaring a Stream Topology
25
Declaring a Stream Topology
26
Declaring a Stream Topology
27
Declaring a Stream Topology
28
Declaring a Stream Topology
29
Declaring a Stream Topology
30
API Design
• goals:
• no magic
• supreme compositionality
• exhaustive model of bounded stream processing
• consequences:
• immutable and reusable stream blueprints
• explicit materialization step
31
http://doc.akka.io/docs/akka-stream-and-http-experimental/1.0-M2/stream-design.html
Declaring and Running a Stream
32
val upper = Source(Iterator from 0).take(10)
val lower = Source(1.second, 1.second, () => Tick)
val source = Source[(Int, Tick)]() { implicit b =>
val zip = Zip[Int, Tick]
val out = UndefinedSink[(Int, Tick)]
upper ~> zip.left ~> out
lower ~> zip.right
out
}
val flow = Flow[(Int, Tick)].map{ case (x, _) => s"tick $x" }
val sink = Sink.foreach(println)
val future = source.connect(flow).runWith(sink)
Declaring and Running a Stream
33
val upper = Source(Iterator from 0).take(10)
val lower = Source(1.second, 1.second, () => Tick)
val source = Source[(Int, Tick)]() { implicit b =>
val zip = Zip[Int, Tick]
val out = UndefinedSink[(Int, Tick)]
upper ~> zip.left ~> out
lower ~> zip.right
out
}
val flow = Flow[(Int, Tick)].map{ case (x, _) => s"tick $x" }
val sink = Sink.foreach(println)
val future = source.connect(flow).runWith(sink)
Declaring and Running a Stream
34
val upper = Source(Iterator from 0).take(10)
val lower = Source(1.second, 1.second, () => Tick)
val source = Source[(Int, Tick)]() { implicit b =>
val zip = Zip[Int, Tick]
val out = UndefinedSink[(Int, Tick)]
upper ~> zip.left ~> out
lower ~> zip.right
out
}
val flow = Flow[(Int, Tick)].map{ case (x, _) => s"tick $x" }
val sink = Sink.foreach(println)
val future = source.connect(flow).runWith(sink)
Materialization
• Akka Streams separate the what from the how
• declarative Source/Flow/Sink DSL to create blueprint
• FlowMaterializer turns this into running Actors
• this allows alternative materialization strategies
• optimization
• verification / validation
• cluster deployment
• only Akka Actors for now, but more to come!
35
Stream Sources
• org.reactivestreams.Publisher[T]
• org.reactivestreams.Subscriber[T]
• Iterator[T] / Iterable[T]
• Code block (function that produces Option[T])
• scala.concurrent.Future[T]
• TickSource
• ActorPublisher
• singleton / empty / failed
• … plus write your own (fully extensible)
36
Stream Sinks
• org.reactivestreams.Publisher[T]
• org.reactivestreams.Subscriber[T]
• ActorSubscriber
• scala.concurrent.Future[T]
• blackhole / foreach / fold / onComplete
• … or create your own
37
Linear Stream Transformations
• Deterministic (like for collections)
• map, filter, collect, grouped, drop, take, groupBy, …
• Time-Based
• takeWithin, dropWithin, groupedWithin, …
• Rate-Detached
• expand, conflate, buffer, …
• asynchronous
• mapAsync, mapAsyncUnordered, flatten, …
38
Nonlinear Stream Transformations
• Fan-In
• merge, concat, zip, …
• Fan-Out
• broadcast, route, balance, unzip, …
39
Akka HTTP
Why do we add an HTTP module?
• Akka is about building distributed applications
• distribution implies integration
• between internal (sub)systems

➜ akka-clusterbasedonakka-remote
• with external systems

➜ akka-http(linguafrancaoftheinternet)
41
Akka HTTP—The Origins: Spray.IO
• Fully embeddable HTTP stack based on Actors
• focused on HTTP integration (toolkit)
• server- and client-side
• seamlessly integrated with Akka
42
Spray.IO Features
• immutable, case class-based HTTP model
• fast, lightweight HTTP client and server
• powerful DSL for server-side API definition
• fully async & non-blocking, actor-friendly,

modular, testable
• based entirely on Scala & Akka Actors
43
Spray.IO Weaknesses
• handling of chunked requests is clunky, incomplete
• dealing with large message entities can be difficult
• some unintuitive corner-cases in routing DSL
• deep implicit argument chains in some cases hard
to debug
• missing features, foremost websocket support
44
Proxying Large Responses
45
Akka HTTP is Spray 2.0
• addressing the weaknesses, polishing the features
• Java API
• simplified module structure
• core improvement: fully stream-based
46
HTTP Stream Topology
47
The Application Stack
48
O/S-level network stack
Java NIO (JDK)
Akka IO
Akka HTTP Core
Akka HTTP
user-
level
Akka IO
• bridges Java NIO with Akka Actors or streams
• supports TCP, UDP and SSL
49
Akka HTTP Core
• implements HTTP/1.1 according to the RFCs
• based upon the TCP facilities of Akka IO
• exposed as freely reusable stream transformations
• low-level and extensible implementation of

HTTP model and spec
50
A Quick View of the HTTP Model
51
case class HttpRequest(
method: HttpMethod = HttpMethods.GET,
uri: Uri = Uri./,
headers: immutable.Seq[HttpHeader] = Nil,
entity: RequestEntity = HttpEntity.Empty,
protocol: HttpProtocol = HttpProtocols.`HTTP/1.1`
) extends HttpMessage
case class HttpResponse(
status: StatusCode = StatusCodes.OK,
headers: immutable.Seq[HttpHeader] = Nil,
entity: ResponseEntity = HttpEntity.Empty,
protocol: HttpProtocol = HttpProtocols.`HTTP/1.1`
) extends HttpMessage
Akka HTTP
• builds upon Akka HTTP Core
• adds (un)marshaling for HttpEntities
• adds (de)compression (gzip / deflate)
• high-level server-side routing DSL
• type-safe and flexible
• highly compositional building blocks (Directives)
• includes common behaviors pre-canned
52
Stream Pipelines
53
TCP SSL HTTP App
Requests
Responses
optional
(not yet)
A Simple Server Example
54
import Directives._
val binding = Http().bind("localhost", 8080)
binding startHandlingWith {
path("order" / HexIntNumber) { id =>
get {
complete(s"Received GET for order $id")
} ~
put {
complete((actor ? Put(id)).mapTo[String])
}
}
}
Summary
When can we have it?
• currently pre-release versions:
• reactive-streams 1.0.0-RC1
• Akka Streams & HTTP 1.0-M2
• still missing:
• Akka HTTP client features
• SSL integration
• websockets
• Akka Streams & HTTP 1.0 expected end of Feb’15
56
Resources
• https://github.com/akka/akka/tree/release-2.3-dev
• http://doc.akka.io/docs/akka-stream-and-http-
experimental/1.0-M2/
• akka-user mailing list
57
©Typesafe 2014 – All Rights Reserved

Weitere ähnliche Inhalte

Was ist angesagt?

How to manage large amounts of data with akka streams
How to manage large amounts of data with akka streamsHow to manage large amounts of data with akka streams
How to manage large amounts of data with akka streamsIgor Mielientiev
 
Practical Akka HTTP - introduction
Practical Akka HTTP - introductionPractical Akka HTTP - introduction
Practical Akka HTTP - introductionŁukasz Sowa
 
Reactive Streams 1.0 and Akka Streams
Reactive Streams 1.0 and Akka StreamsReactive Streams 1.0 and Akka Streams
Reactive Streams 1.0 and Akka StreamsDean Wampler
 
Build Real-Time Streaming ETL Pipelines With Akka Streams, Alpakka And Apache...
Build Real-Time Streaming ETL Pipelines With Akka Streams, Alpakka And Apache...Build Real-Time Streaming ETL Pipelines With Akka Streams, Alpakka And Apache...
Build Real-Time Streaming ETL Pipelines With Akka Streams, Alpakka And Apache...Lightbend
 
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Kafka
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & KafkaBack-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Kafka
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & KafkaAkara Sucharitakul
 
Asynchronous Orchestration DSL on squbs
Asynchronous Orchestration DSL on squbsAsynchronous Orchestration DSL on squbs
Asynchronous Orchestration DSL on squbsAnil Gursel
 
Real-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaReal-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaJoe Stein
 
Service Stampede: Surviving a Thousand Services
Service Stampede: Surviving a Thousand ServicesService Stampede: Surviving a Thousand Services
Service Stampede: Surviving a Thousand ServicesAnil Gursel
 
Understanding Akka Streams, Back Pressure, and Asynchronous Architectures
Understanding Akka Streams, Back Pressure, and Asynchronous ArchitecturesUnderstanding Akka Streams, Back Pressure, and Asynchronous Architectures
Understanding Akka Streams, Back Pressure, and Asynchronous ArchitecturesLightbend
 
Scala usergroup stockholm - reactive integrations with akka streams
Scala usergroup stockholm - reactive integrations with akka streamsScala usergroup stockholm - reactive integrations with akka streams
Scala usergroup stockholm - reactive integrations with akka streamsJohan Andrén
 
Streaming all the things with akka streams
Streaming all the things with akka streams   Streaming all the things with akka streams
Streaming all the things with akka streams Johan Andrén
 
VJUG24 - Reactive Integrations with Akka Streams
VJUG24  - Reactive Integrations with Akka StreamsVJUG24  - Reactive Integrations with Akka Streams
VJUG24 - Reactive Integrations with Akka StreamsJohan Andrén
 
Specs2 whirlwind tour at Scaladays 2014
Specs2 whirlwind tour at Scaladays 2014Specs2 whirlwind tour at Scaladays 2014
Specs2 whirlwind tour at Scaladays 2014Eric Torreborre
 
Building scalable rest service using Akka HTTP
Building scalable rest service using Akka HTTPBuilding scalable rest service using Akka HTTP
Building scalable rest service using Akka HTTPdatamantra
 
Reactive integrations with Akka Streams
Reactive integrations with Akka StreamsReactive integrations with Akka Streams
Reactive integrations with Akka StreamsKonrad Malawski
 
Akka Http , Routes, Streams with Scala
Akka Http , Routes, Streams with ScalaAkka Http , Routes, Streams with Scala
Akka Http , Routes, Streams with ScalaJerry Kuru
 
Reactive Streams: Handling Data-Flow the Reactive Way
Reactive Streams: Handling Data-Flow the Reactive WayReactive Streams: Handling Data-Flow the Reactive Way
Reactive Streams: Handling Data-Flow the Reactive WayRoland Kuhn
 
Building Distributed Systems in Scala
Building Distributed Systems in ScalaBuilding Distributed Systems in Scala
Building Distributed Systems in ScalaAlex Payne
 
A dive into akka streams: from the basics to a real-world scenario
A dive into akka streams: from the basics to a real-world scenarioA dive into akka streams: from the basics to a real-world scenario
A dive into akka streams: from the basics to a real-world scenarioGioia Ballin
 

Was ist angesagt? (20)

How to manage large amounts of data with akka streams
How to manage large amounts of data with akka streamsHow to manage large amounts of data with akka streams
How to manage large amounts of data with akka streams
 
Practical Akka HTTP - introduction
Practical Akka HTTP - introductionPractical Akka HTTP - introduction
Practical Akka HTTP - introduction
 
Akka streams
Akka streamsAkka streams
Akka streams
 
Reactive Streams 1.0 and Akka Streams
Reactive Streams 1.0 and Akka StreamsReactive Streams 1.0 and Akka Streams
Reactive Streams 1.0 and Akka Streams
 
Build Real-Time Streaming ETL Pipelines With Akka Streams, Alpakka And Apache...
Build Real-Time Streaming ETL Pipelines With Akka Streams, Alpakka And Apache...Build Real-Time Streaming ETL Pipelines With Akka Streams, Alpakka And Apache...
Build Real-Time Streaming ETL Pipelines With Akka Streams, Alpakka And Apache...
 
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Kafka
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & KafkaBack-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Kafka
Back-Pressure in Action: Handling High-Burst Workloads with Akka Streams & Kafka
 
Asynchronous Orchestration DSL on squbs
Asynchronous Orchestration DSL on squbsAsynchronous Orchestration DSL on squbs
Asynchronous Orchestration DSL on squbs
 
Real-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaReal-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache Kafka
 
Service Stampede: Surviving a Thousand Services
Service Stampede: Surviving a Thousand ServicesService Stampede: Surviving a Thousand Services
Service Stampede: Surviving a Thousand Services
 
Understanding Akka Streams, Back Pressure, and Asynchronous Architectures
Understanding Akka Streams, Back Pressure, and Asynchronous ArchitecturesUnderstanding Akka Streams, Back Pressure, and Asynchronous Architectures
Understanding Akka Streams, Back Pressure, and Asynchronous Architectures
 
Scala usergroup stockholm - reactive integrations with akka streams
Scala usergroup stockholm - reactive integrations with akka streamsScala usergroup stockholm - reactive integrations with akka streams
Scala usergroup stockholm - reactive integrations with akka streams
 
Streaming all the things with akka streams
Streaming all the things with akka streams   Streaming all the things with akka streams
Streaming all the things with akka streams
 
VJUG24 - Reactive Integrations with Akka Streams
VJUG24  - Reactive Integrations with Akka StreamsVJUG24  - Reactive Integrations with Akka Streams
VJUG24 - Reactive Integrations with Akka Streams
 
Specs2 whirlwind tour at Scaladays 2014
Specs2 whirlwind tour at Scaladays 2014Specs2 whirlwind tour at Scaladays 2014
Specs2 whirlwind tour at Scaladays 2014
 
Building scalable rest service using Akka HTTP
Building scalable rest service using Akka HTTPBuilding scalable rest service using Akka HTTP
Building scalable rest service using Akka HTTP
 
Reactive integrations with Akka Streams
Reactive integrations with Akka StreamsReactive integrations with Akka Streams
Reactive integrations with Akka Streams
 
Akka Http , Routes, Streams with Scala
Akka Http , Routes, Streams with ScalaAkka Http , Routes, Streams with Scala
Akka Http , Routes, Streams with Scala
 
Reactive Streams: Handling Data-Flow the Reactive Way
Reactive Streams: Handling Data-Flow the Reactive WayReactive Streams: Handling Data-Flow the Reactive Way
Reactive Streams: Handling Data-Flow the Reactive Way
 
Building Distributed Systems in Scala
Building Distributed Systems in ScalaBuilding Distributed Systems in Scala
Building Distributed Systems in Scala
 
A dive into akka streams: from the basics to a real-world scenario
A dive into akka streams: from the basics to a real-world scenarioA dive into akka streams: from the basics to a real-world scenario
A dive into akka streams: from the basics to a real-world scenario
 

Ähnlich wie Akka Streams and HTTP

Reactive Streams 1.0.0 and Why You Should Care (webinar)
Reactive Streams 1.0.0 and Why You Should Care (webinar)Reactive Streams 1.0.0 and Why You Should Care (webinar)
Reactive Streams 1.0.0 and Why You Should Care (webinar)Legacy Typesafe (now Lightbend)
 
Mirco Dotta - Akka Streams
Mirco Dotta - Akka StreamsMirco Dotta - Akka Streams
Mirco Dotta - Akka StreamsScala Italy
 
Writing Asynchronous Programs with Scala & Akka
Writing Asynchronous Programs with Scala & AkkaWriting Asynchronous Programs with Scala & Akka
Writing Asynchronous Programs with Scala & AkkaYardena Meymann
 
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0Legacy Typesafe (now Lightbend)
 
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lightbend
 
Springone2gx 2014 Reactive Streams and Reactor
Springone2gx 2014 Reactive Streams and ReactorSpringone2gx 2014 Reactive Streams and Reactor
Springone2gx 2014 Reactive Streams and ReactorStéphane Maldini
 
PSUG #52 Dataflow and simplified reactive programming with Akka-streams
PSUG #52 Dataflow and simplified reactive programming with Akka-streamsPSUG #52 Dataflow and simplified reactive programming with Akka-streams
PSUG #52 Dataflow and simplified reactive programming with Akka-streamsStephane Manciot
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkFabian Hueske
 
Stream processing from single node to a cluster
Stream processing from single node to a clusterStream processing from single node to a cluster
Stream processing from single node to a clusterGal Marder
 
Akka-demy (a.k.a. How to build stateful distributed systems) I/II
 Akka-demy (a.k.a. How to build stateful distributed systems) I/II Akka-demy (a.k.a. How to build stateful distributed systems) I/II
Akka-demy (a.k.a. How to build stateful distributed systems) I/IIPeter Csala
 
API Testing. Streamline your testing process.
API Testing. Streamline your testing process.API Testing. Streamline your testing process.
API Testing. Streamline your testing process.Andrey Oleynik
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkDataWorks Summit
 
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Apex
 
«Scrapy internals» Александр Сибиряков, Scrapinghub
«Scrapy internals» Александр Сибиряков, Scrapinghub«Scrapy internals» Александр Сибиряков, Scrapinghub
«Scrapy internals» Александр Сибиряков, Scrapinghubit-people
 
From Batch to Streaming ET(L) with Apache Apex
From Batch to Streaming ET(L) with Apache ApexFrom Batch to Streaming ET(L) with Apache Apex
From Batch to Streaming ET(L) with Apache ApexDataWorks Summit
 
From Batch to Streaming with Apache Apex Dataworks Summit 2017
From Batch to Streaming with Apache Apex Dataworks Summit 2017From Batch to Streaming with Apache Apex Dataworks Summit 2017
From Batch to Streaming with Apache Apex Dataworks Summit 2017Apache Apex
 

Ähnlich wie Akka Streams and HTTP (20)

Reactive Streams 1.0.0 and Why You Should Care (webinar)
Reactive Streams 1.0.0 and Why You Should Care (webinar)Reactive Streams 1.0.0 and Why You Should Care (webinar)
Reactive Streams 1.0.0 and Why You Should Care (webinar)
 
Mirco Dotta - Akka Streams
Mirco Dotta - Akka StreamsMirco Dotta - Akka Streams
Mirco Dotta - Akka Streams
 
Writing Asynchronous Programs with Scala & Akka
Writing Asynchronous Programs with Scala & AkkaWriting Asynchronous Programs with Scala & Akka
Writing Asynchronous Programs with Scala & Akka
 
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
 
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
 
Springone2gx 2014 Reactive Streams and Reactor
Springone2gx 2014 Reactive Streams and ReactorSpringone2gx 2014 Reactive Streams and Reactor
Springone2gx 2014 Reactive Streams and Reactor
 
PSUG #52 Dataflow and simplified reactive programming with Akka-streams
PSUG #52 Dataflow and simplified reactive programming with Akka-streamsPSUG #52 Dataflow and simplified reactive programming with Akka-streams
PSUG #52 Dataflow and simplified reactive programming with Akka-streams
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache Flink
 
Stream processing from single node to a cluster
Stream processing from single node to a clusterStream processing from single node to a cluster
Stream processing from single node to a cluster
 
Akka-demy (a.k.a. How to build stateful distributed systems) I/II
 Akka-demy (a.k.a. How to build stateful distributed systems) I/II Akka-demy (a.k.a. How to build stateful distributed systems) I/II
Akka-demy (a.k.a. How to build stateful distributed systems) I/II
 
API Testing. Streamline your testing process.
API Testing. Streamline your testing process.API Testing. Streamline your testing process.
API Testing. Streamline your testing process.
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
Reactive Spring 5
Reactive Spring 5Reactive Spring 5
Reactive Spring 5
 
Reactive Streams
Reactive StreamsReactive Streams
Reactive Streams
 
Intro to Akka Streams
Intro to Akka StreamsIntro to Akka Streams
Intro to Akka Streams
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache Flink
 
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
 
«Scrapy internals» Александр Сибиряков, Scrapinghub
«Scrapy internals» Александр Сибиряков, Scrapinghub«Scrapy internals» Александр Сибиряков, Scrapinghub
«Scrapy internals» Александр Сибиряков, Scrapinghub
 
From Batch to Streaming ET(L) with Apache Apex
From Batch to Streaming ET(L) with Apache ApexFrom Batch to Streaming ET(L) with Apache Apex
From Batch to Streaming ET(L) with Apache Apex
 
From Batch to Streaming with Apache Apex Dataworks Summit 2017
From Batch to Streaming with Apache Apex Dataworks Summit 2017From Batch to Streaming with Apache Apex Dataworks Summit 2017
From Batch to Streaming with Apache Apex Dataworks Summit 2017
 

Mehr von Roland Kuhn

Taming Distribution: Formal Protocols for Akka Typed
Taming Distribution: Formal Protocols for Akka TypedTaming Distribution: Formal Protocols for Akka Typed
Taming Distribution: Formal Protocols for Akka TypedRoland Kuhn
 
Distributed systems vs compositionality
Distributed systems vs compositionalityDistributed systems vs compositionality
Distributed systems vs compositionalityRoland Kuhn
 
Reactive Design Patterns — J on the Beach
Reactive Design Patterns — J on the BeachReactive Design Patterns — J on the Beach
Reactive Design Patterns — J on the BeachRoland Kuhn
 
The Newest in Session Types
The Newest in Session TypesThe Newest in Session Types
The Newest in Session TypesRoland Kuhn
 
Akka Typed — between Session Types and the Actor Model
Akka Typed — between Session Types and the Actor ModelAkka Typed — between Session Types and the Actor Model
Akka Typed — between Session Types and the Actor ModelRoland Kuhn
 
Project Gålbma – Actors vs Types
Project Gålbma – Actors vs TypesProject Gålbma – Actors vs Types
Project Gålbma – Actors vs TypesRoland Kuhn
 
Akka and AngularJS – Reactive Applications in Practice
Akka and AngularJS – Reactive Applications in PracticeAkka and AngularJS – Reactive Applications in Practice
Akka and AngularJS – Reactive Applications in PracticeRoland Kuhn
 
Go Reactive: Blueprint for Future Applications
Go Reactive: Blueprint for Future ApplicationsGo Reactive: Blueprint for Future Applications
Go Reactive: Blueprint for Future ApplicationsRoland Kuhn
 
Akka cluster overview at 010dev
Akka cluster overview at 010devAkka cluster overview at 010dev
Akka cluster overview at 010devRoland Kuhn
 
Akka typed-channels
Akka typed-channelsAkka typed-channels
Akka typed-channelsRoland Kuhn
 

Mehr von Roland Kuhn (10)

Taming Distribution: Formal Protocols for Akka Typed
Taming Distribution: Formal Protocols for Akka TypedTaming Distribution: Formal Protocols for Akka Typed
Taming Distribution: Formal Protocols for Akka Typed
 
Distributed systems vs compositionality
Distributed systems vs compositionalityDistributed systems vs compositionality
Distributed systems vs compositionality
 
Reactive Design Patterns — J on the Beach
Reactive Design Patterns — J on the BeachReactive Design Patterns — J on the Beach
Reactive Design Patterns — J on the Beach
 
The Newest in Session Types
The Newest in Session TypesThe Newest in Session Types
The Newest in Session Types
 
Akka Typed — between Session Types and the Actor Model
Akka Typed — between Session Types and the Actor ModelAkka Typed — between Session Types and the Actor Model
Akka Typed — between Session Types and the Actor Model
 
Project Gålbma – Actors vs Types
Project Gålbma – Actors vs TypesProject Gålbma – Actors vs Types
Project Gålbma – Actors vs Types
 
Akka and AngularJS – Reactive Applications in Practice
Akka and AngularJS – Reactive Applications in PracticeAkka and AngularJS – Reactive Applications in Practice
Akka and AngularJS – Reactive Applications in Practice
 
Go Reactive: Blueprint for Future Applications
Go Reactive: Blueprint for Future ApplicationsGo Reactive: Blueprint for Future Applications
Go Reactive: Blueprint for Future Applications
 
Akka cluster overview at 010dev
Akka cluster overview at 010devAkka cluster overview at 010dev
Akka cluster overview at 010dev
 
Akka typed-channels
Akka typed-channelsAkka typed-channels
Akka typed-channels
 

Kürzlich hochgeladen

Strategies for using alternative queries to mitigate zero results
Strategies for using alternative queries to mitigate zero resultsStrategies for using alternative queries to mitigate zero results
Strategies for using alternative queries to mitigate zero resultsJean Silva
 
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...OnePlan Solutions
 
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full RecordingOpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full RecordingShane Coughlan
 
How to submit a standout Adobe Champion Application
How to submit a standout Adobe Champion ApplicationHow to submit a standout Adobe Champion Application
How to submit a standout Adobe Champion ApplicationBradBedford3
 
Precise and Complete Requirements? An Elusive Goal
Precise and Complete Requirements? An Elusive GoalPrecise and Complete Requirements? An Elusive Goal
Precise and Complete Requirements? An Elusive GoalLionel Briand
 
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...OnePlan Solutions
 
Osi security architecture in network.pptx
Osi security architecture in network.pptxOsi security architecture in network.pptx
Osi security architecture in network.pptxVinzoCenzo
 
Post Quantum Cryptography – The Impact on Identity
Post Quantum Cryptography – The Impact on IdentityPost Quantum Cryptography – The Impact on Identity
Post Quantum Cryptography – The Impact on Identityteam-WIBU
 
eSoftTools IMAP Backup Software and migration tools
eSoftTools IMAP Backup Software and migration toolseSoftTools IMAP Backup Software and migration tools
eSoftTools IMAP Backup Software and migration toolsosttopstonverter
 
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...Bert Jan Schrijver
 
UI5ers live - Custom Controls wrapping 3rd-party libs.pptx
UI5ers live - Custom Controls wrapping 3rd-party libs.pptxUI5ers live - Custom Controls wrapping 3rd-party libs.pptx
UI5ers live - Custom Controls wrapping 3rd-party libs.pptxAndreas Kunz
 
SAM Training Session - How to use EXCEL ?
SAM Training Session - How to use EXCEL ?SAM Training Session - How to use EXCEL ?
SAM Training Session - How to use EXCEL ?Alexandre Beguel
 
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingOpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingShane Coughlan
 
SensoDat: Simulation-based Sensor Dataset of Self-driving Cars
SensoDat: Simulation-based Sensor Dataset of Self-driving CarsSensoDat: Simulation-based Sensor Dataset of Self-driving Cars
SensoDat: Simulation-based Sensor Dataset of Self-driving CarsChristian Birchler
 
VictoriaMetrics Q1 Meet Up '24 - Community & News Update
VictoriaMetrics Q1 Meet Up '24 - Community & News UpdateVictoriaMetrics Q1 Meet Up '24 - Community & News Update
VictoriaMetrics Q1 Meet Up '24 - Community & News UpdateVictoriaMetrics
 
Not a Kubernetes fan? The state of PaaS in 2024
Not a Kubernetes fan? The state of PaaS in 2024Not a Kubernetes fan? The state of PaaS in 2024
Not a Kubernetes fan? The state of PaaS in 2024Anthony Dahanne
 
Large Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLarge Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLionel Briand
 
Sending Calendar Invites on SES and Calendarsnack.pdf
Sending Calendar Invites on SES and Calendarsnack.pdfSending Calendar Invites on SES and Calendarsnack.pdf
Sending Calendar Invites on SES and Calendarsnack.pdf31events.com
 
SoftTeco - Software Development Company Profile
SoftTeco - Software Development Company ProfileSoftTeco - Software Development Company Profile
SoftTeco - Software Development Company Profileakrivarotava
 
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesAmazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesKrzysztofKkol1
 

Kürzlich hochgeladen (20)

Strategies for using alternative queries to mitigate zero results
Strategies for using alternative queries to mitigate zero resultsStrategies for using alternative queries to mitigate zero results
Strategies for using alternative queries to mitigate zero results
 
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
 
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full RecordingOpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
OpenChain Education Work Group Monthly Meeting - 2024-04-10 - Full Recording
 
How to submit a standout Adobe Champion Application
How to submit a standout Adobe Champion ApplicationHow to submit a standout Adobe Champion Application
How to submit a standout Adobe Champion Application
 
Precise and Complete Requirements? An Elusive Goal
Precise and Complete Requirements? An Elusive GoalPrecise and Complete Requirements? An Elusive Goal
Precise and Complete Requirements? An Elusive Goal
 
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
 
Osi security architecture in network.pptx
Osi security architecture in network.pptxOsi security architecture in network.pptx
Osi security architecture in network.pptx
 
Post Quantum Cryptography – The Impact on Identity
Post Quantum Cryptography – The Impact on IdentityPost Quantum Cryptography – The Impact on Identity
Post Quantum Cryptography – The Impact on Identity
 
eSoftTools IMAP Backup Software and migration tools
eSoftTools IMAP Backup Software and migration toolseSoftTools IMAP Backup Software and migration tools
eSoftTools IMAP Backup Software and migration tools
 
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
JavaLand 2024 - Going serverless with Quarkus GraalVM native images and AWS L...
 
UI5ers live - Custom Controls wrapping 3rd-party libs.pptx
UI5ers live - Custom Controls wrapping 3rd-party libs.pptxUI5ers live - Custom Controls wrapping 3rd-party libs.pptx
UI5ers live - Custom Controls wrapping 3rd-party libs.pptx
 
SAM Training Session - How to use EXCEL ?
SAM Training Session - How to use EXCEL ?SAM Training Session - How to use EXCEL ?
SAM Training Session - How to use EXCEL ?
 
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingOpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
 
SensoDat: Simulation-based Sensor Dataset of Self-driving Cars
SensoDat: Simulation-based Sensor Dataset of Self-driving CarsSensoDat: Simulation-based Sensor Dataset of Self-driving Cars
SensoDat: Simulation-based Sensor Dataset of Self-driving Cars
 
VictoriaMetrics Q1 Meet Up '24 - Community & News Update
VictoriaMetrics Q1 Meet Up '24 - Community & News UpdateVictoriaMetrics Q1 Meet Up '24 - Community & News Update
VictoriaMetrics Q1 Meet Up '24 - Community & News Update
 
Not a Kubernetes fan? The state of PaaS in 2024
Not a Kubernetes fan? The state of PaaS in 2024Not a Kubernetes fan? The state of PaaS in 2024
Not a Kubernetes fan? The state of PaaS in 2024
 
Large Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLarge Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and Repair
 
Sending Calendar Invites on SES and Calendarsnack.pdf
Sending Calendar Invites on SES and Calendarsnack.pdfSending Calendar Invites on SES and Calendarsnack.pdf
Sending Calendar Invites on SES and Calendarsnack.pdf
 
SoftTeco - Software Development Company Profile
SoftTeco - Software Development Company ProfileSoftTeco - Software Development Company Profile
SoftTeco - Software Development Company Profile
 
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesAmazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
 

Akka Streams and HTTP

  • 1. Akka Streams & HTTP Dr. Roland Kuhn @rolandkuhn — Akka Tech Lead
  • 2. Streams Occur Naturally • traditionally: • file input/output • network connections • more modern: • processing big data with finite memory • real-time data processing (CEP) • serving numerous clients simultaneously with bounded resources (IoT, streaming HTTP APIs) 2
  • 3. 3 What is a Stream? • ephemeral, time-dependent sequence of elements • possibly unbounded in length • therefore focusing on transformations «You cannot step twice into the same stream. For as you are stepping in, other waters are ever flowing on to you.» — Heraclitus
  • 6. Reactive means Message Flow 6 37% discount with code kuhnco
  • 7. Distributed vs. Static Typing • in general a very tough problem • independent entities evolving concurrently • promising progress on Project Gålbma • in many specific cases no type evolution necessary • letting uniform messages flow in safe conduits • making streams even more interesting 7
  • 8. Possible Solutions • the Traditional way: blocking calls 8
  • 9. Possible Solutions • the Push way: buffering and/or dropping
 (optimized for fast consumer) 9
  • 10. Possible Solutions • the Pull way: poor performance
 (optimized for fast producer) 10
  • 11. Possible Solutions • the Reactive way:
 non-blocking & non-dropping & bounded 11
  • 13. Origin and motivation • all participants face the same basic problem • all are building tools for their community • a common solution benefits everybody • interoperability to make best use of efforts • propose to include in future JDK 13
  • 14. Goals • minimal interfaces—essentials only • rigorous specification of semantics • TCK for verification of implementation • complete freedom for many idiomatic APIs • specification should be efficiently implementable 14
  • 15. Reactive Streams • asynchronous & non-blocking • flow of data • flow of demand • minimal coordination and contention • message passing allows for distribution across • applications, nodes, CPUs, threads, actors 15
  • 16. A Data Market using Supply & Demand • data elements flow downstream • demand flows upstream • data elements flow only when there is demand • data in flight is bounded by signaled demand • recipient is in control of maximal incoming data rate 16 Publisher Subscriber data demand
  • 17. Reactive Push–Pull • “push”—when consumer is faster • “pull”—when producer is faster • switches automatically between these • batching demand allows batching data 17 Publisher Subscriber data demand
  • 18. Explicit Demand: One-to-many 18 demand data Splitting the data means merging the demand
  • 19. Explicit Demand: Many-to-one 19 Merging the data means splitting the demand
  • 20. Back-Pressure is Contagious • C is slow • B must slow down • A must slow down 20 CA B
  • 21. • TCP for example has it built-in Back-Pressure can be Propagated 21 CA B networkhosts
  • 22. The Meat 22 trait Publisher[T] { def subscribe(sub: Subscriber[T]): Unit } trait Subscription { def request(n: Long): Unit def cancel(): Unit } trait Subscriber[T] { def onSubscribe(s: Subscription): Unit def onNext(e: T): Unit def onError(t: Throwable): Unit def onComplete(): Unit } Not user-level API Intended as SPI for interop
  • 24. Declaring a Stream Topology 24
  • 25. Declaring a Stream Topology 25
  • 26. Declaring a Stream Topology 26
  • 27. Declaring a Stream Topology 27
  • 28. Declaring a Stream Topology 28
  • 29. Declaring a Stream Topology 29
  • 30. Declaring a Stream Topology 30
  • 31. API Design • goals: • no magic • supreme compositionality • exhaustive model of bounded stream processing • consequences: • immutable and reusable stream blueprints • explicit materialization step 31 http://doc.akka.io/docs/akka-stream-and-http-experimental/1.0-M2/stream-design.html
  • 32. Declaring and Running a Stream 32 val upper = Source(Iterator from 0).take(10) val lower = Source(1.second, 1.second, () => Tick) val source = Source[(Int, Tick)]() { implicit b => val zip = Zip[Int, Tick] val out = UndefinedSink[(Int, Tick)] upper ~> zip.left ~> out lower ~> zip.right out } val flow = Flow[(Int, Tick)].map{ case (x, _) => s"tick $x" } val sink = Sink.foreach(println) val future = source.connect(flow).runWith(sink)
  • 33. Declaring and Running a Stream 33 val upper = Source(Iterator from 0).take(10) val lower = Source(1.second, 1.second, () => Tick) val source = Source[(Int, Tick)]() { implicit b => val zip = Zip[Int, Tick] val out = UndefinedSink[(Int, Tick)] upper ~> zip.left ~> out lower ~> zip.right out } val flow = Flow[(Int, Tick)].map{ case (x, _) => s"tick $x" } val sink = Sink.foreach(println) val future = source.connect(flow).runWith(sink)
  • 34. Declaring and Running a Stream 34 val upper = Source(Iterator from 0).take(10) val lower = Source(1.second, 1.second, () => Tick) val source = Source[(Int, Tick)]() { implicit b => val zip = Zip[Int, Tick] val out = UndefinedSink[(Int, Tick)] upper ~> zip.left ~> out lower ~> zip.right out } val flow = Flow[(Int, Tick)].map{ case (x, _) => s"tick $x" } val sink = Sink.foreach(println) val future = source.connect(flow).runWith(sink)
  • 35. Materialization • Akka Streams separate the what from the how • declarative Source/Flow/Sink DSL to create blueprint • FlowMaterializer turns this into running Actors • this allows alternative materialization strategies • optimization • verification / validation • cluster deployment • only Akka Actors for now, but more to come! 35
  • 36. Stream Sources • org.reactivestreams.Publisher[T] • org.reactivestreams.Subscriber[T] • Iterator[T] / Iterable[T] • Code block (function that produces Option[T]) • scala.concurrent.Future[T] • TickSource • ActorPublisher • singleton / empty / failed • … plus write your own (fully extensible) 36
  • 37. Stream Sinks • org.reactivestreams.Publisher[T] • org.reactivestreams.Subscriber[T] • ActorSubscriber • scala.concurrent.Future[T] • blackhole / foreach / fold / onComplete • … or create your own 37
  • 38. Linear Stream Transformations • Deterministic (like for collections) • map, filter, collect, grouped, drop, take, groupBy, … • Time-Based • takeWithin, dropWithin, groupedWithin, … • Rate-Detached • expand, conflate, buffer, … • asynchronous • mapAsync, mapAsyncUnordered, flatten, … 38
  • 39. Nonlinear Stream Transformations • Fan-In • merge, concat, zip, … • Fan-Out • broadcast, route, balance, unzip, … 39
  • 41. Why do we add an HTTP module? • Akka is about building distributed applications • distribution implies integration • between internal (sub)systems
 ➜ akka-clusterbasedonakka-remote • with external systems
 ➜ akka-http(linguafrancaoftheinternet) 41
  • 42. Akka HTTP—The Origins: Spray.IO • Fully embeddable HTTP stack based on Actors • focused on HTTP integration (toolkit) • server- and client-side • seamlessly integrated with Akka 42
  • 43. Spray.IO Features • immutable, case class-based HTTP model • fast, lightweight HTTP client and server • powerful DSL for server-side API definition • fully async & non-blocking, actor-friendly,
 modular, testable • based entirely on Scala & Akka Actors 43
  • 44. Spray.IO Weaknesses • handling of chunked requests is clunky, incomplete • dealing with large message entities can be difficult • some unintuitive corner-cases in routing DSL • deep implicit argument chains in some cases hard to debug • missing features, foremost websocket support 44
  • 46. Akka HTTP is Spray 2.0 • addressing the weaknesses, polishing the features • Java API • simplified module structure • core improvement: fully stream-based 46
  • 48. The Application Stack 48 O/S-level network stack Java NIO (JDK) Akka IO Akka HTTP Core Akka HTTP user- level
  • 49. Akka IO • bridges Java NIO with Akka Actors or streams • supports TCP, UDP and SSL 49
  • 50. Akka HTTP Core • implements HTTP/1.1 according to the RFCs • based upon the TCP facilities of Akka IO • exposed as freely reusable stream transformations • low-level and extensible implementation of
 HTTP model and spec 50
  • 51. A Quick View of the HTTP Model 51 case class HttpRequest( method: HttpMethod = HttpMethods.GET, uri: Uri = Uri./, headers: immutable.Seq[HttpHeader] = Nil, entity: RequestEntity = HttpEntity.Empty, protocol: HttpProtocol = HttpProtocols.`HTTP/1.1` ) extends HttpMessage case class HttpResponse( status: StatusCode = StatusCodes.OK, headers: immutable.Seq[HttpHeader] = Nil, entity: ResponseEntity = HttpEntity.Empty, protocol: HttpProtocol = HttpProtocols.`HTTP/1.1` ) extends HttpMessage
  • 52. Akka HTTP • builds upon Akka HTTP Core • adds (un)marshaling for HttpEntities • adds (de)compression (gzip / deflate) • high-level server-side routing DSL • type-safe and flexible • highly compositional building blocks (Directives) • includes common behaviors pre-canned 52
  • 53. Stream Pipelines 53 TCP SSL HTTP App Requests Responses optional (not yet)
  • 54. A Simple Server Example 54 import Directives._ val binding = Http().bind("localhost", 8080) binding startHandlingWith { path("order" / HexIntNumber) { id => get { complete(s"Received GET for order $id") } ~ put { complete((actor ? Put(id)).mapTo[String]) } } }
  • 56. When can we have it? • currently pre-release versions: • reactive-streams 1.0.0-RC1 • Akka Streams & HTTP 1.0-M2 • still missing: • Akka HTTP client features • SSL integration • websockets • Akka Streams & HTTP 1.0 expected end of Feb’15 56
  • 58. ©Typesafe 2014 – All Rights Reserved