├── .gitignore ├── README.md ├── build.sbt ├── project ├── build.properties └── plugins.sbt └── src ├── main ├── resources │ ├── application.conf │ └── logback.groovy └── scala │ └── com │ └── experiments │ └── calvin │ ├── BackpressuredActor.scala │ ├── DetailedMessage.scala │ ├── ServerMain.scala │ ├── WebsocketStreamsMain.scala │ ├── chunked │ └── http │ │ └── ChunkedStreamingRoutes.scala │ └── ws │ ├── ChatRoom.scala │ ├── ConnectedUser.scala │ └── WebSocketRoutes.scala └── test └── scala └── com └── experiments └── calvin └── ChunkedStreamingSpec.scala /.gitignore: -------------------------------------------------------------------------------- 1 | 2 | # Created by https://www.gitignore.io/api/intellij,scala 3 | 4 | ### Intellij ### 5 | # Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio and Webstorm 6 | # Reference: https://intellij-support.jetbrains.com/hc/en-us/articles/206544839 7 | 8 | # User-specific stuff: 9 | .idea/workspace.xml 10 | .idea/tasks.xml 11 | .idea/dictionaries 12 | .idea/vcs.xml 13 | .idea/jsLibraryMappings.xml 14 | 15 | # Sensitive or high-churn files: 16 | .idea/dataSources.ids 17 | .idea/dataSources.xml 18 | .idea/dataSources.local.xml 19 | .idea/sqlDataSources.xml 20 | .idea/dynamic.xml 21 | .idea/uiDesigner.xml 22 | 23 | # Gradle: 24 | .idea/gradle.xml 25 | .idea/libraries 26 | 27 | # Mongo Explorer plugin: 28 | .idea/mongoSettings.xml 29 | 30 | ## File-based project format: 31 | *.iws 32 | 33 | ## Plugin-specific files: 34 | 35 | # IntelliJ 36 | /out/ 37 | 38 | # mpeltonen/sbt-idea plugin 39 | .idea_modules/ 40 | 41 | # JIRA plugin 42 | atlassian-ide-plugin.xml 43 | 44 | # Crashlytics plugin (for Android Studio and IntelliJ) 45 | com_crashlytics_export_strings.xml 46 | crashlytics.properties 47 | crashlytics-build.properties 48 | fabric.properties 49 | 50 | ### Intellij Patch ### 51 | # Comment Reason: https://github.com/joeblau/gitignore.io/issues/186#issuecomment-215987721 52 | 53 | # *.iml 54 | # modules.xml 55 | # .idea/misc.xml 56 | # *.ipr 57 | 58 | 59 | ### Scala ### 60 | *.class 61 | *.log 62 | 63 | # sbt specific 64 | .cache 65 | .history 66 | .lib/ 67 | dist/* 68 | target/ 69 | lib_managed/ 70 | src_managed/ 71 | project/boot/ 72 | project/plugins/project/ 73 | 74 | # Scala-IDE specific 75 | .scala_dependencies 76 | .worksheet 77 | 78 | # ENSIME specific 79 | .ensime_cache/ 80 | .ensime -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Examples using Akka HTTP with Streaming 2 | A list of examples that involve streaming with Akka Streams and used together with Akka HTTP. 3 | The Akka-HTTP specific details are isolated into the following traits: 4 | 5 | - [Chunked Streaming Routes](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala) 6 | - [WebSocket Routes](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/ws/WebSocketRoutes.scala) 7 | 8 | The initialization logic to start the Akka HTTP server is kept in [ServerMain](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/ServerMain.scala). 9 | 10 | Focusing on HTTP Chunked Streaming Routes: 11 | 12 | - [`streaming-text`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala#L33) - This uses predefined Sources and throttling in order to give the user a chunked response in a controlled manner 13 | - [`actor-text`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala#L50) - This is more involved, we define an [Actor that respects Akka Streams backpressure](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/BackpressuredActor.scala) and then create a Source from this. 14 | The Akka Scheduler constantly tells the Actor to emit messages into the Stream that the user sees. This logic is placed within the Actor. If you 15 | use a web browser and stop the streaming response then you will cancel the stream and shut down the actor. Feel free to open this up on multiple 16 | browsers since we allocate an actor per request 17 | - [`alt-actor-text`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala#L60) - This is similar to `actor-text` except it uses `mapMaterializedValue` 18 | to access the materialized ActorRef and schedules messages to be sent constantly (in addition to the scheduler sending messages inside the Actor) 19 | - [`live-actor-text`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala#L74) - This is slightly different from the other `actor` endpoints. 20 | It creates a live actor and places it into a Publisher and creates a Source from this. We publish messages in the same way as the previous examples 21 | - [`streaming-json`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala#L100) - This is an example of a JSON streaming route. We define a JSON Marshaller for a case class [here](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/DetailedMessage.scala) and we add a few imports for [streaming support](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala#L92) in order to have streaming JSON. You can customize the separators as well. 22 | - [`consuming-streaming-json`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala#L112) - This is an example of an endpoint that consumes a Streaming JSON request. This also relies on having JSON Streaming support. 23 | - [`streaming-sse-json`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala#L125) - This is an example of Streaming JSON using [Server Sent Events](http://www.html5rocks.com/en/tutorials/eventsource/basics/) with the help of Heiko Seeberger's [Akka-SSE](https://github.com/hseeberger/akka-sse) library. SSEs have better integration with modern day browsers. 24 | 25 | Focusing on the WebSocket Routes: 26 | 27 | - [`ws-simple`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/ws/WebSocketRoutes.scala#L23) - This can be accessed via `ws://localhost:9000/ws-simple`, it uses a stateless Akka Streams Flow in order to echo back the message. 28 | The input to the Flow represents the WebSocket messages coming from the WebSocket Client and the output to the Flow represents the messages that are being sent to the WebSocket Client 29 | - [`ws-chat`](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/ws/WebSocketRoutes.scala#L87) - This is an implementation of an online chat-room, it shows how to integrate Actors with Streams in order to create it. All credits go to [Johan Andrén](https://markatta.com/codemonkey/blog/2016/04/18/chat-with-akka-http-websockets/) 30 | for his excellent work and explanations on setting this up. This example involves creating a Flow from a Source and a Sink. I have provided my explanation and a [diagram](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/ws/WebSocketRoutes.scala#L76) 31 | to help you understand how this works. Essentially a Flow can be modelled as a Sink and a Source underneath the hood as illustrated by the diagram. If you choose to go 32 | about thinking in this manner then you have full control over the Flow (as in what the Flow accepts and what the Flow emits). We use Actors underneath the hood to perform 33 | the coordination between the accepting and emitting of Flow controlling messages to and from WebSocket clients. 34 | 35 | ## Chat Room Flow construction Overview ## 36 | WebSocket Clients connect to the Chat-Room via `ws://localhost:9000/ws-chat` and a Flow is created. Let's take a look at the inner workings of this Flow: 37 | 38 | - First an intermediate Actor is created (per WebSocket Client connection) that is meant to act as the bridge between the WebSocket Actor (more on this below) and the Chat Room Actor 39 | - The Flow is composed from a Sink and a Source. Take a look at the [diagram](https://github.com/calvinlfer/akka-http-streaming-response-examples/blob/master/src/main/scala/com/experiments/calvin/ws/WebSocketRoutes.scala#L76) before coming back here. 40 | - The Sink (inside the Flow) represents messages coming in from WebSocket clients, we use Sink.actorRef(intermediate Actor) so that the intermediate Actor will now receive messages whenever the WebSocket Source sends us messages 41 | - The Source (inside the Flow) represents messages sent to the WebSocket clients, we use Source.actorRef along with `mapMaterializedValue` to get access to the materialized ActorRef that we use to send messages to, in order for messages to be sent to the WebSocket client 42 | - We create a Flow from the Sink and the Source which now represents each WebSocket connection between our server and the WebSocket clients 43 | 44 | **Note:** Websocket clients can be found here: [Online WebSocket Tester](https://www.websocket.org/echo.html), [Dark WebSocket Client](https://chrome.google.com/webstore/detail/dark-websocket-terminal/dmogdjmcpfaibncngoolgljgocdabhke), etc. 45 | 46 | ## Contributions and PRs 47 | 48 | Please feel free to send in PRs and contributions and we can review them together and see whether they are small and cohesive enough to fit into the project 49 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | name := "akka-http-streaming-example" 2 | 3 | version := "1.0" 4 | 5 | scalaVersion := "2.12.3" 6 | 7 | libraryDependencies ++= { 8 | val akka = "com.typesafe.akka" 9 | val akkaV = "2.4.20" 10 | val akkaHttpV = "10.0.6" 11 | val scalaTestV = "3.0.0" 12 | Seq( 13 | akka %% "akka-actor" % akkaV, 14 | akka %% "akka-testkit" % akkaV % "test", 15 | akka %% "akka-slf4j" % akkaV, 16 | akka %% "akka-http-core" % akkaHttpV, 17 | akka %% "akka-http-spray-json" % akkaHttpV, 18 | "de.heikoseeberger" %% "akka-sse" % "3.0.0", 19 | akka %% "akka-http-testkit" % akkaHttpV, 20 | "org.scalactic" %% "scalactic" % scalaTestV, 21 | "org.scalatest" %% "scalatest" % scalaTestV % "test", 22 | "ch.qos.logback" % "logback-classic" % "1.1.7", 23 | "org.codehaus.groovy" % "groovy" % "2.4.7" 24 | ) 25 | } 26 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 0.13.16 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | logLevel := Level.Warn -------------------------------------------------------------------------------- /src/main/resources/application.conf: -------------------------------------------------------------------------------- 1 | akka { 2 | loglevel = DEBUG 3 | stdout-loglevel = DEBUG 4 | loggers = ["akka.event.slf4j.Slf4jLogger"] 5 | } -------------------------------------------------------------------------------- /src/main/resources/logback.groovy: -------------------------------------------------------------------------------- 1 | import ch.qos.logback.core.*; 2 | import ch.qos.logback.classic.encoder.PatternLayoutEncoder; 3 | 4 | appender(name="CONSOLE", clazz=ConsoleAppender) { 5 | encoder(PatternLayoutEncoder) { 6 | pattern = "date=[%date{ISO8601}] level=[%level] [%X{akkaSource}]: %msg\n" 7 | } 8 | } 9 | 10 | root(level=DEBUG, appenderNames=["CONSOLE"]) -------------------------------------------------------------------------------- /src/main/scala/com/experiments/calvin/BackpressuredActor.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin 2 | 3 | import akka.actor.{ActorLogging, Props} 4 | import akka.stream.actor.ActorPublisher 5 | import akka.stream.actor.ActorPublisherMessage.{Cancel, Request} 6 | import com.experiments.calvin.BackpressuredActor._ 7 | 8 | import scala.concurrent.duration._ 9 | import scala.annotation.tailrec 10 | import scala.language.postfixOps 11 | import scala.util.Random 12 | 13 | object BackpressuredActor { 14 | sealed trait Command 15 | case class SplitString(payload: String) extends Command 16 | 17 | sealed trait Event 18 | case object CommandAccepted extends Event 19 | case object CommandDenied extends Event 20 | case class StringHasBeenSplit(list: List[String]) extends Event 21 | 22 | def props = Props[BackpressuredActor] 23 | } 24 | 25 | /** 26 | * An actor that publishes StringHasBeenSplit messages and respects backpressure 27 | * This actor is meant to be created when using a Source.fromActorPublisher to allow integration between Actors 28 | * and Streams. 29 | * 30 | * The idea is that if you want to push some messages from your existing Actors into a Stream, you would 31 | * create a Publisher and feed your existing ActorRef into the Publisher and have the Publisher constantly ask your 32 | * Actor for messages which it emits into the Stream (this is not done here). When the Stream is cancelled, only 33 | * this Actor dies leaving your Actors intact without causing any disruptions. This is really an intermediate actor 34 | * that you use when you want to push data from your Actors into Streams. 35 | * 36 | * Credits: http://doc.akka.io/docs/akka/2.4.10/scala/stream/stream-integrations.html#ActorPublisher 37 | */ 38 | class BackpressuredActor extends ActorPublisher[StringHasBeenSplit] with ActorLogging { 39 | val MaxBufferSize = 100 40 | var buffer = Vector.empty[StringHasBeenSplit] 41 | implicit val ec = context.dispatcher 42 | val cancellable = context.system.scheduler.schedule(0 seconds, 100 milliseconds, self, SplitString(s"Hello! ${Random.nextString(10)}")) 43 | 44 | @tailrec 45 | private def deliverBuffer(): Unit = 46 | if (totalDemand > 0 && isActive) { 47 | // You are allowed to send as many elements as have been requested by the stream subscriber 48 | // total demand is a Long and can be larger than what the buffer has 49 | if (totalDemand <= Int.MaxValue) { 50 | val (sendDownstream, holdOn) = buffer.splitAt(totalDemand.toInt) 51 | buffer = holdOn 52 | // send the stuff downstream 53 | sendDownstream.foreach(onNext) 54 | } else { 55 | val (sendDownStream, holdOn) = buffer.splitAt(Int.MaxValue) 56 | buffer = holdOn 57 | sendDownStream.foreach(onNext) 58 | // recursive call checks whether is more demand before repeating the process 59 | deliverBuffer() 60 | } 61 | } 62 | 63 | 64 | override def receive: Receive = { 65 | case ss: SplitString if buffer.size == MaxBufferSize => 66 | log.warning("received a SplitString message when the buffer is maxed out") 67 | sender() ! CommandDenied 68 | 69 | case ss: SplitString => 70 | sender() ! CommandAccepted 71 | val result = StringHasBeenSplit(ss.payload.split("").toList) 72 | if (buffer.isEmpty && totalDemand > 0 && isActive) { 73 | // send elements to the stream immediately since there is demand from downstream and we 74 | // have not buffered anything so why bother buffering, send immediately 75 | // You send elements to the stream by calling onNext 76 | onNext(result) 77 | } 78 | else { 79 | // there is no demand from downstream so we will store the result in our buffer 80 | // Note that :+= means add to end of Vector 81 | buffer :+= result 82 | } 83 | 84 | // A request from down stream to send more data 85 | // When the stream subscriber requests more elements the ActorPublisherMessage.Request message is 86 | // delivered to this actor, and you can act on that event. The totalDemand is updated automatically. 87 | case Request(_) => deliverBuffer() 88 | 89 | // When the stream subscriber cancels the subscription the ActorPublisherMessage.Cancel message is 90 | // delivered to this actor. If the actor is stopped the stream will be completed, unless it was not 91 | // already terminated with failure, completed or canceled. 92 | case Cancel => 93 | log.info("Stream was cancelled") 94 | cancellable.cancel() 95 | context.stop(self) 96 | 97 | case CommandAccepted => () 98 | 99 | case other => 100 | log.warning(s"Unknown message $other received") 101 | } 102 | } 103 | -------------------------------------------------------------------------------- /src/main/scala/com/experiments/calvin/DetailedMessage.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin 2 | 3 | import java.util.UUID 4 | 5 | import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport 6 | import spray.json.{DefaultJsonProtocol, JsString, JsValue, RootJsonFormat} 7 | 8 | case class DetailedMessage(id: UUID, message: String) 9 | 10 | object DetailMessageJsonProtocol extends SprayJsonSupport with DefaultJsonProtocol { 11 | implicit object UUIDJsonFormat extends RootJsonFormat[UUID] { 12 | override def read(json: JsValue): UUID = UUID.fromString(json.convertTo[String]) 13 | override def write(uuid: UUID): JsValue = JsString(uuid.toString) 14 | } 15 | 16 | implicit val detailedMessageFormat = jsonFormat2(DetailedMessage.apply) 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/com/experiments/calvin/ServerMain.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin 2 | 3 | import akka.actor.ActorSystem 4 | import akka.http.scaladsl.Http 5 | import akka.stream.ActorMaterializer 6 | import akka.http.scaladsl.server.Directives._ 7 | import com.experiments.calvin.chunked.http.ChunkedStreamingRoutes 8 | import com.experiments.calvin.ws.WebSocketRoutes 9 | 10 | object ServerMain extends App with ChunkedStreamingRoutes with WebSocketRoutes { 11 | implicit val actorSystem = ActorSystem(name = "example-actor-system") 12 | implicit val streamMaterializer = ActorMaterializer() 13 | implicit val executionContext = actorSystem.dispatcher 14 | val log = actorSystem.log 15 | val allRoutes = httpStreamingRoutes ~ wsRoutes 16 | 17 | val bindingFuture = Http().bindAndHandle(allRoutes, "localhost", 9000) 18 | bindingFuture 19 | .map(_.localAddress) 20 | .map(addr => s"Bound to $addr") 21 | .foreach(log.info) 22 | } 23 | -------------------------------------------------------------------------------- /src/main/scala/com/experiments/calvin/WebsocketStreamsMain.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin 2 | 3 | import akka.NotUsed 4 | import akka.actor.ActorSystem 5 | import akka.http.scaladsl.Http 6 | import akka.http.scaladsl.model.ws.{Message, TextMessage} 7 | import akka.http.scaladsl.server.Directives._ 8 | import akka.stream.ActorMaterializer 9 | import akka.stream.scaladsl.{BroadcastHub, Flow, Keep, MergeHub, Sink, Source} 10 | 11 | import scala.concurrent.Future 12 | 13 | /** 14 | * A much simpler WebSocket chat system using only Akka Streams with the help of MergeHubSource and BroadcastHub Sink 15 | * Credits: https://markatta.com/codemonkey/blog/2016/10/02/chat-with-akka-http-websockets/ 16 | */ 17 | object WebsocketStreamsMain extends App { 18 | implicit val actorSystem = ActorSystem(name = "example-actor-system") 19 | implicit val streamMaterializer = ActorMaterializer() 20 | implicit val executionContext = actorSystem.dispatcher 21 | val log = actorSystem.log 22 | 23 | /* 24 | many clients -> Merge Hub -> Broadcast Hub -> many clients 25 | Visually 26 | Akka Streams Flow 27 | ________________________________________________________________________________________________________________________________________________________________________________________ 28 | c1 -------->\ | | /->----------- c1 29 | \ | | / 30 | c2 ----------->| Sink ========================(feeds data to)===========> MergeHub Source ->-->-->--> BroadcastHub Sink ======(feeds data to)===========> Source |->->------------ c2 31 | /| that comes from materializing the connected to that comes from materializing the | \ 32 | / | MergeHub Source BroadcastHub Sink | \ 33 | c3 -------->/ |________________________________________________________________________________________________________________________________________________________________________________________| \->---------- c3 34 | 35 | 36 | Runnable Flow (MergeHubSource -> BroadcastHubSink) 37 | 38 | Materializing a MergeHub Source yields a Sink that collects all the emitted elements and emits them in the MergeHub Source (the emitted elements that are collected in the Sink are coming from all WebSocket clients) 39 | Materializing a BroadcastHub Sink yields a Source that broadcasts all elements being collected by the MergeHub Sink (the elements that are emitted/broadcasted in the Source are going to all WebSocket clients) 40 | 41 | This example relies on a materializer 42 | */ 43 | val (chatSink: Sink[String, NotUsed], chatSource: Source[String, NotUsed]) = 44 | MergeHub.source[String].toMat(BroadcastHub.sink[String])(Keep.both).run() 45 | 46 | private val userFlow: Flow[Message, Message, NotUsed] = 47 | Flow[Message].mapAsync(1) { 48 | case TextMessage.Strict(text) => Future.successful(text) 49 | case streamed: TextMessage.Streamed => streamed.textStream.runFold("") { 50 | (acc, next) => acc ++ next 51 | } 52 | } 53 | .via(Flow.fromSinkAndSource(chatSink, chatSource)) 54 | .map[Message](string => TextMessage.Strict(string)) 55 | 56 | def wsChatStreamsOnlyRoute = 57 | path("ws-chat-streams-only") { 58 | handleWebSocketMessages(userFlow) 59 | } 60 | 61 | val bindingFuture = Http().bindAndHandle(wsChatStreamsOnlyRoute, "localhost", 9000) 62 | bindingFuture 63 | .map(_.localAddress) 64 | .map(addr => s"Bound to $addr") 65 | .foreach(log.info) 66 | } 67 | -------------------------------------------------------------------------------- /src/main/scala/com/experiments/calvin/chunked/http/ChunkedStreamingRoutes.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin.chunked.http 2 | 3 | import java.util.UUID 4 | 5 | import akka.actor.ActorSystem 6 | import akka.http.scaladsl.common.EntityStreamingSupport 7 | import akka.http.scaladsl.model.ContentTypes._ 8 | import akka.http.scaladsl.model.HttpEntity 9 | import akka.http.scaladsl.server.Directives._ 10 | import akka.stream.actor.ActorPublisher 11 | import akka.stream.scaladsl.{Flow, Source} 12 | import akka.stream.{ActorMaterializer, ThrottleMode} 13 | import akka.util.ByteString 14 | import com.experiments.calvin.{BackpressuredActor, DetailedMessage} 15 | import com.experiments.calvin.BackpressuredActor.{SplitString, StringHasBeenSplit} 16 | import de.heikoseeberger.akkasse.scaladsl.model.ServerSentEvent 17 | import spray.json.JsonWriter 18 | 19 | import scala.concurrent.{ExecutionContext, Future} 20 | import scala.concurrent.duration._ 21 | import scala.language.postfixOps 22 | import scala.util.Random 23 | 24 | trait ChunkedStreamingRoutes { 25 | implicit val actorSystem: ActorSystem 26 | implicit val streamMaterializer: ActorMaterializer 27 | implicit val executionContext: ExecutionContext 28 | lazy val httpStreamingRoutes = 29 | streamingTextRoute ~ actorStreamingTextRoute ~ altActorStreamingTextRoute ~ actorStreamingTextRouteWithLiveActor ~ 30 | streamingJsonRoute ~ streamingJsonRouteWithSSE ~ consumingStreamingJson 31 | 32 | def streamingTextRoute = 33 | path("streaming-text") { 34 | get { 35 | val sourceOfInformation = Source("Prepare to scroll!") 36 | val sourceOfNumbers = Source(1 to 1000000) 37 | val byteStringSource = sourceOfInformation.concat(sourceOfNumbers) // merge the two sources 38 | .throttle(elements = 1000, per = 1 second, maximumBurst = 1, mode = ThrottleMode.Shaping) 39 | .map(_.toString) 40 | .map(s => ByteString(s + "\n")) 41 | 42 | complete(HttpEntity(`text/plain(UTF-8)`, byteStringSource)) 43 | } 44 | } 45 | 46 | // Credits: 47 | // http://stackoverflow.com/questions/35634283/pushing-messages-via-web-sockets-with-akka-http/ 48 | // http://stackoverflow.com/questions/29072963/how-to-add-elements-to-source-dynamically 49 | def actorStreamingTextRoute = 50 | path("actor-text") { 51 | get { 52 | val source = Source.actorPublisher[StringHasBeenSplit](BackpressuredActor.props) 53 | .mapConcat(ss => ss.list) 54 | .map(s => ByteString(s + "\n")) 55 | complete(HttpEntity(`text/plain(UTF-8)`, source)) 56 | } 57 | } 58 | 59 | def altActorStreamingTextRoute = 60 | path("alt-actor-text") { 61 | get { 62 | // in addition to sending events through the scheduler inside the actor 63 | val source = Source.actorPublisher[StringHasBeenSplit](BackpressuredActor.props) 64 | .map(s => ByteString(s + "\n")) 65 | .mapMaterializedValue(ref => { 66 | actorSystem.scheduler.schedule(0 seconds, 500 milliseconds, ref, SplitString("Calvin says hi")) 67 | ref 68 | }) 69 | complete(HttpEntity(`text/plain(UTF-8)`, source)) 70 | } 71 | } 72 | 73 | def actorStreamingTextRouteWithLiveActor = 74 | path("live-actor-text") { 75 | get { 76 | // create our Backpressured actor 77 | val actorRef = actorSystem.actorOf(BackpressuredActor.props) 78 | // wrap it in a ActorPublisher that will act as a Source[StringHasBeenSplit, _] 79 | val publisher = ActorPublisher[StringHasBeenSplit](actorRef) 80 | val source = Source.fromPublisher(publisher).map(s => ByteString(s + "\n")) 81 | // send messages into the Stream by sending messages to the actor ref will translate into downstream messages 82 | actorSystem.scheduler.schedule(0 seconds, 100 milliseconds, actorRef, SplitString(s"Hello! ${Random.nextString(10)}")) 83 | 84 | complete(HttpEntity(`text/plain(UTF-8)`, source)) 85 | } 86 | } 87 | 88 | 89 | val start = ByteString.empty 90 | val sep = ByteString("\n") 91 | val end = ByteString.empty 92 | 93 | import com.experiments.calvin.DetailMessageJsonProtocol._ 94 | implicit val jsonStreamingSupport = EntityStreamingSupport.json() 95 | .withFramingRenderer(Flow[ByteString].intersperse(start, sep, end)) 96 | 97 | // More customization: 98 | // http://doc.akka.io/docs/akka/2.4.10/scala/http/routing-dsl/source-streaming-support.html#Customising_response_rendering_mode 99 | def streamingJsonRoute = 100 | path("streaming-json") { 101 | get { 102 | val sourceOfNumbers = Source(1 to 1000000) 103 | val sourceOfDetailedMessages = 104 | sourceOfNumbers.map(num => DetailedMessage(UUID.randomUUID(), s"Hello $num")) 105 | .throttle(elements = 1000, per = 1 second, maximumBurst = 1, mode = ThrottleMode.Shaping) 106 | 107 | complete(sourceOfDetailedMessages) 108 | } 109 | } 110 | 111 | def consumingStreamingJson = 112 | path("consuming-streaming-json") { 113 | post { 114 | entity(asSourceOf[DetailedMessage]) { detailedMessageSource => 115 | val messagesSubmitted: Future[Int] = detailedMessageSource.runFold(0){(acc, _) => acc + 1} 116 | complete(messagesSubmitted.map(_.toString)) 117 | } 118 | } 119 | } 120 | 121 | import de.heikoseeberger.akkasse.scaladsl.marshalling.EventStreamMarshalling._ 122 | // More information: 123 | // https://github.com/hseeberger/akka-sse 124 | def streamingJsonRouteWithSSE = 125 | path("streaming-sse-json") { 126 | get { 127 | val sourceOfNumbers = Source(1 to 1000000) 128 | val sourceOfDetailedMessagesForSSE = sourceOfNumbers 129 | .throttle(elements = 1000, per = 1 second, maximumBurst = 1, mode = ThrottleMode.Shaping) 130 | .map(num => DetailedMessage(UUID.randomUUID(), s"Hello $num")) 131 | .map(dm => 132 | // long notation is used to pass in implicit JSON marshaller 133 | wrapWithServerSentEvent(dm) 134 | ) 135 | 136 | complete(sourceOfDetailedMessagesForSSE) 137 | } 138 | } 139 | 140 | private def wrapWithServerSentEvent[T](element: T)(implicit writer: JsonWriter[T]): ServerSentEvent = 141 | ServerSentEvent(data = writer.write(element).compactPrint, `type` = "detailedMessage") 142 | } -------------------------------------------------------------------------------- /src/main/scala/com/experiments/calvin/ws/ChatRoom.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin.ws 2 | 3 | import akka.actor.{Actor, ActorLogging, ActorRef, Terminated} 4 | import com.experiments.calvin.ws.ChatRoom.{ChatMessage, JoinRoom} 5 | 6 | /** 7 | * An actor that represents a chat hub to which all messages that come over 8 | * websockets will be sent. This is essentially broadcasting any incoming messages 9 | * to everyone who is connected 10 | */ 11 | class ChatRoom extends Actor with ActorLogging { 12 | var users: Set[ActorRef] = Set.empty 13 | 14 | override def receive: Receive = { 15 | case JoinRoom => 16 | log.info(s"Received a JoinRoom message from ${sender()}") 17 | users += sender() 18 | // we want to remove the user if it's actor is stopped 19 | context.watch(sender()) 20 | 21 | case cm: ChatMessage => 22 | log.info(s"Received a ChatMessage message from ${sender()}") 23 | users.foreach(_ ! cm) 24 | 25 | case Terminated(user) => 26 | log.info(s"${sender()} is leaving") 27 | users -= user 28 | } 29 | } 30 | 31 | object ChatRoom { 32 | case object JoinRoom 33 | case class ChatMessage(message: String) 34 | } 35 | -------------------------------------------------------------------------------- /src/main/scala/com/experiments/calvin/ws/ConnectedUser.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin.ws 2 | import akka.actor.{Actor, ActorLogging, ActorRef, Props} 3 | import com.experiments.calvin.ws.ChatRoom.{ChatMessage, JoinRoom} 4 | import com.experiments.calvin.ws.ConnectedUser.{Connected, IncomingMessage, OutgoingMessage} 5 | 6 | /** 7 | * This actor represents a connected user over a WebSocket connection and acts as a bridge between the WS Actor and 8 | * the Chat Room. It is responsible for managing incoming messages from the WS actor and messaging being delivered 9 | * from the Chat Room to the WS actor 10 | * 11 | * Additional Notes: 12 | * This actor is the direct recipient of the Sink inside the WebSocket Flow. This actor will get a PoisonPill when 13 | * the WebSocket Client (sending portion) terminates the Stream 14 | * This actor behaves as an intermediary only when it comes to sending messages to the WS client, The BackPressured 15 | * Actor Publisher is the direct integration point for publishing messages into the Stream. We obtain an ActorRef to 16 | * that Actor Publisher when that portion of the Stream is materialized via the mapMaterializedValue. Both this actor 17 | * and the BackPressured Actor Publisher live for the duration of a connected WS Client 18 | */ 19 | class ConnectedUser(chatRoom: ActorRef) extends Actor with ActorLogging { 20 | override def receive: Receive = waiting 21 | 22 | def waiting: Receive = { 23 | // When the user connects, tell the chat room about it so messages 24 | // sent to the chat room are routed here 25 | case Connected(wsActor) => 26 | log.info(s"WS user: $wsActor has connected") 27 | context become connected(wsActor) 28 | chatRoom ! JoinRoom 29 | } 30 | 31 | def connected(wsUser: ActorRef): Receive = { 32 | // any messages coming from the WS client will come here and will be sent to the chat room 33 | case IncomingMessage(text) => 34 | log.debug("Intermediate Actor sending message to chat room") 35 | chatRoom ! ChatMessage(text) 36 | 37 | // any messages coming from the chat room need to be sent to the WS Client 38 | // remember that in this case we are the intermediate bridge and we have to send the message to the ActorPublisher 39 | // in order for the WS client to receive the message 40 | case ChatMessage(message) => 41 | log.debug(s"Intermediate Actor sending message that came from the chat room to $wsUser") 42 | wsUser ! OutgoingMessage(message) 43 | } 44 | } 45 | 46 | object ConnectedUser { 47 | sealed trait UserMessage 48 | case class Connected(outgoing: ActorRef) 49 | case class IncomingMessage(text: String) extends UserMessage 50 | case class OutgoingMessage(text: String) extends UserMessage 51 | 52 | def props(chatRoom: ActorRef) = Props(new ConnectedUser(chatRoom)) 53 | } -------------------------------------------------------------------------------- /src/main/scala/com/experiments/calvin/ws/WebSocketRoutes.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin.ws 2 | 3 | import akka.NotUsed 4 | import akka.actor.{ActorSystem, PoisonPill, Props} 5 | import akka.http.scaladsl.model.ws.{Message, TextMessage} 6 | import akka.http.scaladsl.server.Directives._ 7 | import akka.stream.OverflowStrategy 8 | import akka.stream.scaladsl.{Flow, Sink, Source} 9 | 10 | import scala.language.postfixOps 11 | 12 | /** 13 | * WebSocket routes 14 | * Note: if you do not send anything for over 1 minute then the connection will be closed 15 | */ 16 | trait WebSocketRoutes { 17 | val actorSystem: ActorSystem 18 | lazy val chatRoom = actorSystem.actorOf(Props[ChatRoom], "chat-room") 19 | lazy val wsRoutes = websocketRoute ~ wsChatRoute 20 | 21 | // Note: see http://blog.scalac.io/2015/07/30/websockets-server-with-akka-http.html for something way more complex 22 | def websocketRoute = 23 | path("ws-simple") { 24 | get { 25 | val echoFlow: Flow[Message, Message, _] = Flow[Message].map { 26 | case TextMessage.Strict(text) => TextMessage(s"I got your message: $text!") 27 | case _ => TextMessage(s"Sorry I didn't quite get that") 28 | } 29 | handleWebSocketMessages(echoFlow) 30 | } 31 | } 32 | 33 | private def newUser(): Flow[Message, Message, NotUsed] = { 34 | // create a user actor per webSocket connection that is able to talk to the chat room 35 | val connectedWsActor = actorSystem.actorOf(ConnectedUser.props(chatRoom)) 36 | 37 | // Think about this differently - in the normal case for a stateless Flow, we have incoming messages coming from 38 | // the WS client and outgoing messages going to the WS client 39 | // Here incomingMessages appears to be modelled as a Sink, the idea is that any messages being sent on a Source 40 | // connected to this Sink need to go to an actor (the Source being the WebSocket client) 41 | val incomingMessages: Sink[Message, NotUsed] = 42 | Flow[Message].map { 43 | case TextMessage.Strict(text) => ConnectedUser.IncomingMessage(text) 44 | }.to(Sink.actorRef(connectedWsActor, PoisonPill)) 45 | 46 | // Here outgoingMessages appear to be modelled as a Source, the idea is that any messages being sent by the actor 47 | // (representing the WS client) needs to be sent to the actual WS client over the websocket (being the Sink) 48 | val outgoingMessages: Source[Message, NotUsed] = 49 | Source 50 | .actorRef[ConnectedUser.OutgoingMessage](10, OverflowStrategy.fail) 51 | .mapMaterializedValue { outgoingActor => 52 | // you need to send a Connected message to get the actor in a state 53 | // where it's ready to receive and send messages, we used the mapMaterialized value 54 | // so we can get access to it as soon as this is materialized 55 | connectedWsActor ! ConnectedUser.Connected(outgoingActor) 56 | NotUsed 57 | } 58 | .map { 59 | // Domain Model => WebSocket Message 60 | case ConnectedUser.OutgoingMessage(text) => TextMessage(text) 61 | } 62 | 63 | /* 64 | We create a flow from the Sink and Source - it's this that makes sense because here incoming messages are coming 65 | from the WS Client and being sent to the WS Actor and eventually the chat room and the out going messages that are 66 | coming from the WS actor are being sent over the WebSocket this follows the same idea as the stateless Flow 67 | (incoming from WebSocket client -> outgoing to WebSocket client) 68 | 69 | Above, we see that messages coming into the flow are received by the Sink and we use actors to emit messages to 70 | the source 71 | 72 | To me, the inside of a Flow looks like this: 73 | Flow 74 | ______________________________________________________________________________________________________________________________________ 75 | | | 76 | WebSocket Source ->->->->->->->->->->->->->-| Sink (WebSocket Actor sending message to ConnectedUserActor) Source (ConnectedUserActor sending message to WebSocket Actor |->->->->->->->->->->->-> WebSocket Sink 77 | | | 78 | | Accomplished using Sink.actorRef, any information emitted on Accomplished using Source.actorRef, any OutgoingMessage sent | 79 | | the Stream will end up at the ConnectedUserActor to the materialized Actor will be emitted on the Stream | 80 | (incoming messages) |______________________________________________________________________________________________________________________________________| (outgoing messages) 81 | */ 82 | Flow.fromSinkAndSource(incomingMessages, outgoingMessages) 83 | } 84 | 85 | // Credits: http://markatta.com/codemonkey/blog/2016/04/18/chat-with-akka-http-websockets/ 86 | def wsChatRoute = 87 | path("ws-chat") { 88 | handleWebSocketMessages(newUser()) 89 | } 90 | } 91 | -------------------------------------------------------------------------------- /src/test/scala/com/experiments/calvin/ChunkedStreamingSpec.scala: -------------------------------------------------------------------------------- 1 | package com.experiments.calvin 2 | 3 | import java.util.UUID 4 | 5 | import akka.actor.ActorSystem 6 | import akka.http.scaladsl.model.{ContentTypes, HttpEntity, StatusCodes} 7 | import akka.http.scaladsl.testkit.ScalatestRouteTest 8 | import akka.stream.ActorMaterializer 9 | import com.experiments.calvin.chunked.http.ChunkedStreamingRoutes 10 | import org.scalatest._ 11 | import org.scalatest.concurrent.ScalaFutures 12 | 13 | import scala.concurrent.ExecutionContext 14 | import scala.language.postfixOps 15 | 16 | class ChunkedStreamingSpec extends FunSpec with MustMatchers with ScalatestRouteTest with ScalaFutures { 17 | 18 | describe("Chunked Streaming Tests") { 19 | val chunkedStreamingRoutes = createService() 20 | 21 | // Still pending: need to figure out how to not consume the entire stream, consume only 2 elements and discard 22 | // the rest 23 | // it("streams text when you hit /streaming-text") { 24 | // Get("/streaming-text") ~> chunkedStreamingRoutes.httpStreamingRoutes ~> check { 25 | // val future = response.entity.dataBytes.take(2).map(_.decodeString("UTF-8")).runFold("")(_ + _) 26 | // whenReady(future) { 27 | // result => println(result) 28 | // } 29 | // } 30 | // } 31 | 32 | it("consumes streaming JSON and sends you the count of the elements that were streamed") { 33 | val data = HttpEntity( 34 | ContentTypes.`application/json`, 35 | s""" 36 | |{"id": "c3f67175-5de8-401e-b269-7be8666340d3", "message": "hello"} 37 | |{"id": "c3f67175-5de8-401e-b269-7be8666340d3", "message": "hi there"} 38 | |{"id": "c3f67175-5de8-401e-b269-7be8666340d3", "message": "bye"} 39 | """.stripMargin 40 | ) 41 | Post("/consuming-streaming-json", data) ~> chunkedStreamingRoutes.httpStreamingRoutes ~> check { 42 | status mustBe StatusCodes.OK 43 | responseAs[String] mustBe "3" 44 | } 45 | } 46 | } 47 | 48 | private def createService(): ChunkedStreamingRoutes = new ChunkedStreamingRoutes { 49 | override implicit val actorSystem: ActorSystem = ChunkedStreamingSpec.this.system 50 | override implicit val streamMaterializer: ActorMaterializer = ChunkedStreamingSpec.this.materializer 51 | override implicit val executionContext: ExecutionContext = actorSystem.dispatcher 52 | } 53 | } 54 | --------------------------------------------------------------------------------