├── .gitattributes ├── .github └── workflows │ ├── ci.yml │ └── scala-steward.yml ├── .gitignore ├── .scala-steward.conf ├── LICENSE ├── README.md ├── build.sbt ├── docker ├── config │ └── akhq │ │ └── application.yml └── docker-compose.yml ├── measurements_subset_10000.txt ├── project ├── build.properties └── plugins.sbt └── src ├── main ├── java │ ├── PgVectorEmbeddingStoreExample.java │ ├── actor │ │ ├── DemoMessagesActor.java │ │ ├── HelloWorld.java │ │ ├── HelloWorldBot.java │ │ └── HelloWorldMain.java │ └── util │ │ └── ConnectionStatusChecker.java ├── resources │ ├── 2008_subset.csv │ ├── 63MB.pdf │ ├── ADT_ORM_Hl7Messages.txt │ ├── EN_challenges.srt │ ├── FileWithLotsOfHl7Messages.txt │ ├── KeycloakClient.html │ ├── SSEClientWikipediaEdits.html │ ├── SampleRoutes.html │ ├── WebsocketEcho.html │ ├── application.conf │ ├── assistant.html │ ├── broker.xml │ ├── broker_docker.xml │ ├── en-ner-person.bin │ ├── fileupload.html │ ├── jndi.properties │ ├── js │ │ └── keycloak.js │ ├── keycloak_realm_config.json │ ├── line_protocol_data.txt │ ├── localstack │ │ ├── init_dynamodb.sh │ │ ├── init_firehose.sh │ │ ├── init_kinesis.sh │ │ └── init_sqs.sh │ ├── logback.xml │ ├── opennlp-en-ud-ewt-tokens-1.0-1.9.3.bin │ ├── opennlp-en-ud-ewt-tokens-1.2-2.5.0.bin │ ├── payload.zip │ ├── testfile.jpg │ └── xml_with_base64_embedded.xml └── scala │ ├── actor │ ├── BlockingActor.scala │ ├── BlockingRight.scala │ ├── Buncher.scala │ ├── FaultyActor.scala │ ├── PrintActor.scala │ └── TargetActor.scala │ ├── akkahttp │ ├── HTTPResponseStream.scala │ ├── HttpClientSingleRequest.scala │ ├── HttpFileEcho.scala │ ├── HttpFileEchoStream.scala │ ├── ReverseProxy.scala │ ├── SampleRoutes.scala │ ├── ServerWithCustomDispatcher.scala │ ├── WebsocketChatEcho.scala │ ├── WebsocketEcho.scala │ ├── WebsocketEchoActors.scala │ └── oidc │ │ ├── CORSHandler.scala │ │ ├── JsonSupport.scala │ │ └── OIDCKeycloak.scala │ ├── alpakka │ ├── amqp │ │ └── AmqpEcho.scala │ ├── clickhousedb │ │ └── ClickhouseDB.scala │ ├── dynamodb │ │ └── DynamoDBEcho.scala │ ├── env │ │ ├── FileServer.scala │ │ ├── JMSServerArtemis.scala │ │ ├── KafkaServerEmbedded.scala │ │ ├── KafkaServerTestcontainers.scala │ │ ├── WebsocketServer.scala │ │ └── jms │ │ │ ├── AESBroker.java │ │ │ ├── AESBrokerPlugin.java │ │ │ └── JMSServerActiveMQ.scala │ ├── file │ │ ├── ArchiveHelper.java │ │ ├── DuplicateStream.scala │ │ ├── FileEncDec.java │ │ ├── FileIOEcho.scala │ │ ├── FileRotator.scala │ │ ├── ZipCryptoEcho.scala │ │ └── uploader │ │ │ ├── DirectoryWatcher.scala │ │ │ └── Uploader.scala │ ├── influxdb │ │ ├── InfluxdbReader.scala │ │ └── InfluxdbWriter.java │ ├── jms │ │ ├── JMSTextMessageProducerClient.scala │ │ └── ProcessingApp.scala │ ├── kafka │ │ ├── TextMessageGenerator.scala │ │ ├── TotalFake.scala │ │ ├── WordCountConsumer.scala │ │ ├── WordCountKStreams.java │ │ ├── WordCountProducer.scala │ │ └── avro │ │ │ ├── AvroRecord.scala │ │ │ ├── SimpleAvroConsumer.scala │ │ │ └── SimpleAvroProducer.scala │ ├── kinesis │ │ ├── FirehoseEcho.scala │ │ └── KinesisEcho.scala │ ├── mqtt │ │ ├── MqttEcho.scala │ │ └── MqttPahoEcho.scala │ ├── patterns │ │ └── PassThroughFlowExample.scala │ ├── s3 │ │ └── S3Echo.scala │ ├── sftp │ │ └── SftpEcho.scala │ ├── slick │ │ └── SlickRunner.scala │ ├── sqs │ │ └── SqsEcho.scala │ ├── sse │ │ ├── SSEClientWikipediaEdits.scala │ │ └── SSEHeartbeat.scala │ ├── sse_to_elasticsearch │ │ └── SSEtoElasticsearch.scala │ ├── tcp_to_websockets │ │ ├── hl7mllp │ │ │ ├── AdtMessageFactory.java │ │ │ ├── ExceptionHandler.java │ │ │ ├── HandlingMultipleVersions.java │ │ │ ├── Hl7MllpListener.java │ │ │ ├── Hl7Tcp2Kafka.scala │ │ │ ├── Hl7TcpClient.scala │ │ │ ├── Hl7TcpClientClassic.java │ │ │ ├── MllpProtocol.scala │ │ │ ├── MockApp.java │ │ │ └── OurAdtA01MessageBuilder.java │ │ └── websockets │ │ │ ├── Kafka2SSE.scala │ │ │ ├── Kafka2Websocket.scala │ │ │ ├── WebSocketClient.scala │ │ │ ├── WebsocketClientActor.scala │ │ │ └── WebsocketConnectionStatusActor.scala │ └── xml │ │ └── XmlProcessing.scala │ ├── interop │ └── ReactiveStreamsInterop.scala │ ├── sample │ ├── graphdsl │ │ ├── CompoundFlowFromGraph.scala │ │ ├── DistributeAndMerge.scala │ │ └── WritePrimes.scala │ ├── graphstage │ │ ├── Pulse.scala │ │ ├── StreamEventInspector.scala │ │ └── ThroughputMonitor.scala │ ├── stream │ │ ├── AsyncExecution.scala │ │ ├── AvoidDeadlockByUsingSeparateCustomDispatchers.scala │ │ ├── BasicTransformation.scala │ │ ├── CalculateMedian.scala │ │ ├── FlightDelayStreaming.scala │ │ ├── MergeHubWithDynamicSources.scala │ │ ├── NumberGuessingGame.scala │ │ ├── OneBillionRowChallenge.scala │ │ ├── OneBillionRowChallengeScale.scala │ │ ├── PartitionHubWithDynamicSinks.scala │ │ ├── PossiblyEmpty.scala │ │ ├── PublishToBlockingResource.scala │ │ ├── PublishToSourceQueueFromMultipleThreads.scala │ │ ├── SessionWindow.scala │ │ ├── SlowConsumerDropsElementsOnFastProducer.scala │ │ ├── TcpEcho.scala │ │ ├── TransformCSV.scala │ │ ├── TweetExample.scala │ │ ├── WaitForFlowsToComplete.scala │ │ └── ZipLatestWith.scala │ ├── stream_actor │ │ ├── SimulateWindTurbines.scala │ │ ├── Total.scala │ │ ├── WebSocketClient.scala │ │ ├── WindTurbineData.scala │ │ ├── WindTurbineServer.scala │ │ ├── WindTurbineSimulator.scala │ │ └── typed │ │ │ ├── CustomCache.scala │ │ │ └── CustomCacheRunner.scala │ ├── stream_actor_simple │ │ ├── EncapsulateStreamWithActor.scala │ │ └── PrintMoreNumbers.scala │ ├── stream_divert │ │ ├── AlsoTo.scala │ │ └── DivertTo.scala │ └── stream_shared_state │ │ ├── Blacklist.scala │ │ ├── Dedupe.scala │ │ ├── Deduplicate.scala │ │ ├── DeduplicateConsecutiveElements.scala │ │ ├── DeferredStreamCreation.scala │ │ ├── DownloaderRetry.java │ │ ├── FileLister.java │ │ ├── HandleFirstElementSpecially.scala │ │ ├── LocalFileCacheCaffeine.scala │ │ ├── ParametrizedFlow.scala │ │ ├── SplitAfter.scala │ │ ├── SplitAfterPrefix.scala │ │ ├── SplitWhen2.scala │ │ ├── WindowingExample.scala │ │ └── WindowingExampleScale.scala │ └── tools │ ├── AnthropicCompletions.java │ ├── CompletionsUtil.java │ ├── OpenAICompletions.java │ ├── SrtParser.scala │ └── SubtitleTranslator.scala ├── scalafix ├── resources │ └── META-INF │ │ └── services │ │ └── scalafix.v1.Rule └── scala │ └── fix │ └── PekkoScalafix.scala └── test └── scala ├── DirectoryWatcherSimulation.scala ├── ReverseProxySimulation.scala ├── alpakka ├── clickhousedb │ └── ClickhousedbIT.java ├── dynamodb │ └── DynamoDBEchoIT.java ├── file │ └── DirectoryWatcherSpec.scala ├── firehose │ └── FirehoseEchoIT.java ├── influxdb │ └── InfluxdbIT.java ├── kinesis │ └── KinesisEchoIT.java ├── s3 │ ├── MinioContainer.java │ └── S3EchoMinioIT.java ├── slick │ ├── PostgresClusterEmbedded.java │ ├── SlickEmbeddedIT.java │ └── SlickIT.java ├── sqs │ └── SqsEchoIT.java └── tcp_to_websockets │ └── AlpakkaTrophySpec.scala ├── tools └── SrtParserSpec.scala └── util └── LogFileScanner.scala /.gitattributes: -------------------------------------------------------------------------------- 1 | *.sh text eol=lf 2 | *.sh text eol=lf 3 | *.srt text -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- 1 | name: CI 2 | on: 3 | push: 4 | branches: 5 | - master 6 | - migrate_pekko 7 | jobs: 8 | test: 9 | runs-on: ubuntu-22.04 10 | env: 11 | # define Java options for both official sbt and sbt-extras 12 | JAVA_OPTS: -Xms2048M -Xmx2048M -Xss6M -XX:ReservedCodeCacheSize=256M -Dfile.encoding=UTF-8 13 | JVM_OPTS: -Xms2048M -Xmx2048M -Xss6M -XX:ReservedCodeCacheSize=256M -Dfile.encoding=UTF-8 14 | steps: 15 | - name: Checkout 16 | uses: actions/checkout@v4 17 | # https://github.com/coursier/setup-action 18 | - name: Set up JDK 17 19 | uses: coursier/setup-action@v1 20 | with: 21 | jvm: 17 22 | apps: sbtn 23 | - name: Build and Test 24 | run: sbt -v +test -------------------------------------------------------------------------------- /.github/workflows/scala-steward.yml: -------------------------------------------------------------------------------- 1 | # Run Scala Steward with Github Actions 2 | # Doc: 3 | # https://github.com/scala-steward-org/scala-steward-action 4 | name: Scala Steward 5 | 6 | on: 7 | # Disable to avoid duplicate PRs, 8 | # because https://github.com/scala-steward-org/scala-steward is active again 9 | # schedule: 10 | # # Run every Sunday @ 12PM UTC 11 | # - cron: "0 0 * * 0" 12 | # Possible to execute via "Run workflow" button at the Github workflow page 13 | workflow_dispatch: 14 | 15 | jobs: 16 | scala-steward: 17 | runs-on: ubuntu-latest 18 | name: Launch Scala Steward 19 | steps: 20 | - name: Launch Scala Steward 21 | uses: scala-steward-org/scala-steward-action@v2 22 | with: 23 | github-token: ${{ secrets.GITHUB_TOKEN }} 24 | github-repository: pbernet/akka_streams_tutorial -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *# 2 | *.iml 3 | *.ipr 4 | *.iws 5 | *.pyc 6 | *.tm.epoch 7 | *.vim 8 | */project/boot 9 | */project/build/target 10 | */project/project.target.config-classes 11 | *-shim.sbt 12 | *~ 13 | .#* 14 | .*.swp 15 | .DS_Store 16 | .cache 17 | .cache 18 | .classpath 19 | .codefellow 20 | .ensime* 21 | .eprj 22 | .history 23 | .idea 24 | .manager 25 | .multi-jvm 26 | .project 27 | .scala_dependencies 28 | .scalastyle 29 | .settings 30 | .tags 31 | .tags_sorted_by_file 32 | .target 33 | .worksheet 34 | Makefile 35 | TAGS 36 | lib_managed 37 | logs 38 | project/boot/* 39 | project/plugins/project 40 | src_managed 41 | target 42 | tm*.lck 43 | tm*.log 44 | tm.out 45 | worker*.log 46 | /bin 47 | -------------------------------------------------------------------------------- /.scala-steward.conf: -------------------------------------------------------------------------------- 1 | updates.ignore = [{ groupId = "com.geteventstore", artifactId = "eventstore-client" }] -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2019 Paul Bernet 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /docker/config/akhq/application.yml: -------------------------------------------------------------------------------- 1 | akhq: 2 | connections: 3 | docker-kafka-server: 4 | properties: 5 | bootstrap.servers: "broker:9092" 6 | schema-registry: 7 | type: confluent 8 | url: "http://apicurio:8080/api" 9 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.10.11 -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.14.2") 2 | //The now built in dependencyTree task is usually enough 3 | //https://www.scala-sbt.org/1.x/docs/sbt-1.4-Release-Notes.html#sbt-dependency-graph+is+in-sourced 4 | //addDependencyTreePlugin 5 | 6 | // https://docs.gatling.io/reference/integrations/build-tools/sbt-plugin 7 | addSbtPlugin("io.gatling" % "gatling-sbt" % "4.13.1") -------------------------------------------------------------------------------- /src/main/java/PgVectorEmbeddingStoreExample.java: -------------------------------------------------------------------------------- 1 | import dev.langchain4j.data.embedding.Embedding; 2 | import dev.langchain4j.data.segment.TextSegment; 3 | import dev.langchain4j.model.embedding.EmbeddingModel; 4 | import dev.langchain4j.model.embedding.onnx.allminilml6v2q.AllMiniLmL6V2QuantizedEmbeddingModel; 5 | import dev.langchain4j.store.embedding.EmbeddingMatch; 6 | import dev.langchain4j.store.embedding.EmbeddingSearchRequest; 7 | import dev.langchain4j.store.embedding.EmbeddingStore; 8 | import dev.langchain4j.store.embedding.pgvector.PgVectorEmbeddingStore; 9 | import org.testcontainers.containers.PostgreSQLContainer; 10 | import org.testcontainers.utility.DockerImageName; 11 | 12 | import java.util.List; 13 | 14 | public class PgVectorEmbeddingStoreExample { 15 | 16 | public static void main(String[] args) { 17 | 18 | DockerImageName dockerImageName = DockerImageName.parse("pgvector/pgvector:pg17"); 19 | try (PostgreSQLContainer postgreSQLContainer = new PostgreSQLContainer<>(dockerImageName)) { 20 | postgreSQLContainer.start(); 21 | 22 | EmbeddingModel embeddingModel = new AllMiniLmL6V2QuantizedEmbeddingModel(); 23 | 24 | EmbeddingStore embeddingStore = PgVectorEmbeddingStore.builder() 25 | .host(postgreSQLContainer.getHost()) 26 | .port(postgreSQLContainer.getFirstMappedPort()) 27 | .database(postgreSQLContainer.getDatabaseName()) 28 | .user(postgreSQLContainer.getUsername()) 29 | .password(postgreSQLContainer.getPassword()) 30 | .table("test") 31 | .dimension(embeddingModel.dimension()) 32 | .build(); 33 | 34 | TextSegment segment1 = TextSegment.from("I like football."); 35 | Embedding embedding1 = embeddingModel.embed(segment1).content(); 36 | embeddingStore.add(embedding1, segment1); 37 | 38 | TextSegment segment2 = TextSegment.from("The weather is good today."); 39 | Embedding embedding2 = embeddingModel.embed(segment2).content(); 40 | embeddingStore.add(embedding2, segment2); 41 | 42 | Embedding queryEmbedding = embeddingModel.embed("What is your favourite sport?").content(); 43 | 44 | EmbeddingSearchRequest embeddingSearchRequest = EmbeddingSearchRequest.builder() 45 | .queryEmbedding(queryEmbedding) 46 | .maxResults(1) 47 | .build(); 48 | 49 | List> relevant = embeddingStore.search(embeddingSearchRequest).matches(); 50 | 51 | EmbeddingMatch embeddingMatch = relevant.get(0); 52 | 53 | System.out.println(embeddingMatch.score()); 54 | System.out.println(embeddingMatch.embedded().text()); // I like football. 55 | 56 | postgreSQLContainer.stop(); 57 | } 58 | } 59 | } -------------------------------------------------------------------------------- /src/main/java/actor/DemoMessagesActor.java: -------------------------------------------------------------------------------- 1 | package actor; 2 | 3 | 4 | import org.apache.pekko.Done; 5 | import org.apache.pekko.actor.*; 6 | import org.apache.pekko.pattern.Patterns; 7 | 8 | import java.time.Duration; 9 | import java.util.concurrent.CompletionStage; 10 | 11 | /** 12 | * Untyped Java Actor example 13 | * Doc: 14 | * https://doc.akka.io/docs/akka/current/actors.html 15 | * https://www.baeldung.com/java-completablefuture 16 | */ 17 | public class DemoMessagesActor extends AbstractLoggingActor { 18 | 19 | public static void main(String[] args) { 20 | final ActorSystem system = ActorSystem.create("DemoMessagesActor"); 21 | ActorRef demoActor = system.actorOf(DemoMessagesActor.props(0), "demo"); 22 | system.scheduler() 23 | .scheduleWithFixedDelay(Duration.ofMillis(1000), Duration.ofMillis(1000), demoActor, new GreetingTell("Hi tell (scheduled)"), system.dispatcher(), null); 24 | 25 | CoordinatedShutdown.get(system) 26 | .addTask( 27 | CoordinatedShutdown.PhaseBeforeServiceUnbind(), 28 | "stop", 29 | () -> Patterns.ask(demoActor, new Stop(), Duration.ofSeconds(5)) 30 | .thenApply(reply -> Done.getInstance())); 31 | 32 | //Tell: Fire and forget 33 | demoActor.tell(new GreetingTell("Hi tell"), ActorRef.noSender()); 34 | 35 | //Ask: Wait for answer 36 | final CompletionStage future = Patterns.ask(demoActor, new GreetingAsk("Hi ask"), Duration.ofMillis(1000)); 37 | future.thenAccept(result -> System.out.println("DemoMessagesActor response on Ask: " + result)); 38 | } 39 | 40 | static public class GreetingTell { 41 | private final String from; 42 | 43 | public GreetingTell(String from) { 44 | this.from = from; 45 | } 46 | 47 | public String getGreeter() { 48 | return from; 49 | } 50 | } 51 | 52 | static public class GreetingAsk { 53 | private final String from; 54 | 55 | public GreetingAsk(String from) { 56 | this.from = from; 57 | } 58 | 59 | public String getGreeter() { 60 | return from; 61 | } 62 | } 63 | 64 | static public class Stop { 65 | public Stop() { 66 | } 67 | } 68 | 69 | /** 70 | * Create Props for an actor of this type. 71 | * 72 | * @param initValue The initial value for the counterTell is passed to this actor’s constructor. 73 | * @return a Props for creating this actor, which can then be further configured 74 | * (e.g. calling `.withDispatcher()` on it) 75 | */ 76 | static Props props(Integer initValue) { 77 | // You need to specify the actual type of the returned actor 78 | // since Java 8 lambdas have some runtime type information erased 79 | return Props.create(DemoMessagesActor.class, () -> new DemoMessagesActor(initValue)); 80 | } 81 | 82 | private Integer counterTell; 83 | 84 | private DemoMessagesActor(Integer initValue) { 85 | this.counterTell = initValue; 86 | } 87 | 88 | @Override 89 | public Receive createReceive() { 90 | return receiveBuilder() 91 | .match(GreetingTell.class, g -> { 92 | counterTell++; 93 | log().info("I was greeted by tell {}, {} times", g.getGreeter(), counterTell); 94 | }) 95 | .match(GreetingAsk.class, g -> { 96 | log().info("I was greeted by {}", g.getGreeter()); 97 | getSender().tell("OK", getSelf()); 98 | }) 99 | .match(Stop.class, s -> { 100 | log().info("I was stopped"); 101 | getSender().tell("OK", getSelf()); 102 | }) 103 | .build(); 104 | } 105 | } -------------------------------------------------------------------------------- /src/main/java/actor/HelloWorld.java: -------------------------------------------------------------------------------- 1 | package actor; 2 | 3 | import org.apache.pekko.actor.typed.ActorRef; 4 | import org.apache.pekko.actor.typed.Behavior; 5 | import org.apache.pekko.actor.typed.javadsl.AbstractBehavior; 6 | import org.apache.pekko.actor.typed.javadsl.ActorContext; 7 | import org.apache.pekko.actor.typed.javadsl.Behaviors; 8 | import org.apache.pekko.actor.typed.javadsl.Receive; 9 | 10 | public class HelloWorld extends AbstractBehavior { 11 | 12 | public static final class Greet { 13 | public final String whom; 14 | public final ActorRef replyTo; 15 | 16 | public Greet(String whom, ActorRef replyTo) { 17 | this.whom = whom; 18 | this.replyTo = replyTo; 19 | } 20 | } 21 | 22 | public static final class Greeted { 23 | public final String whom; 24 | public final ActorRef from; 25 | 26 | public Greeted(String whom, ActorRef from) { 27 | this.whom = whom; 28 | this.from = from; 29 | } 30 | } 31 | 32 | public static Behavior create() { 33 | return Behaviors.setup(HelloWorld::new); 34 | } 35 | 36 | private HelloWorld(ActorContext context) { 37 | super(context); 38 | } 39 | 40 | @Override 41 | public Receive createReceive() { 42 | return newReceiveBuilder().onMessage(Greet.class, this::onGreet).build(); 43 | } 44 | 45 | private Behavior onGreet(Greet command) { 46 | getContext().getLog().info("Hello {}!", command.whom); 47 | command.replyTo.tell(new Greeted(command.whom, getContext().getSelf())); 48 | return this; 49 | } 50 | } -------------------------------------------------------------------------------- /src/main/java/actor/HelloWorldBot.java: -------------------------------------------------------------------------------- 1 | package actor; 2 | 3 | import org.apache.pekko.actor.typed.Behavior; 4 | import org.apache.pekko.actor.typed.javadsl.AbstractBehavior; 5 | import org.apache.pekko.actor.typed.javadsl.ActorContext; 6 | import org.apache.pekko.actor.typed.javadsl.Behaviors; 7 | import org.apache.pekko.actor.typed.javadsl.Receive; 8 | 9 | public class HelloWorldBot extends AbstractBehavior { 10 | 11 | public static Behavior create(int max) { 12 | return Behaviors.setup(context -> new HelloWorldBot(context, max)); 13 | } 14 | 15 | private final int max; 16 | private int greetingCounter; 17 | 18 | private HelloWorldBot(ActorContext context, int max) { 19 | super(context); 20 | this.max = max; 21 | } 22 | 23 | @Override 24 | public Receive createReceive() { 25 | return newReceiveBuilder().onMessage(HelloWorld.Greeted.class, this::onGreeted).build(); 26 | } 27 | 28 | private Behavior onGreeted(HelloWorld.Greeted message) { 29 | greetingCounter++; 30 | getContext().getLog().info("Greeting {} for {}", greetingCounter, message.whom); 31 | if (greetingCounter == max) { 32 | return Behaviors.stopped(); 33 | } else { 34 | message.from.tell(new HelloWorld.Greet(message.whom, getContext().getSelf())); 35 | return this; 36 | } 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/actor/HelloWorldMain.java: -------------------------------------------------------------------------------- 1 | package actor; 2 | 3 | import org.apache.pekko.actor.typed.*; 4 | import org.apache.pekko.actor.typed.javadsl.AbstractBehavior; 5 | import org.apache.pekko.actor.typed.javadsl.ActorContext; 6 | import org.apache.pekko.actor.typed.javadsl.Behaviors; 7 | import org.apache.pekko.actor.typed.javadsl.Receive; 8 | 9 | /** 10 | * Akka Typed example 1:1 11 | * Doc: 12 | * https://doc.akka.io/docs/akka/current/typed/actors.html 13 | * 14 | */ 15 | public class HelloWorldMain extends AbstractBehavior { 16 | 17 | public static void main(String[] args) throws Exception { 18 | final ActorSystem system = 19 | ActorSystem.create(HelloWorldMain.create(), "hello"); 20 | 21 | system.tell(new HelloWorldMain.Start("World")); 22 | system.tell(new HelloWorldMain.Start("Akka")); 23 | 24 | Thread.sleep(3000); 25 | system.terminate(); 26 | } 27 | 28 | 29 | // Start message... 30 | public static class Start { 31 | public final String name; 32 | 33 | public Start(String name) { 34 | this.name = name; 35 | } 36 | } 37 | 38 | public static Behavior create() { 39 | return Behaviors.setup(HelloWorldMain::new); 40 | } 41 | 42 | private final ActorRef greeter; 43 | 44 | private HelloWorldMain(ActorContext context) { 45 | super(context); 46 | 47 | final String dispatcherPath = "akka.actor.default-blocking-io-dispatcher"; 48 | Props greeterProps = DispatcherSelector.fromConfig(dispatcherPath); 49 | greeter = getContext().spawn(HelloWorld.create(), "greeter", greeterProps); 50 | } 51 | 52 | @Override 53 | public Receive createReceive() { 54 | return newReceiveBuilder().onMessage(Start.class, this::onStart).build(); 55 | } 56 | 57 | private Behavior onStart(Start command) { 58 | ActorRef replyTo = 59 | getContext().spawn(HelloWorldBot.create(3), command.name); 60 | greeter.tell(new HelloWorld.Greet(command.name, replyTo)); 61 | return this; 62 | } 63 | } -------------------------------------------------------------------------------- /src/main/resources/63MB.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pbernet/akka_streams_tutorial/4f42aa87ccae8afdf0270d0106ae97996623f4d9/src/main/resources/63MB.pdf -------------------------------------------------------------------------------- /src/main/resources/ADT_ORM_Hl7Messages.txt: -------------------------------------------------------------------------------- 1 | MSH|^~\&|1|MCM|LABADT|MCM|198808181126|SECURITY|ADT^A01|NIST-IZ-007.00|P|2.5.1| 2 | EVN|A01|198808181123|| 3 | PID|||PATID1234^5^M11^ADT1^MR^MCM~123456789^^^USSSA^SS||EVERYMAN^ADAM^A^III||19610615|M||C|1200 N ELM STREET^^GREENSBORO^NC^27401-1020 4 | NK1|1|JONES^BARBARA^K|SPO^Spouse^HL70063|171 ZOBERLEIN^^ISHPEMING^MI^49849^| 5 | PV1|1|I|2000^2012^01||||004777^LEBAUER^SIDNEY^J.|||SUR||||9|A0| 6 | 7 | MSH|^~\&|2|DH|LABADT|DH|201301011226||ADT^A01|NIST-IZ-007.00|P|2.5.1| 8 | EVN|A01|201301011223|| 9 | PID|||MRN12345^5^M11||APPLESEED^JOHN^A^III||19710101|M||C|1 DATICA STREET^^MADISON^WI^53005-1020|GL|(414)379-1212|(414)271-3434||S||MRN12345001^2^M10|123456789|987654^NC| 10 | NK1|1|JONES^BARBARA^K|SPO^Spouse^HL70063|171 ZOBERLEIN^^ISHPEMING^MI^49849^| 11 | PV1|1|I|2000^2012^01||||004777^GOOD^SIDNEY^J.|||SUR||||ADM|A0c 12 | 13 | MSH|^~\&|3|EPIC|||20140418173314|1148|ORM^O01|NIST-IZ-007.00|D|2.5.1| 14 | PID|1||20891312^^^^EPI||APPLESEED^JOHN^A^^MR.^||19661201|M||AfrAm|505 S. HAMILTON AVE^^MADISON^WI^53505^US^^^DN |DN|(608)123-4567|(608)123-5678||S|| 11480003|123-45-7890||||^^^WI^^ 15 | PD1|||FACILITY(EAST)^^12345|1173^MATTHEWS^JAMES^A^^^ 16 | PV1|||^^^CARE HEALTH SYSTEMS^^^^^||| |1173^MATTHEWS^JAMES^A^^^||||||||||||610613||||||||||||||||||||||||||||||||V 17 | ORC|NW|987654^EPIC|76543^EPC||Final||^^^20140418170014^^^^||20140418173314|1148^PATTERSON^JAMES^^^^||1173^MATTHEWS^JAMES^A^^^|1133^^^222^^^^^|(618)222-1122|| 18 | OBR|1|363463^EPC|1858^EPC|73610^X-RAY ANKLE 3+ VW^^^X-RAY ANKLE ||||||||||||1173^MATTHEWS^JAMES^A^^^|(608)258-8866||||||||Final||^^^20140418170014^^^^|||||6064^20110614075841^20110614075841^^^^||1148010^20110614075841^20110614075841^X-RAY^^^|^| 19 | DG1||I10|S82^ANKLE FRACTURE^I10|ANKLE FRACTURE|| 20 | 21 | MSH|^~\&|4|X68||NIST Test Iz Reg|201207010822||VXU^V04^VXU_V04|NIST-IZ-007.00|P|2.5.1|||AL|ER 22 | PID|1||MR-99922^^^NIST MPI^MR||Montgomery^Lewis^^^^^L||20010821|M 23 | ORC|RE||IZ-783276^NDA 24 | RXA|0|1|20110215||118^HPV^CVX|999|||01^Historical information - source unspecified^NIP001 25 | 26 | MSH|^~\&|5|X68||NIST Test Iz Reg|201207010822||VXU^V04^VXU_V04|NIST-IZ-007.00|P|2.5.1|||AL|ER 27 | PID|1||MR-99922^^^NIST MPI^MR||Montgomery^Lewis^^^^^L||20010821|M 28 | ORC|RE||IZ-783276^NDA 29 | RXA|0|1|20110215||118^HPV^CVX|999|||01^Historical information - source unspecified^NIP001 30 | -------------------------------------------------------------------------------- /src/main/resources/EN_challenges.srt: -------------------------------------------------------------------------------- 1 | 1 2 | 00:00:01,319 --> 00:00:02,500 3 | [breeze blowing faintly] 4 | 5 | 2 6 | 00:01:09,319 --> 00:01:12,500 7 | Mrs. Price, you can't just show up 8 | unannounced in the middle of a semester, 9 | 10 | 3 11 | 00:01:13,000 --> 00:01:14,658 12 | and remove your son from the school. 13 | 14 | 4 15 | 00:01:15,000 --> 00:01:16,658 16 | This is not acceptable. 17 | 18 | 5 19 | 00:01:20,786 --> 00:01:20,999 20 | Someone so bright and promising, 21 | 22 | 6 23 | 00:01:21,470 --> 00:01:22,139 24 | killing themselves 25 | after the glamour has passed. 26 | 27 | 7 28 | 00:50:07,883 --> 00:50:10,093 29 | I've been feeling better 30 | in the last couple weeks, 31 | 32 | 8 33 | 00:50:10,177 --> 00:50:12,679 34 | and I'd appreciate 35 | you keeping some distance. 36 | 37 | 9 38 | 01:23:35,098 --> 01:23:37,267 39 | We allow ourselves contentment... 40 | 41 | 10 42 | 01:23:38,000 --> 01:23:41,000 43 | and the heart brings us ease 44 | in its good time. 45 | 46 | 11 47 | 01:30:45,710 --> 01:30:47,668 48 | So G-goo came on his own, 49 | 50 | 12 51 | 01:30:48,000 --> 01:30:48,001 52 | wanting a refund of a prepaid 53 | card he bought last year? 54 | 55 | 12 56 | 01:30:49,000 --> 01:30:50,001 57 | Mommy and Daddy will be right next to you and 58 | there may be some organ music as the curtain opens. 59 | 60 | 12 61 | 01:31:48,000 --> 01:31:48,001 62 | The shiny new white and big house will be ready for us 63 | faster than you can say "Jack Robinson" 64 | 65 | 13 66 | 01:32:45,710 --> 01:32:45,715 67 | But I told him I'd ask my boss, 68 | 69 | 14 70 | 01:32:46,710 --> 01:36:46,710 71 | and then I texted you, Detective. 72 | 73 | 100 74 | 02:09:47,919 --> 02:09:50,018 75 | - ALISTAIR: Okay. 76 | - So much love to everybody. 77 | 78 | 101 79 | 02:09:50,053 --> 02:09:51,459 80 | ALISTAIR: Love to you all. 81 | 82 | 102 83 | 02:29:52,529 --> 02:29:53,858 84 | - Good morning. 85 | - Morning. 86 | 87 | 103 88 | 02:29:58,634 --> 02:30:00,436 89 | - There. 90 | - Thank you. 91 | 92 | 200 93 | 03:03:10,382 --> 03:03:14,350 94 | - Samuel? 95 | - Sam. 96 | 97 | 201 98 | 03:03:14,586 --> 03:03:17,120 99 | Sam, we're here 100 | 101 | 202 102 | 03:03:23,194 --> 03:03:25,264 103 | Here. OK! 104 | 105 | 203 106 | 03:06:28,213 --> 03:06:30,182 107 | Dad. 108 | 109 | 204 110 | 03:06:48,867 --> 03:06:52,838 111 | Dad 112 | 113 | 964 114 | 04:00:55,062 --> 04:00:56,397 115 | - Okay? You decided. 116 | - Mm-hmm. 117 | 118 | 965 119 | 04:01:29,591 --> 04:01:31,693 120 | - of-of clowns and acrobats. 121 | - And elephants. 122 | 123 | 966 124 | 04:01:31,726 --> 04:01:34,294 125 | -Uh, "projecting" means it sends them out. 126 | -Happy things. -------------------------------------------------------------------------------- /src/main/resources/SSEClientWikipediaEdits.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | SSE Client for WikipediaEdits 6 | 7 | 8 | 9 |

Check the browser console log...

10 | Chrome is able to display the sse stream 11 | 12 | 22 | 23 | 24 | -------------------------------------------------------------------------------- /src/main/resources/SampleRoutes.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | Sample routes for akka-http 6 | 7 | 8 | entries 9 |
10 | faultyActor 11 |
12 |
13 |
14 |
15 |
16 |
17 | 18 |
19 | 20 | -------------------------------------------------------------------------------- /src/main/resources/broker.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 7 | 8 | 9 | true 10 | NIO 11 | 10 12 | 13 | 14 | 15 | vm://0 16 | tcp://0.0.0.0:21616 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 |
36 | 37 | 38 | 39 | true 40 | 41 | 42 |
43 |
44 | 45 | 46 | 47 | jms.queue.deadLetterQueue 48 | 3 49 | 5000 50 | jms.queue.expiryQueue 51 | true 52 | 100000 53 | 20000 54 | 0 55 | true 56 | PAGE 57 | -1 58 | NOTIFY 59 | 5 60 | true 61 | 62 | 63 | 64 | 65 |
66 |
-------------------------------------------------------------------------------- /src/main/resources/en-ner-person.bin: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pbernet/akka_streams_tutorial/4f42aa87ccae8afdf0270d0106ae97996623f4d9/src/main/resources/en-ner-person.bin -------------------------------------------------------------------------------- /src/main/resources/jndi.properties: -------------------------------------------------------------------------------- 1 | java.naming.factory.initial=org.apache.activemq.artemis.jndi.ActiveMQInitialContextFactory 2 | connectionFactory.ConnectionFactory=vm://0 3 | queue.queue/bootstrapTestQueueJNDI=queue/bootstrapTestQueueJNDI -------------------------------------------------------------------------------- /src/main/resources/line_protocol_data.txt: -------------------------------------------------------------------------------- 1 | testMemLP,sensorID=sensorLP,testTime=1,nPoints=N/A hr=1i 2 | testMemLP,sensorID=sensorLP,testTime=2,nPoints=N/A hr=2i 3 | testMemLP,sensorID=sensorLP,testTime=3,nPoints=N/A hr=3i 4 | testMemLP,sensorID=sensorLP,testTime=4,nPoints=N/A hr=4i 5 | testMemLP,sensorID=sensorLP,testTime=5,nPoints=N/A hr=5i 6 | testMemLP,sensorID=sensorLP,testTime=6,nPoints=N/A hr=6i 7 | testMemLP,sensorID=sensorLP,testTime=7,nPoints=N/A hr=7i 8 | testMemLP,sensorID=sensorLP,testTime=8,nPoints=N/A hr=8i 9 | testMemLP,sensorID=sensorLP,testTime=9,nPoints=N/A hr=9i 10 | testMemLP,sensorID=sensorLP,testTime=10,nPoints=N/A hr=10i -------------------------------------------------------------------------------- /src/main/resources/localstack/init_dynamodb.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | table_name="testTable" 4 | hash_key="Id" 5 | 6 | # Create DynamoDB table, not used because we create it via AWS SDK 7 | #awslocal dynamodb create-table \ 8 | # --table-name "$table_name" \ 9 | # --key-schema AttributeName="$hash_key",KeyType=HASH \ 10 | # --attribute-definitions AttributeName="$hash_key",AttributeType=S \ 11 | # --billing-mode PAY_PER_REQUEST 12 | 13 | # echo "DynamoDB table '$table_name' created successfully with hash key '$hash_key'" 14 | echo "Executed init_dynamodb.sh" -------------------------------------------------------------------------------- /src/main/resources/localstack/init_firehose.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | # Setup Elasticsearch 4 | # Default Elasticsearch endpoint: "es-local.us-east-1.es.localhost.localstack.cloud:4566" 5 | awslocal es create-elasticsearch-domain --domain-name es-local 6 | 7 | # Setup S3 8 | awslocal s3 mb s3://kinesis-activity-backup-local 9 | 10 | # Setup Firehose 11 | awslocal firehose create-delivery-stream --delivery-stream-name activity-to-elasticsearch-local --elasticsearch-destination-configuration "RoleARN=arn:aws:iam::000000000000:role/Firehose-Reader-Role,DomainARN=arn:aws:es:us-east-1:000000000000:domain/es-local,IndexName=activity,TypeName=activity,S3BackupMode=AllDocuments,S3Configuration={RoleARN=arn:aws:iam::000000000000:role/Firehose-Reader-Role,BucketARN=arn:aws:s3:::kinesis-activity-backup-local}" 12 | 13 | echo "Wait for Elasticsearch setup to complete..." -------------------------------------------------------------------------------- /src/main/resources/localstack/init_kinesis.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | awslocal kinesis create-stream --stream-name kinesisDataStreamProvisioned --shard-count 1 4 | echo "Initialized." -------------------------------------------------------------------------------- /src/main/resources/localstack/init_sqs.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | awslocal sqs create-queue --queue-name mysqs-queue --region us-east-1 4 | echo "Initialized AWS standard queue." -------------------------------------------------------------------------------- /src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | %d{HH:mm:ss.SSS} | %-10.10thread | %5p | %-25.25logger{25} | %m%n 7 | utf8 8 | 9 | 10 | 11 | logs/application.log 12 | false 13 | 14 | %d{yyyy-MM-dd} | %d{HH:mm:ss.SSS} | %-20.20thread | %5p | %-25.25logger{25} | %m%n 15 | utf8 16 | 17 | 18 | 19 | 20 | 8192 21 | true 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 38 | 39 | 40 | 41 | 42 | 43 | -------------------------------------------------------------------------------- /src/main/resources/opennlp-en-ud-ewt-tokens-1.0-1.9.3.bin: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pbernet/akka_streams_tutorial/4f42aa87ccae8afdf0270d0106ae97996623f4d9/src/main/resources/opennlp-en-ud-ewt-tokens-1.0-1.9.3.bin -------------------------------------------------------------------------------- /src/main/resources/opennlp-en-ud-ewt-tokens-1.2-2.5.0.bin: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pbernet/akka_streams_tutorial/4f42aa87ccae8afdf0270d0106ae97996623f4d9/src/main/resources/opennlp-en-ud-ewt-tokens-1.2-2.5.0.bin -------------------------------------------------------------------------------- /src/main/resources/payload.zip: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pbernet/akka_streams_tutorial/4f42aa87ccae8afdf0270d0106ae97996623f4d9/src/main/resources/payload.zip -------------------------------------------------------------------------------- /src/main/resources/testfile.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pbernet/akka_streams_tutorial/4f42aa87ccae8afdf0270d0106ae97996623f4d9/src/main/resources/testfile.jpg -------------------------------------------------------------------------------- /src/main/scala/actor/BlockingActor.scala: -------------------------------------------------------------------------------- 1 | package actor 2 | 3 | import org.apache.pekko.actor.typed.Behavior 4 | import org.apache.pekko.actor.typed.scaladsl.Behaviors 5 | 6 | object BlockingActor { 7 | def apply(): Behavior[Int] = 8 | Behaviors.receive { (context, i) => 9 | context.log.info(s"Started: $i by ${Thread.currentThread().getName}") 10 | //block for 5 seconds, representing blocking I/O, etc 11 | Thread.sleep(5000) 12 | context.log.info(s"Finished: $i") 13 | Behaviors.same 14 | } 15 | } -------------------------------------------------------------------------------- /src/main/scala/actor/BlockingRight.scala: -------------------------------------------------------------------------------- 1 | package actor 2 | 3 | import org.apache.pekko.actor.typed.scaladsl.Behaviors 4 | import org.apache.pekko.actor.typed.{ActorSystem, DispatcherSelector} 5 | 6 | /** 7 | * If blocking (eg by an external resource) is required, 8 | * a custom dispatcher (see application.conf) avoids 9 | * thread starvation of the default dispatcher 10 | * 11 | * Stolen from: 12 | * https://github.com/raboof/akka-blocking-dispatcher 13 | * 14 | * See [[sample.stream.WaitForFlowsToComplete]] for use of custom dispatcher in a stream 15 | * 16 | */ 17 | object BlockingRight extends App { 18 | 19 | val root = Behaviors.setup[Nothing] { context => 20 | (1 to 50).foreach { i => 21 | // non blocking actor running on default-dispatcher 22 | context.spawn(PrintActor(), s"nonblocking-$i") ! i 23 | // blocking actor running on custom-dispatcher 24 | context.spawn( 25 | BlockingActor(), 26 | s"blocking-$i", 27 | DispatcherSelector.fromConfig("custom-dispatcher-for-blocking") 28 | ) ! i 29 | } 30 | 31 | Behaviors.empty 32 | } 33 | 34 | val system = ActorSystem[Nothing](root, "BlockingRight") 35 | } 36 | -------------------------------------------------------------------------------- /src/main/scala/actor/Buncher.scala: -------------------------------------------------------------------------------- 1 | package actor 2 | 3 | import org.apache.pekko.actor.typed.scaladsl.Behaviors 4 | import org.apache.pekko.actor.typed.{ActorRef, ActorSystem, Behavior} 5 | 6 | import scala.collection.immutable 7 | import scala.concurrent.duration.* 8 | 9 | /** 10 | * This akka typed actor FSM example demonstrates how to: 11 | * - Model states using different behaviors 12 | * - Model storing data at each state by representing the behavior as a method 13 | * - Implement state timeouts 14 | * 15 | * Doc: 16 | * https://doc.akka.io/docs/akka/current/typed/fsm.html 17 | */ 18 | object Buncher extends App { 19 | 20 | val root = Behaviors.setup[Nothing] { context => 21 | val buncherActor = context.spawn(Buncher(), "buncherActor") 22 | val targetActor = context.spawn(TargetActor(), "targetActor") 23 | 24 | buncherActor ! SetTarget(targetActor) 25 | buncherActor ! Queue(42) 26 | buncherActor ! Queue("abc") 27 | Thread.sleep(2000) //Provoke flushing by Timeout 28 | buncherActor ! Queue(43) 29 | buncherActor ! Queue(44) 30 | buncherActor ! Flush 31 | buncherActor ! Queue(45) 32 | 33 | Behaviors.empty 34 | } 35 | 36 | val system = ActorSystem[Nothing](root, "Buncher") 37 | 38 | 39 | // received events: the type of the message this Actor supports 40 | sealed trait Event 41 | final case class SetTarget(ref: ActorRef[Batch]) extends Event 42 | final case class Queue(obj: Any) extends Event 43 | case object Flush extends Event 44 | private case object Timeout extends Event 45 | 46 | // sent event: to targetActor 47 | final case class Batch(obj: immutable.Seq[Any]) 48 | 49 | // internal data 50 | sealed trait Data 51 | case object Uninitialized extends Data 52 | final case class Todo(target: ActorRef[Batch], queue: immutable.Seq[Any]) extends Data 53 | 54 | 55 | // initial state 56 | def apply(): Behavior[Event] = idle(Uninitialized) 57 | 58 | // states of the FSM represented as behaviors 59 | 60 | private def idle(data: Data): Behavior[Event] = Behaviors.receiveMessage[Event] { message => 61 | (message, data) match { 62 | case (SetTarget(ref), Uninitialized) => 63 | // action: add obj to queue 64 | // transition: to the state idle 65 | idle(Todo(ref, Vector.empty)) 66 | case (Queue(obj), t @ Todo(_, queue)) => 67 | // action: add obj to vector 68 | // transition: to the state active 69 | active(t.copy(queue = queue :+ obj)) 70 | case _ => 71 | Behaviors.unhandled 72 | } 73 | } 74 | 75 | private def active(data: Todo): Behavior[Event] = 76 | Behaviors.withTimers[Event] { timers => 77 | // send msg Timeout after 1 sec (instead of FSM state timeout) 78 | timers.startSingleTimer(Timeout, 1.second) 79 | Behaviors.receiveMessagePartial { 80 | case Flush | Timeout => 81 | // action: send queue to targetActor 82 | // transition: to the state idle with empty queue 83 | data.target ! Batch(data.queue) 84 | idle(data.copy(queue = Vector.empty)) 85 | case Queue(obj) => 86 | // action: add obj to queue 87 | // transition: to the state active 88 | active(data.copy(queue = data.queue :+ obj)) 89 | } 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /src/main/scala/actor/FaultyActor.scala: -------------------------------------------------------------------------------- 1 | package actor 2 | 3 | import actor.FaultyActor.DoIt 4 | import akkahttp.SampleRoutes.FaultyActorResponse 5 | import org.apache.pekko.actor.{Actor, Status} 6 | import org.slf4j.{Logger, LoggerFactory} 7 | 8 | import java.util.concurrent.ThreadLocalRandom 9 | 10 | object FaultyActor { 11 | case class DoIt() 12 | } 13 | 14 | class FaultyActor extends Actor { 15 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 16 | var totalAttempts: Int = 0 17 | 18 | override def receive: Receive = { 19 | case DoIt() => 20 | totalAttempts = totalAttempts + 1 21 | 22 | // TODO Add processing with Future 23 | // https://medium.com/@linda0511ny/error-handling-in-akka-actor-with-future-ded3da0579dd 24 | val randomTime = ThreadLocalRandom.current.nextInt(0, 5) * 100 25 | logger.info(s"Attempt: $totalAttempts - Working for: $randomTime ms") 26 | 27 | val start = System.currentTimeMillis() 28 | while ((System.currentTimeMillis() - start) < randomTime) { 29 | // This restarts the actor 30 | if (randomTime >= 300) throw new RuntimeException("BOOM - server failure") 31 | } 32 | sender() ! FaultyActorResponse(totalAttempts) 33 | } 34 | 35 | override def preRestart(reason: Throwable, message: Option[Any]): Unit = { 36 | logger.error(s"Failed with original failure: $reason") 37 | super.preRestart(reason, message) 38 | sender() ! Status.Failure(reason) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/scala/actor/PrintActor.scala: -------------------------------------------------------------------------------- 1 | package actor 2 | 3 | import org.apache.pekko.actor.typed.Behavior 4 | import org.apache.pekko.actor.typed.scaladsl.Behaviors 5 | 6 | object PrintActor { 7 | def apply(): Behavior[Integer] = 8 | Behaviors.receive { (ctx, i) => 9 | ctx.log.info(s"Finished: $i by ${Thread.currentThread().getName}") 10 | Behaviors.same 11 | } 12 | } -------------------------------------------------------------------------------- /src/main/scala/actor/TargetActor.scala: -------------------------------------------------------------------------------- 1 | package actor 2 | 3 | import org.apache.pekko.actor.typed.Behavior 4 | import org.apache.pekko.actor.typed.scaladsl.Behaviors 5 | 6 | object TargetActor { 7 | def apply(): Behavior[Buncher.Batch] = 8 | Behaviors.receive { (ctx, i) => 9 | ctx.log.info(i.toString) 10 | Behaviors.same 11 | } 12 | } -------------------------------------------------------------------------------- /src/main/scala/akkahttp/HttpClientSingleRequest.scala: -------------------------------------------------------------------------------- 1 | package akkahttp 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.http.scaladsl.Http 5 | import org.apache.pekko.http.scaladsl.model.{HttpRequest, HttpResponse} 6 | 7 | import scala.concurrent.Future 8 | import scala.util.{Failure, Success} 9 | 10 | /** 11 | * Http client for a single request from Doc: 12 | * https://doc.akka.io/docs/akka-http/current/client-side/request-level.html#request-level-client-side-api 13 | * 14 | */ 15 | object HttpClientSingleRequest extends App { 16 | implicit val system: ActorSystem = ActorSystem() 17 | 18 | import system.dispatcher 19 | 20 | val responseFuture: Future[HttpResponse] = Http().singleRequest(HttpRequest(uri = "https://akka.io")) 21 | responseFuture 22 | .onComplete { 23 | case Success(res) => 24 | // Even if we don’t care about the response entity, we must consume it 25 | res.entity.discardBytes() 26 | println(s"Success: ${res.status}") 27 | case Failure(ex) => sys.error(s"Something wrong: ${ex.getMessage}") 28 | } 29 | } -------------------------------------------------------------------------------- /src/main/scala/akkahttp/ServerWithCustomDispatcher.scala: -------------------------------------------------------------------------------- 1 | package akkahttp 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.http.scaladsl.Http 5 | import org.apache.pekko.http.scaladsl.model.{HttpResponse, StatusCodes} 6 | import org.apache.pekko.http.scaladsl.server.Directives.{complete, extractRequest, path, withExecutionContext} 7 | import org.apache.pekko.http.scaladsl.server.Route 8 | import org.slf4j.{Logger, LoggerFactory} 9 | 10 | import scala.concurrent.{ExecutionContextExecutor, Future} 11 | import scala.util.{Failure, Success} 12 | 13 | /** 14 | * By default, route directives run on the default dispatcher. 15 | * The `withExecutionContext` directive alone doesn't force execution on the custom dispatcher 16 | * it only makes the custom execution context available. 17 | * Wrapping a task in a Future explicitly runs the task on the custom execution context. 18 | * 19 | * Full example for this answer: 20 | * https://stackoverflow.com/questions/79141989/executioncontext-issue-in-akka-http-server/79145603#79145603 21 | */ 22 | object ServerWithCustomDispatcher extends App { 23 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 24 | implicit val system: ActorSystem = ActorSystem() 25 | 26 | implicit val myExCon: ExecutionContextExecutor = system.dispatchers.lookup( 27 | "custom-dispatcher-fork-join" 28 | ) 29 | 30 | val route: Route = { 31 | path("hello") { 32 | withExecutionContext(myExCon) { 33 | extractRequest { request => 34 | // Move the operation inside a Future to ensure it runs on the custom dispatcher 35 | val result = Future { 36 | logger.info(s"Got request from client: ${request.getHeader("User-Agent")}") 37 | val msg = s"Execution context: $myExCon with thread: ${Thread.currentThread.getName}" 38 | logger.info(msg) 39 | HttpResponse(StatusCodes.OK, entity = s"$msg") 40 | }(myExCon) 41 | complete(result) 42 | } 43 | } 44 | } 45 | } 46 | 47 | val bindingFuture = Http().newServerAt("localhost", 9000).bindFlow(route) 48 | 49 | bindingFuture.onComplete { 50 | case Success(b) => 51 | println("Server started, listening on: http://" + b.localAddress) 52 | case Failure(e) => 53 | println(s"Server could not bind to... Exception message: ${e.getMessage}") 54 | system.terminate() 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /src/main/scala/akkahttp/oidc/CORSHandler.scala: -------------------------------------------------------------------------------- 1 | package akkahttp.oidc 2 | 3 | import org.apache.pekko.http.scaladsl.model.HttpMethods.* 4 | import org.apache.pekko.http.scaladsl.model.headers.* 5 | import org.apache.pekko.http.scaladsl.model.{HttpResponse, StatusCodes} 6 | import org.apache.pekko.http.scaladsl.server.Directives.{complete, options, respondWithHeaders, *} 7 | import org.apache.pekko.http.scaladsl.server.{Directive0, Route} 8 | 9 | trait CORSHandler { 10 | 11 | private val corsResponseHeaders = List( 12 | `Access-Control-Allow-Origin`.*, 13 | `Access-Control-Allow-Credentials`(true), 14 | `Access-Control-Allow-Headers`("Authorization", "Content-Type", "X-Requested-With")) 15 | 16 | private def addAccessControlHeaders: Directive0 = { 17 | respondWithHeaders(corsResponseHeaders) 18 | } 19 | 20 | private def preflightRequestHandler: Route = options { 21 | complete(HttpResponse(StatusCodes.OK). 22 | withHeaders(`Access-Control-Allow-Methods`(OPTIONS, POST, PUT, GET, DELETE))) 23 | } 24 | 25 | def corsHandler(r: Route): Route = addAccessControlHeaders { 26 | preflightRequestHandler ~ r 27 | } 28 | 29 | def addCORSHeaders(response: HttpResponse): HttpResponse = 30 | response.withHeaders(corsResponseHeaders) 31 | } -------------------------------------------------------------------------------- /src/main/scala/akkahttp/oidc/JsonSupport.scala: -------------------------------------------------------------------------------- 1 | package akkahttp.oidc 2 | 3 | import io.circe.* 4 | import io.circe.generic.semiauto.{deriveDecoder, deriveEncoder} 5 | 6 | trait JsonSupport { 7 | 8 | case class Keys(keys: Seq[KeyData]) 9 | case class KeyData(kid: String, n: String, e: String) 10 | 11 | implicit val keyDataEncoder: Encoder[KeyData] = deriveEncoder[KeyData] 12 | implicit val keyDataDecoder: Decoder[KeyData] = deriveDecoder[KeyData] 13 | 14 | implicit val keysEncoder: Encoder[Keys] = deriveEncoder[Keys] 15 | implicit val keysDecoder: Decoder[Keys] = deriveDecoder[Keys] 16 | 17 | case class UserKeycloak(firstName: Option[String], lastName: Option[String], email: Option[String]) 18 | 19 | implicit val userEncoder: Encoder[UserKeycloak] = deriveEncoder[UserKeycloak] 20 | implicit val userDecoder: Decoder[UserKeycloak] = deriveDecoder[UserKeycloak] 21 | 22 | case class UsersKeycloak(users: Seq[UserKeycloak]) 23 | 24 | implicit val usersEncoder: Encoder[UsersKeycloak] = deriveEncoder[UsersKeycloak] 25 | implicit val usersDecoder: Decoder[UsersKeycloak] = deriveDecoder[UsersKeycloak] 26 | 27 | } 28 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/clickhousedb/ClickhouseDB.scala: -------------------------------------------------------------------------------- 1 | package alpakka.clickhousedb 2 | 3 | import com.crobox.clickhouse.ClickhouseClient 4 | import com.crobox.clickhouse.stream.{ClickhouseSink, Insert} 5 | import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory} 6 | import org.apache.pekko.Done 7 | import org.apache.pekko.actor.ActorSystem 8 | import org.apache.pekko.stream.scaladsl.{Framing, Sink, Source} 9 | import org.apache.pekko.util.ByteString 10 | import org.slf4j.{Logger, LoggerFactory} 11 | 12 | import scala.concurrent.duration.DurationInt 13 | import scala.concurrent.{Await, Future} 14 | import scala.util.Try 15 | 16 | 17 | /** 18 | * DB access via the Scala client 19 | * Run with integration test: alpakka.clickhousedb.ClickhousedbIT 20 | * 21 | * We use the format `JSONEachRow` 22 | * https://clickhouse.com/docs/en/interfaces/formats#jsoneachrow 23 | * All possible formats: 24 | * https://clickhouse.com/docs/en/interfaces/formats 25 | */ 26 | class ClickhouseDB(httpPort: Int) { 27 | private val logger: Logger = LoggerFactory.getLogger(this.getClass) 28 | 29 | implicit val system: ActorSystem = ActorSystem() 30 | 31 | import system.dispatcher 32 | 33 | // Tweak config with mapped HTTP port from container 34 | val tweakedConf: Config = ConfigFactory.empty() 35 | .withValue("crobox.clickhouse.client.connection.port", ConfigValueFactory.fromAnyRef(httpPort)) 36 | .withFallback(ConfigFactory.load()) 37 | 38 | val client = new ClickhouseClient(Some[Config](tweakedConf)) 39 | logger.info(s"Connected to server version: ${client.serverVersion}") 40 | 41 | def testRead(): String = { 42 | val result = Await.result(client.query("SELECT 1"), 10.seconds) 43 | logger.info(s"Got query result: $result") 44 | result.trim 45 | } 46 | 47 | def writeAll(noOfRecords: Integer): Future[Done] = { 48 | Source(1 to noOfRecords) 49 | .map(id => Insert("test.my_table", s"{\"myfloat_nullable\": $id, \"mystr\": $id, \"myint_id\": $id}")) 50 | .wireTap((insert: Insert) => logger.debug(s"Insert record with type JSONEachRow: $insert")) 51 | .runWith(ClickhouseSink.toSink(tweakedConf, client)) 52 | } 53 | 54 | // The most intuitive way to read the streamed records 55 | def readAllSource(): Future[Int] = { 56 | val resultFut = client.source("SELECT * FROM test.my_table ORDER BY myint_id ASC FORMAT JSONEachRow SETTINGS output_format_json_named_tuples_as_objects=1;") 57 | .wireTap((line: String) => logger.debug(s"Raw JSON record: $line")) 58 | .runWith(Sink.seq) 59 | 60 | resultFut.flatMap(each => Future(each.size)) 61 | } 62 | 63 | // An alternative way to read, allows for more control, eg while massaging the result 64 | def readAllSourceByteString(): Future[Int] = { 65 | val resultFut = client.sourceByteString("SELECT * FROM test.my_table ORDER BY myint_id ASC FORMAT JSONEachRow SETTINGS output_format_json_named_tuples_as_objects=1;") 66 | .wireTap((allLines: ByteString) => logger.debug("Raw JSON records all-in-one: \n" + allLines.utf8String)) 67 | .via(Framing.delimiter(ByteString.fromString(System.lineSeparator()), 1024)) 68 | .wireTap(eachLine => logger.debug(s"Raw JSON record: ${eachLine.utf8String}")) 69 | .runWith(Sink.seq) 70 | 71 | resultFut.flatMap(each => Future(each.size)) 72 | } 73 | 74 | def countRows(): Int = { 75 | val resFut = client 76 | .query(s"SELECT COUNT(*) FROM test.my_table") 77 | .map(res => Try(res.stripLineEnd.toInt).getOrElse(0)) 78 | Await.result(resFut, 10.seconds) 79 | } 80 | } 81 | 82 | object ClickhouseDB extends App { 83 | 84 | def apply(httpPort: Int) = new ClickhouseDB(httpPort: Int) 85 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/env/KafkaServerEmbedded.scala: -------------------------------------------------------------------------------- 1 | package alpakka.env 2 | 3 | import io.github.embeddedkafka.{EmbeddedKafka, EmbeddedKafkaConfig} 4 | 5 | /** 6 | * In-Memory Kafka broker (no persistence) 7 | * 8 | * Doc: 9 | * https://github.com/embeddedkafka/embedded-kafka 10 | * 11 | * Alternatives: 12 | * - Use the docker Kafka broker from: /docker/docker-compose.yml 13 | * eg by cmd line: docker-compose up -d broker 14 | * - Run [[KafkaServerTestcontainers]] and adjust tmp port in producer/consumer classes 15 | * - Setup Kafka server manually, see: https://kafka.apache.org/quickstart 16 | * - Use Confluent Cloud, see: https://www.confluent.io/confluent-cloud/#view-pricing 17 | */ 18 | object KafkaServerEmbedded extends App { 19 | implicit val config: EmbeddedKafkaConfig = EmbeddedKafkaConfig(kafkaPort = 29092, zooKeeperPort = 2181) 20 | EmbeddedKafka.start() 21 | 22 | sys.addShutdownHook { 23 | println("Got control-c cmd from shell or SIGTERM, about to shutdown...") 24 | EmbeddedKafka.stop() 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/env/KafkaServerTestcontainers.scala: -------------------------------------------------------------------------------- 1 | package alpakka.env 2 | 3 | import org.slf4j.{Logger, LoggerFactory} 4 | import org.testcontainers.containers.KafkaContainer 5 | import org.testcontainers.utility.DockerImageName 6 | 7 | /** 8 | * Uses testcontainers.org to run the 9 | * latest Kafka-Version from Confluent 10 | * See also Kafka broker from: /docker/docker-compose.yml 11 | * 12 | * Alternative: [[KafkaServerEmbedded]] 13 | * 14 | * Doc: 15 | * https://www.testcontainers.org/modules/kafka 16 | * https://doc.akka.io/docs/alpakka-kafka/current/testing-testcontainers.html 17 | * https://doc.akka.io/docs/alpakka-kafka/current/testing.html 18 | */ 19 | class KafkaServerTestcontainers { 20 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 21 | val kafkaVersion = "7.7.0" 22 | val imageName = s"confluentinc/cp-kafka:$kafkaVersion" 23 | val originalPort = 9093 24 | var mappedPort = 1111 25 | val kafkaContainer: KafkaContainer = new KafkaContainer(DockerImageName.parse(imageName)). 26 | withExposedPorts(originalPort) 27 | 28 | def run(): Unit = { 29 | kafkaContainer.start() 30 | mappedPort = kafkaContainer.getMappedPort(originalPort) 31 | logger.info(s"Running Kafka: $imageName on mapped port: $mappedPort") 32 | } 33 | 34 | def stop(): Unit = { 35 | kafkaContainer.stop() 36 | } 37 | } 38 | 39 | object KafkaServerTestcontainers extends App { 40 | val server = new KafkaServerTestcontainers() 41 | server.run() 42 | 43 | sys.ShutdownHookThread{ 44 | println("Got control-c cmd from shell or SIGTERM, about to shutdown...") 45 | server.stop() 46 | } 47 | 48 | Thread.currentThread.join() 49 | 50 | def apply(): KafkaServerTestcontainers = new KafkaServerTestcontainers() 51 | def mappedPort(): Int = server.mappedPort 52 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/env/WebsocketServer.scala: -------------------------------------------------------------------------------- 1 | package alpakka.env 2 | 3 | import org.apache.pekko.actor.{ActorSystem, Terminated} 4 | import org.apache.pekko.http.scaladsl.Http 5 | import org.apache.pekko.http.scaladsl.model.ws.* 6 | import org.apache.pekko.http.scaladsl.server.Directives.* 7 | import org.apache.pekko.http.scaladsl.server.Route 8 | import org.apache.pekko.http.scaladsl.server.directives.WebSocketDirectives 9 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 10 | import org.slf4j.{Logger, LoggerFactory} 11 | 12 | import scala.concurrent.duration.* 13 | import scala.concurrent.{Await, Future} 14 | import scala.language.postfixOps 15 | import scala.util.{Failure, Success} 16 | 17 | /** 18 | * Websocket echo server 19 | * 20 | */ 21 | class WebsocketServer extends WebSocketDirectives { 22 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 23 | implicit val system: ActorSystem = ActorSystem() 24 | 25 | import system.dispatcher 26 | 27 | val (address, port) = ("127.0.0.1", 6002) 28 | var serverBinding: Future[Http.ServerBinding] = _ 29 | 30 | def run(): Unit = { 31 | server(address, port) 32 | } 33 | 34 | def stop(): Future[Terminated] = { 35 | logger.info("About to shutdown...") 36 | val fut = serverBinding.map(serverBinding => serverBinding.terminate(hardDeadline = 3.seconds)) 37 | logger.info("Waiting for connections to terminate...") 38 | val onceAllConnectionsTerminated = Await.result(fut, 10.seconds) 39 | logger.info("Connections terminated") 40 | onceAllConnectionsTerminated.flatMap { _ => system.terminate() 41 | } 42 | } 43 | 44 | private def server(address: String, port: Int): Unit = { 45 | 46 | def echoFlow: Flow[Message, Message, Any] = 47 | Flow[Message].mapConcat { 48 | case tm: TextMessage => 49 | logger.info(s"WebsocketServer received: $tm") 50 | TextMessage(Source.single("ACK: ") ++ tm.textStream) :: Nil 51 | case bm: BinaryMessage => 52 | // ignore binary messages but drain content to avoid the stream being clogged 53 | bm.dataStream.runWith(Sink.ignore) 54 | Nil 55 | } 56 | 57 | val websocketRoute: Route = 58 | path("echo") { 59 | handleWebSocketMessages(echoFlow) 60 | } 61 | 62 | val bindingFuture = Http().newServerAt(address, port).bindFlow(websocketRoute) 63 | bindingFuture.onComplete { 64 | case Success(b) => 65 | logger.info("WebsocketServer started, listening on: " + b.localAddress) 66 | serverBinding = bindingFuture 67 | case Failure(e) => 68 | logger.info(s"Server could not bind to $address:$port. Exception message: ${e.getMessage}") 69 | stop() 70 | } 71 | } 72 | 73 | sys.ShutdownHookThread { 74 | logger.info("Got control-c cmd from shell or SIGTERM, about to shutdown...") 75 | stop() 76 | } 77 | } 78 | 79 | object WebsocketServer extends App { 80 | val server = new WebsocketServer() 81 | server.run() 82 | 83 | def apply() = new WebsocketServer() 84 | 85 | def stop() = server.stop() 86 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/env/jms/AESBrokerPlugin.java: -------------------------------------------------------------------------------- 1 | package alpakka.env.jms; 2 | 3 | import org.apache.activemq.broker.Broker; 4 | import org.apache.activemq.broker.BrokerPluginSupport; 5 | import org.bouncycastle.jce.provider.BouncyCastleProvider; 6 | import org.slf4j.Logger; 7 | import org.slf4j.LoggerFactory; 8 | 9 | import java.security.Security; 10 | 11 | /** 12 | * Inspired by: 13 | * https://github.com/justinreock-roguewave/activemq-aes-plugin 14 | * 15 | * TODO implement hooks for start/stop 16 | * 17 | */ 18 | public class AESBrokerPlugin extends BrokerPluginSupport { 19 | private static final Logger LOGGER = LoggerFactory.getLogger(AESBrokerPlugin.class); 20 | 21 | public Broker installPlugin(Broker broker) { 22 | LOGGER.info("About to install AES payload encryption plugin, using SecurityProvider BC (Bouncy Castle)"); 23 | 24 | Security.addProvider(new BouncyCastleProvider()); 25 | AESBroker aesBroker = null; 26 | try { 27 | aesBroker = new AESBroker(broker); 28 | // Because of race condition: preProcessDispatch is called before AESBroker is initialized 29 | Thread.sleep(1000); 30 | } catch (Exception e) { 31 | LOGGER.error("Exception during installation AES encryption plugin: ", e); 32 | } 33 | 34 | LOGGER.info("Successfully installed AES payload encryption plugin"); 35 | return aesBroker; 36 | } 37 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/env/jms/JMSServerActiveMQ.scala: -------------------------------------------------------------------------------- 1 | package alpakka.env.jms 2 | 3 | import org.apache.activemq.broker.{BrokerPlugin, BrokerService} 4 | import org.slf4j.{Logger, LoggerFactory} 5 | 6 | /** 7 | * Embedded old school ActiveMQ JMSServer to experiment with: 8 | * - KahaDB persistence (in java.io.tmpdir) 9 | * - AES encryption for the payload via AESBrokerPlugin 10 | * 11 | * Alternative: Embedded Artemis JMSServer [[JMSServerArtemis]] 12 | * https://activemq.apache.org/components/artemis/documentation 13 | * Search for: Embedding Apache ActiveMQ Artemis 14 | * 15 | * Issues: 16 | * Messages are processed, but random NPE occur while decrypting messages in AESBroker 17 | * 18 | */ 19 | object JMSServerActiveMQ extends App { 20 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 21 | val broker = new BrokerService() 22 | val host: String = "localhost" 23 | val port = 21616 24 | val serverUrl = s"tcp://$host:$port" 25 | 26 | broker.addConnector(serverUrl) 27 | broker.setBrokerName(host) 28 | 29 | broker.setPersistent(true) 30 | broker.setDataDirectory(System.getProperty("java.io.tmpdir")) 31 | 32 | val aesPlugin = new AESBrokerPlugin() 33 | broker.setPlugins(Array[BrokerPlugin](aesPlugin)) 34 | // For now the secret is passed via JVM system property 35 | System.setProperty("activemq.aeskey", "1234567890123456") 36 | 37 | broker.setAdvisorySupport(false) 38 | broker.setUseJmx(true) 39 | broker.setSchedulerSupport(false) 40 | 41 | broker.setUseShutdownHook(true) 42 | broker.start() 43 | 44 | if (broker.isStarted) { 45 | logger.info(s"JMSServer is started with available processors: ${Runtime.getRuntime.availableProcessors()}") 46 | } 47 | 48 | Thread.currentThread.join() 49 | } 50 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/file/ArchiveHelper.java: -------------------------------------------------------------------------------- 1 | package alpakka.file; 2 | 3 | import org.apache.pekko.util.ByteString; 4 | 5 | import java.io.ByteArrayInputStream; 6 | import java.io.ByteArrayOutputStream; 7 | import java.util.HashMap; 8 | import java.util.Map; 9 | import java.util.zip.ZipEntry; 10 | import java.util.zip.ZipInputStream; 11 | 12 | /** 13 | * Because we don't have support for un-archive in Alpakka files module yet 14 | *

15 | * Inspired by: 16 | * [[docs.javadsl.ArchiveHelper]] 17 | */ 18 | public class ArchiveHelper { 19 | private final static int CHUNK_SIZE = 1024; 20 | 21 | public Map unzip(ByteString zipArchive) throws Exception { 22 | 23 | Map result = new HashMap<>(); 24 | ZipEntry entry; 25 | 26 | try (ZipInputStream zis = new ZipInputStream(new ByteArrayInputStream(zipArchive.toArray()))) { 27 | while ((entry = zis.getNextEntry()) != null) { 28 | int count; 29 | byte[] data = new byte[CHUNK_SIZE]; 30 | 31 | ByteArrayOutputStream dest = new ByteArrayOutputStream(); 32 | while ((count = zis.read(data, 0, CHUNK_SIZE)) != -1) { 33 | dest.write(data, 0, count); 34 | } 35 | dest.flush(); 36 | dest.close(); 37 | zis.closeEntry(); 38 | result.putIfAbsent(entry.getName(), ByteString.fromArray(dest.toByteArray())); 39 | } 40 | } 41 | return result; 42 | } 43 | } 44 | 45 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/file/DuplicateStream.scala: -------------------------------------------------------------------------------- 1 | package alpakka.file 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.{FileIO, Sink, Source, StreamConverters} 5 | import org.apache.pekko.util.ByteString 6 | 7 | import java.io.FileInputStream 8 | import java.nio.file.Paths 9 | import scala.concurrent.Future 10 | import scala.util.{Failure, Success} 11 | 12 | /** 13 | * Process FileInputStream in two steps: 14 | * - Duplicate with alsoTo 15 | * - Do sth else with original stream 16 | * Should also work with ByteArrayInputStream 17 | */ 18 | object DuplicateStream extends App { 19 | implicit val system: ActorSystem = ActorSystem() 20 | 21 | import system.dispatcher 22 | 23 | val sourceFileName = "63MB.pdf" 24 | val sourceFilePath = s"src/main/resources/$sourceFileName" 25 | val fileInputStream = new FileInputStream(sourceFilePath) 26 | val source: Source[ByteString, Any] = 27 | StreamConverters.fromInputStream(() => fileInputStream, chunkSize = 10 * 1024) 28 | 29 | val alsoToSink = FileIO.toPath(Paths.get("outputAlsoTo.pdf")) 30 | val alsoToSink2 = FileIO.toPath(Paths.get("outputAlsoTo2.pdf")) 31 | 32 | val sink = FileIO.toPath(Paths.get("output.pdf")) 33 | 34 | // Step 1 35 | val done: Future[Seq[ByteString]] = source 36 | .alsoTo(alsoToSink) 37 | .alsoTo(alsoToSink2) 38 | .runWith(Sink.seq) 39 | 40 | // Step 2 41 | done.onComplete { 42 | case Success(seq) => 43 | println("Continue processing...") 44 | Source(seq).runWith(sink) 45 | system.terminate() 46 | case Failure(e) => 47 | println(s"Failure: $e") 48 | system.terminate() 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/file/FileEncDec.java: -------------------------------------------------------------------------------- 1 | package alpakka.file; 2 | 3 | import java.io.*; 4 | import java.nio.file.Path; 5 | import java.nio.file.Paths; 6 | import java.util.Base64; 7 | 8 | /** 9 | * Java 8 streaming enc/dec roundtrip "Byte by Byte" 10 | *

11 | * Avoids OutOfMemoryError: 12 | * https://stackoverflow.com/questions/9579874/out-of-memory-when-encoding-file-to-base64 13 | *

14 | * Same with pekko-streams: alpakka.file.FileIOEcho 15 | * 16 | */ 17 | public class FileEncDec { 18 | public static void main(String[] args) { 19 | 20 | String filename = "testfile.jpg"; 21 | Path path = Paths.get("src/main/resources/" + filename); 22 | 23 | try (FileInputStream fis = new FileInputStream(path.toFile())) { 24 | Base64.Encoder enc1 = Base64.getEncoder(); 25 | Base64.Encoder enc2 = Base64.getMimeEncoder(); 26 | Base64.Encoder enc3 = Base64.getUrlEncoder(); 27 | OutputStream os1 = enc1.wrap(new FileOutputStream(filename + "1.enc")); 28 | OutputStream os2 = enc2.wrap(new FileOutputStream(filename + "2.enc")); 29 | OutputStream os3 = enc3.wrap(new FileOutputStream(filename + "3.enc")); 30 | int _byte; 31 | while ((_byte = fis.read()) != -1) { 32 | os1.write(_byte); 33 | os2.write(_byte); 34 | os3.write(_byte); 35 | } 36 | os1.close(); 37 | os2.close(); 38 | os3.close(); 39 | } catch (IOException ioe) { 40 | System.err.printf("I/O error: %s%n", ioe.getMessage()); 41 | } 42 | try (FileOutputStream fos1 = new FileOutputStream("1" + filename); 43 | FileOutputStream fos2 = new FileOutputStream("2" + filename); 44 | FileOutputStream fos3 = new FileOutputStream("3" + filename)) { 45 | Base64.Decoder dec1 = Base64.getDecoder(); 46 | Base64.Decoder dec2 = Base64.getMimeDecoder(); 47 | Base64.Decoder dec3 = Base64.getUrlDecoder(); 48 | InputStream is1 = dec1.wrap(new FileInputStream(filename + "1.enc")); 49 | InputStream is2 = dec2.wrap(new FileInputStream(filename + "2.enc")); 50 | InputStream is3 = dec3.wrap(new FileInputStream(filename + "3.enc")); 51 | int _byte; 52 | while ((_byte = is1.read()) != -1) 53 | fos1.write(_byte); 54 | while ((_byte = is2.read()) != -1) 55 | fos2.write(_byte); 56 | while ((_byte = is3.read()) != -1) 57 | fos3.write(_byte); 58 | is1.close(); 59 | is2.close(); 60 | is3.close(); 61 | } catch (IOException ioe) { 62 | System.err.printf("I/O error: %s%n", ioe.getMessage()); 63 | } 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/file/FileIOEcho.scala: -------------------------------------------------------------------------------- 1 | package alpakka.file 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.IOResult 5 | import org.apache.pekko.stream.scaladsl.FileIO 6 | 7 | import java.nio.file.Paths 8 | import scala.concurrent.Future 9 | import scala.util.{Failure, Success} 10 | 11 | /** FileIO echo flow with chunked base64 encoding/decoding: 12 | * 13 | * testfile.jpg -> base64 encoding -> testfile.enc -> base64 decoding -> testfile_result.jpg 14 | * 15 | * Remark: 16 | * - The chunkSize of the encoding file source MUST be a multiples of 3 byte, eg 3000 17 | * 18 | * @see [[https://stackoverflow.com/questions/7920780/is-it-possible-to-base64-encode-a-file-in-chunks]] 19 | * see also: [[alpakka.file.FileEncDec]] 20 | */ 21 | object FileIOEcho extends App { 22 | implicit val system: ActorSystem = ActorSystem() 23 | 24 | import system.dispatcher 25 | 26 | val sourceFileName = "src/main/resources/testfile.jpg" 27 | val encFileName = "testfile.enc" 28 | val resultFileName = "testfile_result.jpg" 29 | 30 | val sourceOrig = FileIO.fromPath(Paths.get(sourceFileName), chunkSize = 3000) 31 | val sinkEnc = FileIO.toPath(Paths.get(encFileName)) 32 | 33 | val doneEnc = sourceOrig 34 | //.wireTap(each => println(s"Chunk enc: $each")) 35 | .map(each => each.encodeBase64) 36 | .runWith(sinkEnc) 37 | 38 | doneEnc.onComplete { 39 | case Success(_) => 40 | val sourceEnc = FileIO.fromPath(Paths.get(encFileName)) 41 | val sinkDec = FileIO.toPath(Paths.get(resultFileName)) 42 | 43 | val doneDec = sourceEnc 44 | //.wireTap(each => println(s"Chunk dec: $each")) 45 | .map(each => each.decodeBase64) 46 | .runWith(sinkDec) 47 | terminateWhen(doneDec) 48 | case Failure(ex) => println(s"Exception: $ex") 49 | } 50 | 51 | def terminateWhen(done: Future[IOResult]): Unit = { 52 | done.onComplete { 53 | case Success(_) => 54 | println(s"Flow Success. Written file: $resultFileName About to terminate...") 55 | system.terminate() 56 | case Failure(e) => 57 | println(s"Flow Failure: $e. About to terminate...") 58 | system.terminate() 59 | } 60 | } 61 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/file/FileRotator.scala: -------------------------------------------------------------------------------- 1 | package alpakka.file 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.connectors.file.scaladsl.LogRotatorSink 5 | import org.apache.pekko.stream.scaladsl.{FileIO, Flow, Keep, Source} 6 | import org.apache.pekko.util.ByteString 7 | import org.slf4j.{Logger, LoggerFactory} 8 | 9 | import java.io.File 10 | import java.nio.file.Path 11 | import java.nio.file.StandardOpenOption.* 12 | 13 | /** 14 | * Inspired by: 15 | * https://discuss.lightbend.com/t/writing-element-each-to-its-own-file-problem-on-last-element/7696 16 | * 17 | * The issue mentioned is fixed now with: 18 | * https://github.com/akka/alpakka/pull/2559 19 | * 20 | * So all .txt files are written with the correct content 21 | * 22 | */ 23 | object FileRotator extends App { 24 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 25 | implicit val system: ActorSystem = ActorSystem() 26 | 27 | import system.dispatcher 28 | 29 | val logRotatorSink = { 30 | LogRotatorSink.withSinkFactory( 31 | triggerGeneratorCreator = 32 | () => n => Some(new File(s"file${n.decodeString("UTF-8")}.txt").toPath), 33 | sinkFactory = 34 | (path: Path) => 35 | Flow[ByteString].toMat(FileIO.toPath(path, Set(CREATE, WRITE, TRUNCATE_EXISTING, SYNC)))(Keep.right) 36 | ) 37 | } 38 | 39 | val done = 40 | Source(1 to 4) 41 | .map(i => ByteString.fromString(i.toString)) 42 | .runWith(logRotatorSink) 43 | 44 | done.onComplete(_ => system.terminate()) 45 | } 46 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/jms/JMSTextMessageProducerClient.scala: -------------------------------------------------------------------------------- 1 | package alpakka.jms 2 | 3 | import com.typesafe.config.Config 4 | import org.apache.activemq.ActiveMQConnectionFactory 5 | import org.apache.pekko.Done 6 | import org.apache.pekko.actor.ActorSystem 7 | import org.apache.pekko.stream.ThrottleMode 8 | import org.apache.pekko.stream.connectors.jms._ 9 | import org.apache.pekko.stream.connectors.jms.scaladsl.JmsProducer 10 | import org.apache.pekko.stream.scaladsl.{Sink, Source} 11 | import org.slf4j.{Logger, LoggerFactory} 12 | 13 | import javax.jms.ConnectionFactory 14 | import scala.concurrent.Future 15 | import scala.concurrent.duration._ 16 | 17 | /** 18 | * Works together with [[ProcessingApp]] 19 | * Shows how to use ConnectionRetrySettings/SendRetrySettings of the Alpakka JMS connector, 20 | * together with the failover meccano provided by ActiveMQ/Artemis libs 21 | * 22 | */ 23 | object JMSTextMessageProducerClient { 24 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 25 | implicit val system: ActorSystem = ActorSystem() 26 | 27 | val connectionRetrySettings = ConnectionRetrySettings(system) 28 | .withConnectTimeout(10.seconds) 29 | .withInitialRetry(100.millis) 30 | .withBackoffFactor(2.0d) 31 | .withMaxBackoff(1.minute) 32 | .withMaxRetries(10) 33 | 34 | val sendRetrySettings = SendRetrySettings(system) 35 | .withInitialRetry(20.millis) 36 | .withBackoffFactor(1.5d) 37 | .withMaxBackoff(500.millis) 38 | .withMaxRetries(10) 39 | 40 | // The "failover:" part in the brokerURL instructs the ActiveMQ lib to reconnect on network failure 41 | val connectionFactory = new ActiveMQConnectionFactory("artemis", "artemis", "failover:tcp://127.0.0.1:21616") 42 | 43 | def main(args: Array[String]): Unit = { 44 | jmsTextMessageProducerClient(connectionFactory) 45 | } 46 | 47 | private def jmsTextMessageProducerClient(connectionFactory: ConnectionFactory) = { 48 | val producerConfig: Config = system.settings.config.getConfig(JmsProducerSettings.configPath) 49 | val jmsProducerSink: Sink[JmsTextMessage, Future[Done]] = JmsProducer.sink( 50 | JmsProducerSettings(producerConfig, connectionFactory).withQueue("test-queue") 51 | .withConnectionRetrySettings(connectionRetrySettings) 52 | .withSendRetrySettings(sendRetrySettings) 53 | .withSessionCount(1) 54 | ) 55 | 56 | Source(1 to 2000000) 57 | .throttle(10, 1.second, 10, ThrottleMode.shaping) 58 | .wireTap(number => logger.info(s"SEND Msg with TRACE_ID: $number")) 59 | .map { number => 60 | JmsTextMessage(s"Payload: ${number.toString}") 61 | .withProperty("TRACE_ID", number) //custom TRACE_ID 62 | .withHeader(JmsCorrelationId.create(number.toString)) //The JMS way 63 | } 64 | //.wireTap(each => println(each.getHeaders)) 65 | .runWith(jmsProducerSink) 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/kafka/TextMessageGenerator.scala: -------------------------------------------------------------------------------- 1 | package alpakka.kafka 2 | 3 | import java.util.concurrent.ThreadLocalRandom 4 | import scala.collection.mutable.ListBuffer 5 | 6 | /** 7 | * Inspired by: 8 | * https://github.com/oel/akka-streams-text-mining/blob/master/src/main/scala/ngrams/TextMessage.scala 9 | * 10 | * Shorter implementation: 11 | * https://alvinalexander.com/scala/creating-random-strings-in-scala 12 | * 13 | */ 14 | object TextMessageGenerator { 15 | val alphabetSet: Set[Char] = ('a' to 'z').toSet 16 | val alphabets: Seq[Char] = alphabetSet.toList 17 | val vowelSet: Set[Char] = Set('a', 'e', 'i', 'o', 'u') 18 | val vowels: Seq[Char] = vowelSet.toList 19 | val consonantSet: Set[Char] = alphabetSet -- vowelSet 20 | val consonants: Seq[Char] = consonantSet.toList 21 | 22 | // Subset of Punct character class """!"#$%&'()*+,-./:;<=>?@[\]^_`{|}~""" 23 | val puncts: String = """.,;?!""" 24 | 25 | def random: ThreadLocalRandom = ThreadLocalRandom.current 26 | 27 | def randomChar: Char = alphabets(random.nextInt(0, alphabets.length)) 28 | 29 | def mostlyVowelChar: Char = { 30 | // 4/5th chance of vowel 31 | val isVowel: Boolean = if (random.nextInt(0, 5) > 0) true else false 32 | if (isVowel) vowels(random.nextInt(0, vowels.length)) else consonants(random.nextInt(0, consonants.length)) 33 | } 34 | 35 | def maybeUpperChar: Char = { 36 | // 1/5th chance of uppercase 37 | val isUppercase: Boolean = if (random.nextInt(0, 5) == 0) true else false 38 | if (isUppercase) Character.toUpperCase(randomChar) else randomChar 39 | } 40 | 41 | // Generate a word within a range of lengths 42 | def genRandWord(minLen: Int, maxLen: Int): String = { 43 | val word = new ListBuffer[Char]() 44 | 45 | val wordLen: Int = random.nextInt(minLen, maxLen + 1) 46 | 47 | for (i <- 1 to wordLen) { 48 | val char = if (i == 1) maybeUpperChar else if (i % 2 == 0) mostlyVowelChar else randomChar 49 | word += char 50 | } 51 | 52 | word.mkString 53 | } 54 | 55 | def genRandTextWithKeyword(minWordsInText: Int, maxWordsInText: Int, 56 | minWordLen: Int = 2, maxWordLen: Int = 8, 57 | minWordsInClause: Int = 1, maxWordsInClause: Int = 10, keyword: String 58 | ): String = { 59 | 60 | val randomLevel: Double = 0.05 61 | val text = new ListBuffer[String]() 62 | 63 | val numWordsInText: Int = random.nextInt(minWordsInText, maxWordsInText + 1) 64 | 65 | var wordCount: Int = 0 66 | var textLen: Int = 0 67 | 68 | while (wordCount < numWordsInText) { 69 | val numWords = random.nextInt(minWordsInClause, maxWordsInClause + 1) 70 | 71 | val numWordsInClause = if (numWordsInText - wordCount < numWords) numWordsInText - wordCount else 72 | numWords 73 | 74 | var clauseLen: Int = 0 75 | 76 | // Generate a clause 77 | for (i <- 1 to numWordsInClause) { 78 | val word: String = genRandWord(minWordLen, maxWordLen) 79 | text += word 80 | 81 | if (math.random() < randomLevel) text += " " + keyword 82 | 83 | clauseLen += word.length 84 | wordCount += 1 85 | 86 | if (i < numWordsInClause) { 87 | text += " " 88 | clauseLen += 1 89 | } 90 | } 91 | 92 | // Add a punctuation 93 | text += puncts.charAt(random.nextInt(0, puncts.length)).toString 94 | clauseLen += 1 95 | 96 | if (wordCount < numWordsInText) { 97 | text += " " 98 | clauseLen += 1 99 | } 100 | 101 | textLen += clauseLen 102 | } 103 | 104 | // println(s"textLen (in chars): is $textLen") 105 | text.mkString 106 | } 107 | } 108 | 109 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/kafka/TotalFake.scala: -------------------------------------------------------------------------------- 1 | package alpakka.kafka 2 | 3 | import alpakka.kafka.TotalFake.{IncrementMessage, IncrementWord} 4 | import org.apache.pekko.Done 5 | import org.apache.pekko.actor.Actor 6 | import org.apache.pekko.kafka.ConsumerMessage.CommittableMessage 7 | 8 | /** 9 | * Keep the state of: 10 | * - WORD count for keyword "fakeNews" 11 | * - MESSAGE count for messages which contain the keyword "fakeNews" 12 | * 13 | * Note that a message can contain several "fakeNews" keywords 14 | */ 15 | object TotalFake { 16 | case class IncrementWord(msg: CommittableMessage[String, java.lang.Long], id: String) 17 | case class IncrementMessage(msg: CommittableMessage[String, java.lang.Long], id: String) 18 | } 19 | 20 | class TotalFake extends Actor { 21 | var totalWords: Int = 0 22 | var totalNews: Int = 0 23 | 24 | override def receive: Receive = { 25 | case IncrementWord(msg, id) => 26 | val newValue = msg.record.value().toInt 27 | 28 | if (msg.record.key().equalsIgnoreCase("fakeNews")) { 29 | println(s"$id - WORD count fakeNews: $newValue (+ ${newValue - totalWords})") 30 | totalWords = newValue 31 | } 32 | sender() ! Done 33 | 34 | case IncrementMessage(msg, id) => 35 | val newValue = msg.record.value.toInt 36 | 37 | totalNews += newValue 38 | println(s"$id - MESSAGE count: $totalNews (+ $newValue)") 39 | sender() ! Done 40 | } 41 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/kafka/WordCountProducer.scala: -------------------------------------------------------------------------------- 1 | package alpakka.kafka 2 | 3 | import org.apache.kafka.clients.producer.ProducerRecord 4 | import org.apache.kafka.common.errors.{NetworkException, UnknownTopicOrPartitionException} 5 | import org.apache.kafka.common.serialization.StringSerializer 6 | import org.apache.pekko.actor.ActorSystem 7 | import org.apache.pekko.kafka.ProducerMessage.Message 8 | import org.apache.pekko.kafka.ProducerSettings 9 | import org.apache.pekko.kafka.scaladsl.Producer 10 | import org.apache.pekko.stream.ThrottleMode 11 | import org.apache.pekko.stream.scaladsl.{Keep, Sink, Source} 12 | import org.apache.pekko.{Done, NotUsed} 13 | 14 | import scala.concurrent.Future 15 | import scala.concurrent.duration.* 16 | 17 | /** 18 | * Produce unbounded text messages to the topic wordcount-input 19 | * 20 | */ 21 | object WordCountProducer extends App { 22 | implicit val system: ActorSystem = ActorSystem() 23 | 24 | import system.dispatcher 25 | 26 | val bootstrapServers = "localhost:29092" 27 | 28 | val topic = "wordcount-input" 29 | val fakeNewsKeyword = "fakeNews" 30 | 31 | 32 | // initial msg in topic, required to create the topic before any consumer subscribes to it 33 | val InitialMsg = "truth" 34 | 35 | val partition0 = 0 36 | 37 | val producerSettings = ProducerSettings(system, new StringSerializer, new StringSerializer) 38 | .withBootstrapServers(bootstrapServers) 39 | 40 | def initializeTopic(topic: String): Unit = { 41 | val producer = producerSettings.createKafkaProducer() 42 | producer.send(new ProducerRecord(topic, partition0, null: String, InitialMsg)) 43 | } 44 | 45 | def produce(topic: String, messageMap: Map[Int, String], settings: ProducerSettings[String, String] = producerSettings): Future[Done] = { 46 | 47 | val source = Source.fromIterator(() => { 48 | Iterator.continually { 49 | val nextInt = java.util.concurrent.ThreadLocalRandom.current().nextInt(messageMap.size) 50 | val nextString = messageMap.getOrElse(nextInt, "N/A") 51 | println("Next Message: " + nextString) 52 | nextString 53 | } 54 | }) 55 | .map(each => { 56 | val recordWithCurrentTimestamp = new ProducerRecord(topic, null: String, each) 57 | Message(recordWithCurrentTimestamp, NotUsed) 58 | }) 59 | .throttle(100, 100.milli, 10, ThrottleMode.shaping) 60 | .viaMat(Producer.flexiFlow(settings))(Keep.right) 61 | 62 | source.runWith(Sink.ignore) 63 | } 64 | 65 | sys.addShutdownHook { 66 | println("Got control-c cmd from shell, about to shutdown...") 67 | } 68 | 69 | initializeTopic(topic) 70 | val randomMap: Map[Int, String] = TextMessageGenerator.genRandTextWithKeyword(1000, 1000, 3, 5, 5, 10, WordCountProducer.fakeNewsKeyword).split("([!?.])").toList.zipWithIndex.toMap.map(_.swap) 71 | val done = produce(topic, randomMap) 72 | 73 | done.recover { 74 | case e: NetworkException => 75 | println(s"NetworkException $e occurred. Retry...") 76 | produce(topic, randomMap) 77 | case e: UnknownTopicOrPartitionException => 78 | println(s"UnknownTopicOrPartitionException $e occurred. Retry...") 79 | produce(topic, randomMap) 80 | case e: RuntimeException => 81 | println(s"RuntimeException $e occurred. Shutdown...") 82 | system.terminate() 83 | } 84 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/kafka/avro/AvroRecord.scala: -------------------------------------------------------------------------------- 1 | package alpakka.kafka.avro 2 | 3 | case class AvroRecord(str1: String, str2: String, int1: Int) -------------------------------------------------------------------------------- /src/main/scala/alpakka/kafka/avro/SimpleAvroConsumer.scala: -------------------------------------------------------------------------------- 1 | package alpakka.kafka.avro 2 | 3 | import com.sksamuel.avro4s.* 4 | import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} 5 | import org.apache.kafka.common.serialization.StringDeserializer 6 | 7 | import java.time.Duration 8 | import java.util.Properties 9 | import scala.jdk.CollectionConverters.* 10 | 11 | /** 12 | * Not pekko streams related 13 | * 14 | * Prerequisite: 15 | * Run [[alpakka.env.KafkaServerEmbedded]] 16 | * Run [[alpakka.kafka.avro.SimpleAvroProducer]] 17 | */ 18 | object SimpleAvroConsumer extends App { 19 | val props = new Properties() 20 | props.put("bootstrap.servers", "localhost:29092") 21 | props.put("group.id", "mygroup") 22 | props.put("key.deserializer", classOf[StringDeserializer].getName) 23 | props.put("value.deserializer", classOf[AvroDeserializer].getName) 24 | 25 | val consumer = new KafkaConsumer[String, AvroRecord](props) 26 | consumer.subscribe(List("avro-topic").asJava) 27 | 28 | var running = true 29 | while (running) { 30 | val records = consumer.poll(Duration.ofMillis(100)) 31 | for (record: ConsumerRecord[String, AvroRecord] <- records.asScala) { 32 | val avroRecord = record.value() 33 | println(s"Receiving record: str1=${avroRecord.str1}, str2=${avroRecord.str2}, int1=${avroRecord.int1}") 34 | } 35 | } 36 | } 37 | 38 | class AvroDeserializer extends org.apache.kafka.common.serialization.Deserializer[AvroRecord] { 39 | override def deserialize(topic: String, data: Array[Byte]): AvroRecord = { 40 | val avroSchema = AvroSchema[AvroRecord] 41 | val avroInputStream = AvroInputStream.binary[AvroRecord].from(data).build(avroSchema) 42 | val result = avroInputStream.iterator.next() 43 | avroInputStream.close() 44 | result 45 | } 46 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/kafka/avro/SimpleAvroProducer.scala: -------------------------------------------------------------------------------- 1 | package alpakka.kafka.avro 2 | 3 | import com.sksamuel.avro4s.* 4 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} 5 | import org.apache.kafka.common.serialization.StringSerializer 6 | 7 | import java.util.Properties 8 | 9 | /** 10 | * Not pekko streams related 11 | * 12 | * Prerequisite: 13 | * Run [[alpakka.env.KafkaServerEmbedded]] 14 | */ 15 | object SimpleAvroProducer extends App { 16 | val props = new Properties() 17 | props.put("bootstrap.servers", "localhost:29092") 18 | props.put("key.serializer", classOf[StringSerializer].getName) 19 | props.put("value.serializer", classOf[AvroSerializer].getName) 20 | 21 | val producer = new KafkaProducer[String, AvroRecord](props) 22 | 23 | try for (i <- 0 until 100) { 24 | val avroRecord = AvroRecord(s"Str 1-$i", s"Str 2-$i", i) 25 | println(s"Sending record: $avroRecord") 26 | 27 | val record = new ProducerRecord[String, AvroRecord]("avro-topic", avroRecord) 28 | producer.send(record) 29 | 30 | Thread.sleep(100) 31 | } finally producer.close() 32 | } 33 | 34 | class AvroSerializer extends org.apache.kafka.common.serialization.Serializer[AvroRecord] { 35 | override def serialize(topic: String, data: AvroRecord): Array[Byte] = { 36 | val baos = new java.io.ByteArrayOutputStream() 37 | val avroOutputStream = AvroOutputStream.binary[AvroRecord].to(baos).build() 38 | avroOutputStream.write(data) 39 | avroOutputStream.close() 40 | baos.toByteArray 41 | } 42 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/patterns/PassThroughFlowExample.scala: -------------------------------------------------------------------------------- 1 | package alpakka.patterns 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.scaladsl.{Flow, Source, *} 6 | import org.apache.pekko.stream.{FlowShape, Graph, OverflowStrategy} 7 | 8 | import java.time.LocalDateTime 9 | 10 | /** 11 | * Doc: 12 | * https://doc.akka.io/docs/alpakka/current/patterns.html#passthrough 13 | * 14 | * Use [[PassThroughFlow]] to pass original value to subsequent stage 15 | * 16 | * Applied in [[alpakka.kafka.WordCountConsumer]] 17 | * 18 | */ 19 | object PassThroughFlowExample extends App { 20 | implicit val system: ActorSystem = ActorSystem() 21 | 22 | val sourceOfOriginalValues = Source(1 to 100) 23 | .map(origValue => (origValue.toString, LocalDateTime.now())) 24 | 25 | val esotericSlowFlow = Flow[(String, LocalDateTime)] 26 | .buffer(1, OverflowStrategy.dropHead) 27 | .map { s => Thread.sleep(2000); s } 28 | .scan(Map[String, LocalDateTime]())((m, s) => m + (s._1 -> s._2)) 29 | .extrapolate(Iterator.continually(_), Some(Map.empty)) // no backpressure, emit always a element 30 | .buffer(1, OverflowStrategy.dropHead) 31 | 32 | sourceOfOriginalValues.via(PassThroughFlow(esotericSlowFlow)) 33 | .runWith(Sink.foreach(t => println(s"Reached sink: originalValue: ${t._2}, resultMap: ${t._1}"))) 34 | } 35 | 36 | object PassThroughFlow { 37 | def apply[A, T](processingFlow: Flow[A, T, NotUsed]): Graph[FlowShape[A, (T, A)], NotUsed] = 38 | apply[A, T, (T, A)](processingFlow, Keep.both) 39 | 40 | def apply[A, T, O](processingFlow: Flow[A, T, NotUsed], output: (T, A) => O): Graph[FlowShape[A, O], NotUsed] = 41 | Flow.fromGraph(GraphDSL.create() { implicit builder => { 42 | import GraphDSL.Implicits.* 43 | 44 | val broadcast = builder.add(Broadcast[A](2)) 45 | val zip = builder.add(ZipWith[T, A, O]((left, right) => output(left, right))) 46 | 47 | broadcast.out(0) ~> processingFlow ~> zip.in0 48 | broadcast.out(1) ~> zip.in1 49 | 50 | FlowShape(broadcast.in, zip.out) 51 | } 52 | }) 53 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/tcp_to_websockets/hl7mllp/AdtMessageFactory.java: -------------------------------------------------------------------------------- 1 | package alpakka.tcp_to_websockets.hl7mllp; 2 | 3 | import ca.uhn.hl7v2.HL7Exception; 4 | import ca.uhn.hl7v2.model.Message; 5 | 6 | import java.io.IOException; 7 | 8 | public class AdtMessageFactory { 9 | 10 | public static Message createMessage(String messageType) throws HL7Exception, IOException { 11 | 12 | //This patterns enables you to build other message types 13 | if ( messageType.equals("A01") ) 14 | { 15 | return new OurAdtA01MessageBuilder().Build(); 16 | } 17 | 18 | //if other types of ADT messages are needed, then implement your builders here 19 | throw new RuntimeException(String.format("%s message type is not supported yet. Extend this if you need to", messageType)); 20 | 21 | } 22 | } -------------------------------------------------------------------------------- /src/main/scala/alpakka/tcp_to_websockets/hl7mllp/ExceptionHandler.java: -------------------------------------------------------------------------------- 1 | package alpakka.tcp_to_websockets.hl7mllp; 2 | 3 | import ca.uhn.hl7v2.HL7Exception; 4 | import ca.uhn.hl7v2.protocol.ReceivingApplicationExceptionHandler; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | import java.util.Map; 9 | 10 | /** 11 | * Hook for additional error processing depending on exception type or the exception message 12 | * Basic error processing should be done in {@link HL7Exception} 13 | * 14 | */ 15 | public class ExceptionHandler implements ReceivingApplicationExceptionHandler { 16 | private static final Logger LOGGER = LoggerFactory.getLogger(ExceptionHandler.class); 17 | 18 | @Override 19 | /* 20 | * @return theOutgoingNegativeAcknowledgementMessage (possibly enhanced) 21 | */ 22 | public String processException(String theIncomingMessage, Map theIncomingMetadata, String theOutgoingNegativeAcknowledgementMessage, Exception theException) { 23 | 24 | LOGGER.error("The error message was:" + theException.getMessage() + "\n"); 25 | 26 | return theOutgoingNegativeAcknowledgementMessage; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/tcp_to_websockets/hl7mllp/MllpProtocol.scala: -------------------------------------------------------------------------------- 1 | package alpakka.tcp_to_websockets.hl7mllp 2 | 3 | /** 4 | * Doc MLLP: 5 | * http://hl7.ihelse.net/hl7v3/infrastructure/transport/transport_mllp.html 6 | * and [[ca.uhn.hl7v2.llp.MllpConstants]] 7 | */ 8 | trait MllpProtocol { 9 | 10 | //MLLP messages begin after hex "0x0B" and continue until "0x1C|0x0D" 11 | val START_OF_BLOCK = "\u000b" //0x0B 12 | val END_OF_BLOCK = "\u001c" //0x1C 13 | val CARRIAGE_RETURN = "\r" //0x0D 14 | 15 | def encodeMllp(message: String): String = { 16 | START_OF_BLOCK + message + END_OF_BLOCK + CARRIAGE_RETURN 17 | } 18 | 19 | // The HAPI parser needs /r as segment terminator, but this is not printable 20 | def printable(message: String): String = { 21 | message.replace("\r", "\n") 22 | } 23 | 24 | def printableShort(message: String): String = { 25 | printable(message).take(20).concat("...") 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/tcp_to_websockets/websockets/WebsocketClientActor.scala: -------------------------------------------------------------------------------- 1 | package alpakka.tcp_to_websockets.websockets 2 | 3 | import alpakka.tcp_to_websockets.websockets.WebsocketClientActor.* 4 | import org.apache.commons.lang3.exception.ExceptionUtils 5 | import org.apache.pekko.actor.{Actor, ActorLogging, ActorRef, ActorSystem, Props} 6 | import org.apache.pekko.http.scaladsl.model.StatusCode 7 | 8 | import scala.concurrent.ExecutionContextExecutor 9 | import scala.concurrent.duration.* 10 | 11 | 12 | case class ConnectionException(cause: String) extends RuntimeException 13 | 14 | object WebsocketClientActor { 15 | def props(id: String, endpoint: String, websocketConnectionStatusActor: ActorRef): Props = 16 | Props(new WebsocketClientActor(id, endpoint, websocketConnectionStatusActor)) 17 | 18 | final case object Upgraded 19 | final case object Connected 20 | final case object Terminated 21 | final case class ConnectionFailure(ex: Throwable) 22 | final case class FailedUpgrade(statusCode: StatusCode) 23 | final case class SendMessage(msg: String) 24 | 25 | } 26 | 27 | class WebsocketClientActor(id: String, endpoint: String, websocketConnectionStatusActor: ActorRef) 28 | extends Actor with ActorLogging { 29 | implicit private val system: ActorSystem = context.system 30 | implicit private val executionContext: ExecutionContextExecutor = system.dispatcher 31 | 32 | val webSocketClient: WebSocketClient = WebSocketClient(id, endpoint, self) 33 | 34 | override def receive: Receive = startup //initial state 35 | 36 | private def startup: Receive = { 37 | case Upgraded => 38 | log.info(s"Client$id: WebSocket upgraded") 39 | case FailedUpgrade(statusCode) => 40 | log.error(s"Client$id: failed to upgrade WebSocket connection: $statusCode") 41 | websocketConnectionStatusActor ! WebsocketConnectionStatusActor.Terminated 42 | throw ConnectionException(statusCode.toString()) 43 | case ConnectionFailure(ex) => 44 | log.error(s"Client $id: failed to establish WebSocket connection: $ex") 45 | websocketConnectionStatusActor ! WebsocketConnectionStatusActor.Terminated 46 | throw ConnectionException(ExceptionUtils.getRootCause(ex).getMessage) 47 | case Connected => 48 | log.info(s"Client $id: WebSocket connected") 49 | websocketConnectionStatusActor ! WebsocketConnectionStatusActor.Connected 50 | context.become(running) 51 | case SendMessage(msg) => 52 | log.warning(s"In state startup. Can not receive message: $msg. Resend after 2 seconds") 53 | system.scheduler.scheduleOnce(2.seconds, self, SendMessage(msg)) 54 | } 55 | 56 | private def running: Receive = { 57 | case SendMessage(msg) => 58 | log.info(s"About to send message to WebSocket: $msg") 59 | webSocketClient.sendToWebsocket(msg) 60 | case Terminated => 61 | log.error(s"Client $id: WebSocket connection terminated") 62 | websocketConnectionStatusActor ! WebsocketConnectionStatusActor.Terminated 63 | throw ConnectionException(s"Client $id: WebSocket connection terminated") 64 | case ConnectionFailure(ex) => 65 | log.error(s"Client $id: ConnectionFailure occurred: $ex") 66 | websocketConnectionStatusActor ! WebsocketConnectionStatusActor.Terminated 67 | throw ConnectionException(ExceptionUtils.getRootCause(ex).getMessage) 68 | } 69 | 70 | override def postStop(): Unit = { 71 | websocketConnectionStatusActor ! WebsocketConnectionStatusActor.Terminated 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/tcp_to_websockets/websockets/WebsocketConnectionStatusActor.scala: -------------------------------------------------------------------------------- 1 | package alpakka.tcp_to_websockets.websockets 2 | 3 | import alpakka.tcp_to_websockets.websockets.WebsocketConnectionStatusActor.{Connected, ConnectionStatus, Terminated} 4 | import org.apache.pekko.actor.{Actor, ActorLogging, ActorSystem, Props} 5 | 6 | import scala.concurrent.ExecutionContextExecutor 7 | 8 | 9 | object WebsocketConnectionStatusActor { 10 | def props(id: String, endpoint: String): Props = 11 | Props(new WebsocketConnectionStatusActor(id, endpoint)) 12 | 13 | final case object Connected 14 | final case object Terminated 15 | final case object ConnectionStatus 16 | 17 | } 18 | 19 | class WebsocketConnectionStatusActor(id: String, endpoint: String) 20 | extends Actor with ActorLogging { 21 | implicit private val system: ActorSystem = context.system 22 | implicit private val executionContext: ExecutionContextExecutor = system.dispatcher 23 | 24 | var isConnected = false 25 | 26 | override def receive: Receive = { 27 | case Connected => 28 | isConnected = true 29 | log.info(s"Client $id: connected to: $endpoint") 30 | 31 | case Terminated => 32 | isConnected = false 33 | log.info(s"Client $id: terminated from: $endpoint") 34 | 35 | case ConnectionStatus => 36 | sender() ! isConnected 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/scala/alpakka/xml/XmlProcessing.scala: -------------------------------------------------------------------------------- 1 | package alpakka.xml 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.connectors.xml.scaladsl.XmlParsing 5 | import org.apache.pekko.stream.connectors.xml.{EndElement, StartElement, TextEvent} 6 | import org.apache.pekko.stream.scaladsl.{FileIO, Sink, Source} 7 | import org.apache.pekko.util.ByteString 8 | 9 | import java.nio.file.Paths 10 | import java.util.Base64 11 | import scala.concurrent.Future 12 | import scala.util.{Failure, Success} 13 | 14 | /** 15 | * Parse XML file to get a stream of consecutive events of type `ParseEvent`. 16 | * 17 | * As a side effect: 18 | * Detect embedded base64 encoded `application/jpg` files, 19 | * extract and decode them in memory and write to disk 20 | * 21 | */ 22 | 23 | object XmlProcessing extends App { 24 | implicit val system: ActorSystem = ActorSystem() 25 | 26 | import system.dispatcher 27 | 28 | val resultFileName = "extracted_from_xml" 29 | 30 | val done = FileIO.fromPath(Paths.get("src/main/resources/xml_with_base64_embedded.xml")) 31 | .via(XmlParsing.parser) 32 | .statefulMap(() => State(new StringBuilder(), 1, ""))( 33 | (state, nextElem) => { 34 | nextElem match { 35 | case s: StartElement if s.attributes.contains("mediaType") => 36 | state.base64Content.clear() 37 | val mediaType = s.attributes.head._2 38 | val fileEnding = mediaType.split("/").toList.reverse.head 39 | println(s"mediaType: $mediaType / file ending: $fileEnding") 40 | state.fileEnding = fileEnding 41 | (state, Nil) 42 | case s: EndElement if s.localName == "embeddedDoc" => 43 | Source.single(ByteString(state.base64Content.toString)) 44 | .map(each => ByteString(Base64.getMimeDecoder.decode(each.toByteBuffer))) 45 | .runWith(FileIO.toPath(Paths.get(s"${state.counter}-$resultFileName.${state.fileEnding}"))) 46 | state.counter = state.counter + 1 47 | (state, Nil) 48 | case t: TextEvent => 49 | println(s"TextEvent with (chunked) content: ${t.text}") 50 | state.base64Content.append(t.text) 51 | (state, Nil) 52 | case _ => 53 | (state, Nil) 54 | } 55 | }, 56 | // Cleanup function, we return the last state 57 | state => Some(state)) 58 | 59 | .runWith(Sink.ignore) 60 | 61 | terminateWhen(done) 62 | 63 | 64 | def terminateWhen(done: Future[?]): Unit = { 65 | done.onComplete { 66 | case Success(_) => 67 | println("Flow Success. About to terminate...") 68 | system.terminate() 69 | case Failure(e) => 70 | println(s"Flow Failure: $e. About to terminate...") 71 | system.terminate() 72 | } 73 | } 74 | 75 | case class State(base64Content: StringBuilder, var counter: Int, var fileEnding: String) 76 | } 77 | -------------------------------------------------------------------------------- /src/main/scala/interop/ReactiveStreamsInterop.scala: -------------------------------------------------------------------------------- 1 | package interop 2 | 3 | import io.reactivex.rxjava3.core.Flowable 4 | import org.apache.camel.FluentProducerTemplate 5 | import org.apache.camel.component.reactive.streams.api.{CamelReactiveStreams, CamelReactiveStreamsService} 6 | import org.apache.camel.impl.DefaultCamelContext 7 | import org.apache.pekko.actor.ActorSystem 8 | import org.apache.pekko.stream.ThrottleMode 9 | import org.apache.pekko.stream.scaladsl.{Sink, Source} 10 | import org.reactivestreams.Publisher 11 | import org.slf4j.{Logger, LoggerFactory} 12 | import reactor.core.publisher.Flux 13 | 14 | import java.time.Duration 15 | import java.util.concurrent.TimeUnit 16 | import scala.concurrent.Future 17 | import scala.concurrent.duration.DurationInt 18 | 19 | /** 20 | * Show reactive streams interop by using Apache Camel "Reactive Streams" component 21 | * to distribute messages to different consumers: 22 | * - Reactor 23 | * - RxJava 24 | * - pekko-streams 25 | * 26 | * Doc: 27 | * https://doc.akka.io/docs/akka/current/stream/reactive-streams-interop.html 28 | * https://camel.apache.org/components/3.18.x/reactive-streams-component.html 29 | * https://projectreactor.io/docs/core/release/reference/ 30 | * https://github.com/ReactiveX/RxJava 31 | */ 32 | object ReactiveStreamsInterop extends App { 33 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 34 | 35 | implicit val system: ActorSystem = ActorSystem() 36 | 37 | import system.dispatcher 38 | 39 | val camel = new DefaultCamelContext() 40 | val rsCamel: CamelReactiveStreamsService = CamelReactiveStreams.get(camel) 41 | camel.start() 42 | 43 | // Consumer endpoint with Camel 44 | val publisher: Publisher[String] = rsCamel.from("vm:words", classOf[String]) 45 | 46 | // Slow consumer with Reactor 3 47 | Flux.from(publisher) 48 | .delayElements(Duration.ofMillis(2000)) 49 | .map(each => each.toUpperCase()) 50 | .doOnNext(each => logger.info(s"Consumed with Reactor: $each")) 51 | .subscribe() 52 | 53 | // Slow consumer with RxJava 3 54 | Flowable.fromPublisher(publisher) 55 | .delay(2L, TimeUnit.SECONDS) 56 | .map(each => each.toUpperCase()) 57 | .doOnNext(each => logger.info(s"Consumed with RxJava: $each")) 58 | .subscribe() 59 | 60 | // Slow consumer with pekko-streams 61 | Source.fromPublisher(publisher) 62 | .throttle(2, 2.seconds, 2, ThrottleMode.shaping) 63 | .map(each => each.toUpperCase()) 64 | .wireTap(each => logger.info(s"Consumed with pekko-streams: $each")) 65 | .runWith(Sink.ignore) 66 | 67 | // Sender endpoint with Camel 68 | val template: FluentProducerTemplate = camel.createFluentProducerTemplate 69 | 70 | Source(1 to 10) 71 | .throttle(1, 1.seconds, 1, ThrottleMode.shaping) 72 | .mapAsync(1) { i => 73 | template 74 | .withBody(s"Camel$i") 75 | .to("vm:words") 76 | .send 77 | Future(i) 78 | }.runWith(Sink.ignore) 79 | } 80 | -------------------------------------------------------------------------------- /src/main/scala/sample/graphdsl/CompoundFlowFromGraph.scala: -------------------------------------------------------------------------------- 1 | package sample.graphdsl 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.scaladsl.{Broadcast, Flow, GraphDSL, Merge, Sink, Source} 6 | import org.apache.pekko.stream.{FlowShape, UniformFanInShape, UniformFanOutShape} 7 | 8 | /** 9 | * A GraphDSL example, to assemble and start a compound of 10 | * parallel operations (= processorFlows), each executed async. 11 | * 12 | * A similar example with the Flow API operators: [[sample.stream.AsyncExecution]] 13 | * 14 | * Inspired by: 15 | * https://groups.google.com/forum/#!topic/akka-user/Dh8q7TcP2SI 16 | * 17 | */ 18 | object CompoundFlowFromGraph extends App { 19 | implicit val system: ActorSystem = ActorSystem() 20 | 21 | import system.dispatcher 22 | 23 | val processorFlow1: Flow[Int, Int, NotUsed] = Flow[Int].map(_ * 2).wireTap(each => println(s"Processed by Flow1: $each")) 24 | val processorFlow2: Flow[Int, Int, NotUsed] = Flow[Int].map(_ * 3).wireTap(each => println(s"Processed by Flow2: $each")) 25 | val processorFlows: Seq[Flow[Int, Int, NotUsed]] = List(processorFlow1, processorFlow2) 26 | 27 | def compoundFlowFrom[T](indexFlows: Seq[Flow[T, T, NotUsed]]): Flow[T, T, NotUsed] = { 28 | require(indexFlows.nonEmpty, "Cannot create compound flow without any flows to combine") 29 | 30 | Flow.fromGraph(GraphDSL.create() { implicit b => 31 | import org.apache.pekko.stream.scaladsl.GraphDSL.Implicits.* 32 | 33 | val broadcast: UniformFanOutShape[T, T] = b.add(Broadcast(indexFlows.size)) 34 | val merge: UniformFanInShape[T, T] = b.add(Merge(indexFlows.size)) 35 | 36 | indexFlows.foreach(each => broadcast ~> each.async ~> merge) 37 | 38 | FlowShape(broadcast.in, merge.out) 39 | }) 40 | } 41 | 42 | val compoundFlow: Flow[Int, Int, NotUsed] = compoundFlowFrom(processorFlows) 43 | 44 | Source(1 to 100) 45 | .via(compoundFlow) 46 | .runWith(Sink.foreach(each => println(s"Reached sink: $each"))) 47 | .onComplete(_ => system.terminate()) 48 | } -------------------------------------------------------------------------------- /src/main/scala/sample/graphdsl/DistributeAndMerge.scala: -------------------------------------------------------------------------------- 1 | package sample.graphdsl 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.* 6 | import org.apache.pekko.stream.scaladsl.* 7 | 8 | import scala.concurrent.Future 9 | import scala.util.hashing.MurmurHash3 10 | 11 | 12 | /** 13 | * Inspired by, stolen from: 14 | * https://gist.github.com/calvinlfer/cc4ea90328834a95a89ce99aeb998a63 15 | * 16 | * Concepts: 17 | * - Flow that distributes messages (according to a hashing function) across sub-flows 18 | * - The idea is to have ordered processing per sub-flow but parallel processing across sub-flows 19 | * 20 | * Similar examples: 21 | * - https://blog.colinbreck.com/partitioning-akka-streams-to-maximize-throughput 22 | * - https://doc.akka.io/docs/akka/current/stream/stream-cookbook.html#balancing-jobs-to-a-fixed-pool-of-workers 23 | */ 24 | object DistributeAndMerge extends App { 25 | implicit val system: ActorSystem = ActorSystem() 26 | 27 | import system.dispatcher 28 | 29 | private def sampleAsyncCall(x: Int): Future[Int] = Future { 30 | Thread.sleep((x * 100L) % 10) 31 | println(s"Async call for value: $x processed by: ${Thread.currentThread().getName}") 32 | x 33 | } 34 | 35 | // @formatter:off 36 | /** 37 | * Example based on numBuckets = 3 38 | * --- bucket 1 flow --- ~mapAsync(parallelism)~ --- 39 | * |------------------| / \|---------------| 40 | * Open inlet[A] --- | Partition Fan Out| --- bucket 2 flow --- ~mapAsync(parallelism)~ -----| Merge Fan In | --- Open outlet[B] 41 | * |------------------| \ /|---------------| 42 | * --- bucket 3 flow --- ~mapAsync(parallelism)~ --- 43 | * 44 | * @param numBuckets the number of sub-flows to create 45 | * @param parallelism the mapAsync (ordered) parallelism per sub-flow 46 | * @param hash the hashing function used to decide 47 | * @param fn the mapping function to be used for mapAsync 48 | * @tparam A is the input stream of elements of type A 49 | * @tparam B is the output streams of elements of type B 50 | * @return a Flow of elements from type A to type B 51 | */ 52 | // @formatter:on 53 | private def hashingDistribution[A, B](numBuckets: Int, 54 | parallelism: Int, 55 | hash: A => Int, 56 | fn: A => Future[B]): Flow[A, B, NotUsed] = { 57 | Flow.fromGraph(GraphDSL.create() { implicit builder => 58 | import GraphDSL.Implicits.* 59 | val numPorts = numBuckets 60 | val partitioner = 61 | builder.add(Partition[A](outputPorts = numPorts, partitioner = a => math.abs(hash(a)) % numPorts)) 62 | val merger = builder.add(Merge[B](inputPorts = numPorts, eagerComplete = false)) 63 | 64 | Range(0, numPorts).foreach { eachPort => 65 | partitioner.out(eachPort) ~> Flow[A].mapAsync(parallelism)(fn) ~> merger.in(eachPort) 66 | } 67 | 68 | FlowShape(partitioner.in, merger.out) 69 | }) 70 | } 71 | 72 | Source(1 to 10) 73 | .via( 74 | hashingDistribution[Int, Int]( 75 | numBuckets = 3, 76 | parallelism = 2, 77 | hash = element => MurmurHash3.stringHash(element.toString), //Hashing function: String => Int 78 | fn = sampleAsyncCall 79 | ) 80 | ) 81 | .runWith(Sink.foreach(each => println(s"Reached sink: $each"))) 82 | .onComplete(_ => system.terminate()) 83 | } 84 | -------------------------------------------------------------------------------- /src/main/scala/sample/graphdsl/WritePrimes.scala: -------------------------------------------------------------------------------- 1 | package sample.graphdsl 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.* 6 | import org.apache.pekko.stream.scaladsl.* 7 | import org.apache.pekko.util.ByteString 8 | 9 | import java.nio.file.Paths 10 | import java.util.concurrent.ThreadLocalRandom 11 | import scala.concurrent.ExecutionContextExecutor 12 | import scala.concurrent.duration.* 13 | import scala.util.{Failure, Success} 14 | 15 | /** 16 | * Construct a graph with the GraphDSL 17 | * https://doc.akka.io/docs/akka/current/stream/stream-graphs.html?language=scala#constructing-graphs 18 | * 19 | */ 20 | object WritePrimes extends App { 21 | implicit val system: ActorSystem = ActorSystem() 22 | implicit val ec: ExecutionContextExecutor = system.dispatcher 23 | 24 | val maxRandomNumberSize = 100 25 | val primeSource: Source[Int, NotUsed] = 26 | Source.fromIterator(() => Iterator.continually(ThreadLocalRandom.current().nextInt(maxRandomNumberSize))) 27 | .take(100) 28 | .filter(rnd => isPrime(rnd)) 29 | // neighbor +2 is also prime? 30 | .filter(prime => isPrime(prime + 2)) 31 | 32 | val fileSink = FileIO.toPath(Paths.get("target/primes.txt")) 33 | val slowSink = Flow[Int] 34 | .throttle(1, 1.seconds, 1, ThrottleMode.shaping) 35 | .map(i => ByteString(i.toString + "\n")) 36 | .toMat(fileSink)((_, bytesWritten) => bytesWritten) 37 | val consoleSink = Sink.foreach[Int](each => println(s"Reached console sink: $each")) 38 | 39 | // Additional processing flow, to show the nature of the composition 40 | val sharedDoubler = Flow[Int].map(_ * 2) 41 | 42 | // partition primes to both sinks using graph DSL 43 | // Alternatives: 44 | // partition: 45 | // https://doc.akka.io/docs/akka/current/stream/operators/Partition.html 46 | // alsoTo: 47 | // https://doc.akka.io/docs/akka/current/stream/stream-flows-and-basics.html 48 | val graph = GraphDSL.createGraph(slowSink, consoleSink)((_, _)) { implicit builder => 49 | (slow, console) => 50 | import GraphDSL.Implicits.* 51 | val broadcastSplitter = builder.add(Broadcast[Int](2)) // the splitter - like a Unix tee 52 | primeSource ~> broadcastSplitter ~> sharedDoubler ~> slow // connect source to splitter, other side to slow sink (via sharedDoubler) 53 | broadcastSplitter ~> sharedDoubler ~> console // connect other side of splitter to console sink (via sharedDoubler) 54 | ClosedShape 55 | } 56 | val materialized = RunnableGraph.fromGraph(graph).run() 57 | 58 | materialized._2.onComplete { 59 | case Success(_) => 60 | // Grace time to allow writing the last entry to fileSink 61 | Thread.sleep(500) 62 | system.terminate() 63 | case Failure(e) => 64 | println(s"Failure: ${e.getMessage}") 65 | system.terminate() 66 | } 67 | 68 | def isPrime(n: Int): Boolean = { 69 | if (n <= 1) false 70 | else if (n == 2) true 71 | else !(2 until n).exists(x => n % x == 0) 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /src/main/scala/sample/graphstage/Pulse.scala: -------------------------------------------------------------------------------- 1 | package sample.graphstage 2 | 3 | import org.apache.pekko.stream.stage.* 4 | import org.apache.pekko.stream.{Attributes, FlowShape, Inlet, Outlet} 5 | 6 | import scala.concurrent.duration.{DurationInt, FiniteDuration} 7 | 8 | final class Pulse[T](interval: FiniteDuration, initiallyOpen: Boolean = false) 9 | extends GraphStage[FlowShape[T, T]] { 10 | 11 | val in = Inlet[T]("Pulse.in") 12 | val out = Outlet[T]("Pulse.out") 13 | val shape = FlowShape(in, out) 14 | 15 | def createLogic(inheritedAttributes: Attributes): GraphStageLogic = 16 | new TimerGraphStageLogic(shape) with InHandler with OutHandler { 17 | 18 | setHandlers(in, out, this) 19 | 20 | override def preStart(): Unit = if (!initiallyOpen) startPulsing() 21 | override def onPush(): Unit = if (isAvailable(out)) push(out, grab(in)) 22 | override def onPull(): Unit = if (!pulsing) { 23 | pull(in) 24 | startPulsing() 25 | } 26 | 27 | override protected def onTimer(timerKey: Any): Unit = { 28 | if (isAvailable(out) && !isClosed(in) && !hasBeenPulled(in)) pull(in) 29 | } 30 | 31 | private def startPulsing(): Unit = { 32 | pulsing = true 33 | scheduleWithFixedDelay("PulseTimer", 100.millis, interval) 34 | } 35 | private var pulsing = false 36 | } 37 | 38 | override def toString = "Pulse" 39 | } -------------------------------------------------------------------------------- /src/main/scala/sample/graphstage/StreamEventInspector.scala: -------------------------------------------------------------------------------- 1 | package sample.graphstage 2 | 3 | import org.apache.pekko.stream.stage.{GraphStage, GraphStageLogic, InHandler, OutHandler} 4 | import org.apache.pekko.stream.{Attributes, FlowShape, Inlet, Outlet} 5 | import org.slf4j.{Logger, LoggerFactory} 6 | 7 | /** 8 | * Example of an atomic operator created with GraphStage, which may be used to wire tap a stream. 9 | * Inspired by: https://gist.github.com/hochgi/cc354f9b80ca427a4f4d7313c78e4350 10 | * 11 | * From the doc: 12 | * https://doc.akka.io/docs/akka/current/stream/stream-customize.html#custom-processing-with-graphstage 13 | * 14 | * A GraphStage can be used to create arbitrary atomic operators with any number of input or output ports. 15 | * GraphStages are atomic and allow state to be maintained inside it in a safe way. 16 | * GraphStage is a counterpart of the GraphDSL.create() method which creates new stream processing operators by composing others. 17 | * 18 | * See also: 19 | * https://doc.akka.io/docs/akka/current/stream/operators/Source-or-Flow/monitor.html 20 | * 21 | * Hooks: 22 | * @param onUpstreamFinishInspection 23 | * @param onUpstreamFailureInspection 24 | * @param onDownstreamFinishInspection 25 | * @param onPushInspection 26 | * @param onPullInspection 27 | * @tparam Elem 28 | */ 29 | class StreamEventInspector[Elem](onUpstreamFinishInspection: () => Unit = () => {}, 30 | onUpstreamFailureInspection: Throwable => Unit = _ => {}, 31 | onDownstreamFinishInspection: Throwable => Unit = _ => {}, 32 | onPushInspection: Elem => Unit = (_: Elem) => {}, 33 | onPullInspection: () => Unit = () => {} 34 | ) extends GraphStage[FlowShape[Elem, Elem]] { 35 | 36 | private val in = Inlet[Elem]("StreamEventInspector.in") 37 | private val out = Outlet[Elem]("StreamEventInspector.out") 38 | 39 | override val shape: FlowShape[Elem, Elem] = FlowShape(in, out) 40 | 41 | override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) { 42 | 43 | setHandler( 44 | in, 45 | new InHandler { 46 | override def onPush(): Unit = { 47 | val elem = grab(in) 48 | onPushInspection(elem) 49 | push(out, elem) 50 | } 51 | 52 | override def onUpstreamFailure(ex: Throwable): Unit = { 53 | onUpstreamFailureInspection(ex) 54 | super.onUpstreamFailure(ex) 55 | } 56 | 57 | override def onUpstreamFinish(): Unit = { 58 | onUpstreamFinishInspection() 59 | super.onUpstreamFinish() 60 | } 61 | } 62 | ) 63 | 64 | setHandler( 65 | out, 66 | new OutHandler { 67 | override def onPull(): Unit = { 68 | onPullInspection() 69 | pull(in) 70 | } 71 | 72 | override def onDownstreamFinish(cause: Throwable): Unit = { 73 | onDownstreamFinishInspection(cause) 74 | super.onDownstreamFinish(cause: Throwable) 75 | } 76 | } 77 | ) 78 | } 79 | } 80 | 81 | object StreamEventInspector { 82 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 83 | 84 | def apply[T](context: String, printElem: T => String): StreamEventInspector[T] = { 85 | val ctx = "[" + context + "] " 86 | new StreamEventInspector[T]( 87 | () => logger.info(ctx + "upstream completed"), 88 | ex => logger.error(ctx + "upstream failure", ex), 89 | ex => logger.error(ctx + "downstream completed", ex), 90 | el => logger.info(ctx + printElem(el)), 91 | () => logger.info(ctx + "downstream pulled") 92 | ) 93 | } 94 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream/AsyncExecution.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.Done 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.ActorAttributes 6 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 7 | import org.slf4j.{Logger, LoggerFactory} 8 | 9 | import scala.concurrent.Future 10 | import scala.util.{Failure, Success} 11 | 12 | /** 13 | * Each stream element is processed by each flow stage A/B/C (in parallel) 14 | * Show the effects of using a custom dispatcher on stage B to guard (potentially) blocking behaviour 15 | * 16 | * Inspired by: 17 | * http://akka.io/blog/2016/07/06/threading-and-concurrency-in-akka-streams-explained 18 | * 19 | * See also [[sample.stream.WaitForFlowsToComplete]] 20 | * 21 | */ 22 | object AsyncExecution extends App { 23 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 24 | implicit val system: ActorSystem = ActorSystem() 25 | 26 | import system.dispatcher 27 | 28 | def stage(name: String) = 29 | Flow[Int] 30 | .wireTap(index => logger.info(s"Stage $name processing element $index by ${Thread.currentThread().getName}")) 31 | 32 | def stageBlocking(name: String) = 33 | Flow[Int] 34 | .wireTap(index => logger.info(s"Stage $name processing element $index by ${Thread.currentThread().getName}")) 35 | .wireTap(_ => Thread.sleep(5000)) 36 | .withAttributes(ActorAttributes.dispatcher("custom-dispatcher-for-blocking")) 37 | 38 | def sinkBlocking: Sink[Int, Future[Done]] = 39 | Sink.foreach { (index: Int) => 40 | Thread.sleep(2000) 41 | logger.info(s"Slow sink processing element $index by ${Thread.currentThread().getName}") 42 | } 43 | //Adding a custom dispatcher creates an async boundary 44 | //see discussion in: https://discuss.lightbend.com/t/how-can-i-make-sure-that-fileio-frompath-is-picking-up-my-dispatcher/6528/4 45 | .withAttributes(ActorAttributes.dispatcher("custom-dispatcher-for-blocking")) 46 | 47 | 48 | val done = Source(1 to 10) 49 | .via(stage("A")).async 50 | //When activated instead of alsoTo(sinkBlocking): elements for stage C are held up by stage B 51 | //.via(stageBlocking("B")).async 52 | .alsoTo(sinkBlocking).async 53 | .via(stage("C")).async 54 | .runWith(Sink.ignore) 55 | 56 | //With alsoTo(sinkBlocking) the stages A and C signal "done" too early and thus would terminate the whole stream 57 | //The reason for this is the custom dispatcher in sinkBlocking 58 | //terminateWhen(done) 59 | 60 | def terminateWhen(done: Future[?]): Unit = { 61 | done.onComplete { 62 | case Success(_) => 63 | println("Flow Success. About to terminate...") 64 | system.terminate() 65 | case Failure(e) => 66 | println(s"Flow Failure: $e. About to terminate...") 67 | system.terminate() 68 | } 69 | } 70 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream/AvoidDeadlockByUsingSeparateCustomDispatchers.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.ClosedShape 5 | import org.apache.pekko.stream.scaladsl.* 6 | import org.apache.pekko.stream.scaladsl.GraphDSL.Implicits.* 7 | import org.apache.pekko.util.ByteString 8 | 9 | import java.io.{BufferedInputStream, BufferedOutputStream} 10 | import java.nio.file.Paths 11 | import scala.concurrent.duration.Duration 12 | import scala.concurrent.{Await, Future} 13 | 14 | /** 15 | * Reproducer and implemented solution of this discussion: 16 | * https://stackoverflow.com/questions/71264640/akka-stream-hangs-when-starting-more-than-15-external-processes-using-processbui 17 | * 18 | * See also other examples with custom-dispatchers 19 | */ 20 | object AvoidDeadlockByUsingSeparateCustomDispatchers extends App { 21 | implicit val system: ActorSystem = ActorSystem() 22 | 23 | import system.dispatcher 24 | 25 | // This may be bigger than the pool size of the custom-dispatchers 26 | val PROCESSES_COUNT = 50 27 | 28 | println(s"Running with $PROCESSES_COUNT processes...") 29 | 30 | def executeCmdOnStream(cmd: String): Flow[ByteString, ByteString, ?] = { 31 | val convertProcess = new ProcessBuilder(cmd).start 32 | val pipeIn = new BufferedOutputStream(convertProcess.getOutputStream) 33 | val pipeOut = new BufferedInputStream(convertProcess.getInputStream) 34 | Flow 35 | .fromSinkAndSource( 36 | // The important thing is to use two *different* (custom) dispatchers to avoid the dead lock 37 | // The chosen ones do the job, but may not be optimal 38 | StreamConverters.fromOutputStream(() => pipeIn).async("custom-dispatcher-fork-join"), 39 | StreamConverters.fromInputStream(() => pipeOut).async("custom-dispatcher-for-blocking")) 40 | } 41 | 42 | val source = Source(1 to 100) 43 | .map(element => { 44 | println(s"--emit: $element") 45 | ByteString(element) 46 | }) 47 | 48 | val sinksList = (1 to PROCESSES_COUNT).map(i => { 49 | Flow[ByteString] 50 | .via(executeCmdOnStream("cat")) 51 | .toMat(FileIO.toPath(Paths.get(s"process-$i.txt")))(Keep.right) 52 | }) 53 | 54 | val graph = GraphDSL.create(sinksList) { implicit builder => 55 | sinks => 56 | 57 | val broadcast = builder.add(Broadcast[ByteString](sinks.size)) 58 | source ~> broadcast.in 59 | for (i <- broadcast.outlets.indices) { 60 | broadcast.out(i) ~> sinks(i) 61 | } 62 | ClosedShape 63 | } 64 | 65 | Await.result(Future.sequence(RunnableGraph.fromGraph(graph).run()), Duration.Inf) 66 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream/BasicTransformation.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 5 | 6 | /** 7 | * Inspired by: 8 | * https://stackoverflow.com/questions/35120082/how-to-get-started-with-akka-streams 9 | * 10 | */ 11 | object BasicTransformation { 12 | implicit val system: ActorSystem = ActorSystem() 13 | import system.dispatcher 14 | 15 | def main(args: Array[String]): Unit = { 16 | val text = 17 | """|Lorem Ipsum is simply dummy text of the printing and typesetting industry. 18 | |Lorem Ipsum has been the industry's standard dummy text ever since the 1500s, 19 | |when an unknown printer took a galley of type and scrambled it to make a type 20 | |specimen book.""".stripMargin 21 | 22 | val source = Source.fromIterator(() => text.split("\\s").iterator) 23 | val sink = Sink.foreach[String](println) 24 | val flow = Flow[String].map(x => x.toUpperCase) 25 | val result = source.via(flow).runWith(sink) 26 | result.onComplete(_ => system.terminate()) 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream/CalculateMedian.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.ThrottleMode 5 | import org.apache.pekko.stream.scaladsl.Source 6 | 7 | import java.util.concurrent.ThreadLocalRandom 8 | import scala.annotation.tailrec 9 | import scala.concurrent.ExecutionContextExecutor 10 | import scala.concurrent.duration.* 11 | import scala.language.postfixOps 12 | 13 | /** 14 | * Inspired by: 15 | * https://stackoverflow.com/questions/4662292/scala-median-implementation 16 | * 17 | * To calculate the "all time median of medians grouped by 5" we would need to store the values (eg in an actor) 18 | * 19 | */ 20 | 21 | object CalculateMedian extends App { 22 | implicit val system: ActorSystem = ActorSystem() 23 | implicit val ec: ExecutionContextExecutor = system.dispatcher 24 | 25 | val maxRandomNumber = 100 26 | val source = Source.fromIterator(() => Iterator.continually(ThreadLocalRandom.current().nextDouble(maxRandomNumber))) 27 | 28 | source 29 | .throttle(1, 10.millis, 1, ThrottleMode.shaping) 30 | .groupedWithin(100, 1.second) 31 | //.map{each => println(each); each} 32 | .map(each => medianOfMedians(each.toArray)) 33 | .runForeach(result => println(s"Median of Median (grouped by 5) over the last 100 elements: $result")) 34 | .onComplete(_ => system.terminate()) 35 | 36 | 37 | @tailrec def findKMedian(arr: Array[Double], k: Int)(implicit choosePivot: Array[Double] => Double): Double = { 38 | val a = choosePivot(arr) 39 | val (s, b) = arr.partition(a >) 40 | if (s.length == k) a 41 | // The following test is used to avoid infinite repetition 42 | else if (s.isEmpty) { 43 | val (s, b) = arr.partition(a ==) 44 | if (s.length > k) a 45 | else findKMedian(b, k - s.length) 46 | } else if (s.length < k) findKMedian(b, k - s.length) 47 | else findKMedian(s, k) 48 | } 49 | 50 | def medianUpTo5(five: Array[Double]): Double = { 51 | def order2(a: Array[Double], i: Int, j: Int): Unit = { 52 | if (a(i) > a(j)) { 53 | val t = a(i) 54 | a(i) = a(j) 55 | a(j) = t 56 | } 57 | } 58 | 59 | def pairs(a: Array[Double], i: Int, j: Int, k: Int, l: Int) = { 60 | if (a(i) < a(k)) { 61 | order2(a, j, k) 62 | a(j) 63 | } 64 | else { 65 | order2(a, i, l) 66 | a(i) 67 | } 68 | } 69 | 70 | if (five.length < 2) { 71 | return five(0) 72 | } 73 | order2(five, 0, 1) 74 | if (five.length < 4) return if (five.length == 2 || five(2) < five(0)) five(0) 75 | else if (five(2) > five(1)) five(1) 76 | else five(2) 77 | order2(five, 2, 3) 78 | if (five.length < 5) pairs(five, 0, 1, 2, 3) 79 | else if (five(0) < five(2)) { 80 | order2(five, 1, 4) 81 | pairs(five, 1, 4, 2, 3) 82 | } 83 | else { 84 | order2(five, 3, 4) 85 | pairs(five, 0, 1, 3, 4) 86 | } 87 | } 88 | 89 | @tailrec 90 | def medianOfMedians(arr: Array[Double]): Double = { 91 | val medians = arr.grouped(5).map(medianUpTo5).toArray 92 | if (medians.length <= 5) medianUpTo5(medians) 93 | else medianOfMedians(medians) 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream/MergeHubWithDynamicSources.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.DelayOverflowStrategy 6 | import org.apache.pekko.stream.scaladsl.{Flow, MergeHub, RunnableGraph, Sink, Source} 7 | 8 | import scala.collection.parallel.CollectionConverters.* 9 | import scala.concurrent.duration.* 10 | 11 | /** 12 | * Inspired by: 13 | * http://doc.akka.io/docs/akka/current/scala/stream/stream-dynamic.html#dynamic-fan-in-and-fan-out-with-mergehub-broadcasthub-and-partitionhub 14 | * 15 | * Similar example: [[PublishToSourceQueueFromMultipleThreads]] 16 | * 17 | */ 18 | object MergeHubWithDynamicSources extends App { 19 | implicit val system: ActorSystem = ActorSystem() 20 | 21 | val slowSinkConsumer: Sink[Seq[String], NotUsed] = 22 | Flow[Seq[String]] 23 | .delay(1.seconds, DelayOverflowStrategy.backpressure) 24 | .to(Sink.foreach(e => println(s"Reached Sink: $e"))) 25 | 26 | // Attach a MergeHub Source to the consumer. This will materialize to a corresponding Sink 27 | val runnableGraph: RunnableGraph[Sink[String, NotUsed]] = 28 | MergeHub.source[String](perProducerBufferSize = 16) 29 | .groupedWithin(10, 2.seconds) 30 | .to(slowSinkConsumer) 31 | 32 | // By running/materializing the graph we get back a Sink, and hence are able to feed elements into it 33 | // This Sink can be materialized any number of times, and every element that enters the Sink reaches the slowSinkConsumer 34 | val toConsumer: Sink[String, NotUsed] = runnableGraph.run() 35 | 36 | def fastDynamicSource(sourceId: Int, toConsumer: Sink[String, NotUsed]) = { 37 | Source(1 to 10) 38 | .map { each => println(s"Produced: $sourceId.$each"); s"$sourceId.$each" } 39 | .runWith(toConsumer) 40 | } 41 | 42 | // Add dynamic producer sources 43 | // If the consumer cannot keep up, then ALL of the producers are backpressured 44 | (1 to 10).par.foreach(each => fastDynamicSource(each, toConsumer)) 45 | } 46 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream/NumberGuessingGame.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.* 5 | import org.apache.pekko.{Done, NotUsed} 6 | 7 | import scala.concurrent.{ExecutionContext, Future} 8 | import scala.io.StdIn 9 | import scala.util.Random 10 | 11 | /** 12 | * Generated with: https://claude.ai/chat 13 | * Prompt: Generate a game with pekko streams 14 | * 15 | * This is the first attempt 1:1 copy/paste 16 | * A decent explanation was delivered as well 17 | * Pretty cool 18 | */ 19 | object NumberGuessingGame extends App { 20 | implicit val system: ActorSystem = ActorSystem("NumberGuessingGame") 21 | implicit val ec: ExecutionContext = system.dispatcher 22 | 23 | val targetNumber = Random.nextInt(100) + 1 // Random number between 1 and 100 24 | var attempts = 0 25 | 26 | def gameLogic(guess: Int): String = { 27 | attempts += 1 28 | if (guess < targetNumber) { 29 | "Too low! Try again." 30 | } else if (guess > targetNumber) { 31 | "Too high! Try again." 32 | } else { 33 | s"Congratulations! You guessed the number $targetNumber in $attempts attempts." 34 | } 35 | } 36 | 37 | val guessSource: Source[Int, NotUsed] = Source.unfold(()) { _ => 38 | print("Enter your guess (1-100), or 'q' to quit: ") 39 | val input = StdIn.readLine() 40 | if (input.toLowerCase == "q") { 41 | None 42 | } else { 43 | Some((), input.toInt) 44 | } 45 | } 46 | 47 | val gameFlow: Flow[Int, String, NotUsed] = Flow[Int].map(gameLogic) 48 | 49 | val printSink: Sink[String, Future[Done]] = Sink.foreach[String](println) 50 | 51 | val gameGraph: RunnableGraph[Future[Done]] = guessSource 52 | .via(gameFlow) 53 | .takeWhile(result => !result.startsWith("Congratulations"), inclusive = true) 54 | .toMat(printSink)(Keep.right) 55 | 56 | val gameFuture: Future[Done] = gameGraph.run() 57 | 58 | gameFuture.onComplete { _ => 59 | println("Game over!") 60 | system.terminate() 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream/OneBillionRowChallenge.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.scaladsl.* 6 | import org.apache.pekko.util.ByteString 7 | 8 | import java.nio.file.Paths 9 | import scala.util.{Failure, Success} 10 | 11 | /** 12 | * Inspired by the Java One Billion Row Challenge 13 | * https://github.com/gunnarmorling/1brc 14 | * https://github.com/gunnarmorling/1brc/discussions/categories/show-and-tell 15 | * 16 | * A humble 1st implementation 17 | * Focus is on readability/maintainability 18 | * Lacks parallelization in file reading 19 | * Runtime is around 8 minutes on i7-11850H 20 | * 21 | * Similar to: [[FlightDelayStreaming]] 22 | */ 23 | object OneBillionRowChallenge extends App { 24 | implicit val system: ActorSystem = ActorSystem() 25 | 26 | import system.dispatcher 27 | 28 | // Wire generated 1 Billion records resource file 29 | val sourceOfRows = FileIO.fromPath(Paths.get("measurements_subset_10000.txt"), chunkSize = 100 * 1024) 30 | .via(Framing.delimiter(ByteString(System.lineSeparator), maximumFrameLength = 256, allowTruncation = true) 31 | .map(_.utf8String)) 32 | 33 | def stringArrayToMeasurement(cols: Array[String]) = Measurement(cols(0), cols(1).toFloat) 34 | 35 | val csvToMeasurement: Flow[String, Measurement, NotUsed] = Flow[String] 36 | .map(_.split(";")) 37 | .map(stringArrayToMeasurement) 38 | 39 | val aggregate: Flow[Measurement, MeasurementAggregate, NotUsed] = 40 | Flow[Measurement] 41 | // maxSubstreams must be larger than the number of locations in the file 42 | .groupBy(420, _.location, allowClosedSubstreamRecreation = true) 43 | .fold(MeasurementAggregate("", 0, 0, 0, 0)) { 44 | (ma: MeasurementAggregate, m: Measurement) => 45 | val count = ma.count + 1 46 | val totalTemp = ma.totalTemp + m.temperature 47 | val minTemp = Math.min(ma.minTemp, m.temperature) 48 | val maxTemp = Math.max(ma.maxTemp, m.temperature) 49 | MeasurementAggregate(m.location, count, totalTemp, minTemp, maxTemp) 50 | } 51 | .mergeSubstreams 52 | 53 | val done = sourceOfRows 54 | .via(csvToMeasurement) 55 | .via(aggregate) 56 | .runWith(Sink.seq) 57 | 58 | done.onComplete { 59 | case Success(seq: Seq[MeasurementAggregate]) => 60 | print("Results: \n") 61 | seq.toList.sortWith(_.location < _.location).foreach(println) 62 | println(s"Run with: " + Runtime.getRuntime.availableProcessors + " cores") 63 | println("Flow Success. About to terminate...") 64 | system.terminate() 65 | case Failure(e) => 66 | println(s"Flow Failure: ${e.getMessage}. About to terminate...") 67 | system.terminate() 68 | } 69 | } 70 | 71 | case class Measurement(location: String, temperature: Float) 72 | 73 | case class MeasurementAggregate(location: String, count: Int, totalTemp: Float, minTemp: Float, maxTemp: Float) { 74 | override def toString = s"Location: $location: AVG: ${totalTemp / count} MIN: $minTemp MAX: $maxTemp" 75 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream/PartitionHubWithDynamicSinks.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.ThrottleMode 6 | import org.apache.pekko.stream.scaladsl.{Keep, PartitionHub, RunnableGraph, Source} 7 | 8 | import scala.concurrent.ExecutionContextExecutor 9 | import scala.concurrent.duration.* 10 | 11 | /** 12 | * Inspired by: 13 | * https://doc.akka.io/docs/akka/current/stream/stream-dynamic.html 14 | * 15 | * Partitioning functions: 16 | * - partitionRoundRobin: Route to only one consumer (incl. slow consumers) 17 | * - partitionToFastestConsumer: Route to the fastest consumer (based on queueSize) 18 | * 19 | */ 20 | object PartitionHubWithDynamicSinks extends App { 21 | implicit val system: ActorSystem = ActorSystem() 22 | implicit val ec: ExecutionContextExecutor = system.dispatcher 23 | 24 | val producer = Source.tick(1.second, 100.millis, "message").zipWith(Source(1 to 100))((a, b) => s"$a-$b") 25 | 26 | // A new instance of the partitioner functions and its state is created for each materialization of the PartitionHub 27 | def partitionRoundRobin(): (PartitionHub.ConsumerInfo, String) => Long = { 28 | var i = -1L 29 | 30 | (info: PartitionHub.ConsumerInfo, _: String) => { 31 | i += 1 32 | info.consumerIdByIdx((i % info.size).toInt) 33 | } 34 | } 35 | 36 | def partitionToFastestConsumer(): (PartitionHub.ConsumerInfo, String) => Long = { 37 | (info: PartitionHub.ConsumerInfo, each: String) => info.consumerIds.minBy(id => info.queueSize(id)) 38 | } 39 | 40 | // Attach a PartitionHub Sink to the producer. This will materialize to a corresponding Source 41 | // We need to use toMat and Keep.right since by default the materialized value to the left is used 42 | val runnableGraph: RunnableGraph[Source[String, NotUsed]] = 43 | producer.toMat(PartitionHub.statefulSink( 44 | //Switch the partitioning function 45 | //() => partitionRoundRobin(), 46 | () => partitionToFastestConsumer(), 47 | startAfterNrOfConsumers = 2, bufferSize = 1))(Keep.right) 48 | 49 | // By running/materializing the producer, we get back a Source, which 50 | // gives us access to the elements published by the producer 51 | val fromProducer: Source[String, NotUsed] = runnableGraph.run() 52 | 53 | // Attach three dynamic fan-out sinks to the PartitionHub 54 | fromProducer.runForeach(msg => println("fast consumer1 received: " + msg)) 55 | fromProducer.throttle(100, 1.millis, 10, ThrottleMode.Shaping) 56 | .runForeach(msg => println("slow consumer2 received: " + msg)) 57 | fromProducer.throttle(100, 2.millis, 10, ThrottleMode.Shaping) 58 | .runForeach(msg => println("really slow consumer3 received: " + msg)) 59 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream/PossiblyEmpty.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.{Sink, Source} 5 | import org.slf4j.{Logger, LoggerFactory} 6 | 7 | import scala.util.{Failure, Success} 8 | 9 | /** 10 | * Inspired by: 11 | * https://stackoverflow.com/questions/70921445/how-to-handle-java-util-nosuchelementexception-reduce-over-empty-stream-akka-st 12 | * 13 | * Gracefully fold/reduce over a possibly empty stream (eg due to filtering) 14 | * 15 | */ 16 | object PossiblyEmpty extends App { 17 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 18 | implicit val system: ActorSystem = ActorSystem() 19 | 20 | import system.dispatcher 21 | 22 | val printSink = Sink.foreach[Int](each => logger.info(s"Reached sink: $each")) 23 | 24 | val possiblyEmpty = Source(Seq(1, 3, 5)) // vs Seq(1, 2, 3, 5) 25 | .filter(_ % 2 == 0) 26 | .fold(0)(_ + _) // fold allows for empty collections (vs reduce) 27 | 28 | // not interested in results 29 | val done = possiblyEmpty.runWith(printSink) 30 | 31 | val results = possiblyEmpty.runWith(Sink.seq) 32 | results.onComplete { 33 | case Success(results) => 34 | logger.info(s"Successfully processed: ${results.size} elements") 35 | system.terminate() 36 | case Failure(exception) => 37 | logger.info(s"The stream failed with: $exception}") 38 | system.terminate() 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream/PublishToBlockingResource.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.DelayOverflowStrategy 6 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 7 | 8 | import java.util.concurrent.{ArrayBlockingQueue, BlockingQueue} 9 | import scala.collection.parallel.CollectionConverters.* 10 | import scala.concurrent.duration.* 11 | import scala.util.Failure 12 | 13 | /** 14 | * n parallel clients -> blocking resource (simulated by java.util.concurrent.BlockingQueue) -> slowSink 15 | * 16 | * Doc: 17 | * https://doc.akka.io/docs/akka/current/stream/operators/Source/unfoldResource.html 18 | */ 19 | object PublishToBlockingResource extends App { 20 | implicit val system: ActorSystem = ActorSystem() 21 | 22 | import system.dispatcher 23 | 24 | val slowSink: Sink[Seq[Int], NotUsed] = 25 | Flow[Seq[Int]] 26 | .delay(1.seconds, DelayOverflowStrategy.backpressure) 27 | .to(Sink.foreach(e => println(s"Reached sink: $e"))) 28 | 29 | val blockingResource: BlockingQueue[Int] = new ArrayBlockingQueue[Int](100) 30 | 31 | // Start a new `Source` from some (third party) blocking resource which can be opened, read and closed 32 | val source: Source[Int, NotUsed] = 33 | Source.unfoldResource[Int, BlockingQueue[Int]]( 34 | () => blockingResource, //open 35 | (q: BlockingQueue[Int]) => Some(q.take()), //read 36 | (_: BlockingQueue[Int]) => {}) //close 37 | 38 | source 39 | .groupedWithin(10, 2.seconds) 40 | .watchTermination()((_, done) => done.onComplete { 41 | case Failure(err) => 42 | println(s"Flow failed: $err") 43 | case each => println(s"Server flow terminated: $each") 44 | }) 45 | .runWith(slowSink) 46 | 47 | // simulate n clients that publish to blockingResource 48 | (1 to 1000).par.foreach(value => blockingResource.put(value)) 49 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream/PublishToSourceQueueFromMultipleThreads.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.* 5 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source, SourceQueueWithComplete} 6 | import org.apache.pekko.{Done, NotUsed} 7 | import org.slf4j.{Logger, LoggerFactory} 8 | 9 | import scala.collection.parallel.CollectionConverters.* 10 | import scala.concurrent.Future 11 | import scala.concurrent.duration.* 12 | import scala.util.{Failure, Success} 13 | 14 | /** 15 | * n parallel publishing clients -> sourceQueue -> slowSink 16 | * 17 | * Doc: 18 | * https://doc.akka.io/docs/akka/current/stream/actor-interop.html?language=scala#source-queue 19 | * 20 | * Doc buffers: 21 | * https://doc.akka.io/docs/akka/current/stream/stream-rate.html#buffers-in-akka-streams 22 | * 23 | * Similar example: [[MergeHubWithDynamicSources]] 24 | * 25 | * Open issue: 26 | * https://github.com/akka/akka/issues/26696 27 | * 28 | * See also: 29 | * [[BoundedSourceQueue]] (= a sync variant of SourceQueue with OverflowStrategy.dropNew) 30 | * 31 | */ 32 | object PublishToSourceQueueFromMultipleThreads extends App { 33 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 34 | implicit val system: ActorSystem = ActorSystem() 35 | 36 | import system.dispatcher 37 | 38 | val bufferSize = 100 39 | val maxConcurrentOffers = 1000 40 | val numberOfPublishingClients = 1000 41 | 42 | val slowSink: Sink[Seq[Int], NotUsed] = 43 | Flow[Seq[Int]] 44 | .delay(2.seconds, DelayOverflowStrategy.backpressure) 45 | .to(Sink.foreach(e => logger.info(s"Reached sink: $e"))) 46 | 47 | val sourceQueue: SourceQueueWithComplete[Int] = 48 | Source 49 | .queue[Int](bufferSize, OverflowStrategy.backpressure, maxConcurrentOffers) 50 | .groupedWithin(10, 1.seconds) 51 | .to(slowSink) 52 | .run() 53 | 54 | val doneConsuming = sourceQueue.watchCompletion() // never completes 55 | signalWhen(doneConsuming, "consuming") 56 | 57 | simulatePublishingFromMultipleThreads() 58 | 59 | private def simulatePublishingFromMultipleThreads(): Unit = { 60 | (1 to numberOfPublishingClients).par.foreach(offerToSourceQueue) 61 | } 62 | 63 | private def offerToSourceQueue(each: Int) = { 64 | sourceQueue.offer(each).map { 65 | case QueueOfferResult.Enqueued => logger.info(s"enqueued $each") 66 | case QueueOfferResult.Dropped => logger.info(s"dropped $each") 67 | case QueueOfferResult.Failure(ex) => logger.info(s"Offer failed: $ex") 68 | case QueueOfferResult.QueueClosed => logger.info("Source Queue closed") 69 | } 70 | } 71 | 72 | private def signalWhen(done: Future[Done], operation: String): Unit = { 73 | done.onComplete { 74 | case Success(_) => 75 | logger.info(s"Finished: $operation") 76 | case Failure(e) => 77 | logger.info(s"Failure: $e About to terminate...") 78 | system.terminate() 79 | } 80 | } 81 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream/SessionWindow.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.* 5 | 6 | import scala.concurrent.Future 7 | import scala.util.{Failure, Success} 8 | 9 | /** 10 | * Split ordered events into `session windows` 11 | * using the event timestamp 12 | * 13 | * Similar to: [[tools.SubtitleTranslator]] 14 | * 15 | */ 16 | object SessionWindow extends App { 17 | implicit val system: ActorSystem = ActorSystem() 18 | 19 | import system.dispatcher 20 | 21 | val maxGap = 5 // between session windows 22 | 23 | case class Event[T](timestamp: Long, data: T) 24 | 25 | private def groupBySessionWindow(maxGap: Long) = 26 | Flow[Event[String]].statefulMap(() => List.empty[Event[String]])( 27 | (stateList, nextElem) => { 28 | val newStateList = stateList :+ nextElem 29 | val lastElem = if (stateList.isEmpty) nextElem else stateList.reverse.head 30 | val calcGap = nextElem.timestamp - lastElem.timestamp 31 | if (calcGap < maxGap) { 32 | // (list for next iteration, list of output elements) 33 | (newStateList, Nil) 34 | } 35 | else { 36 | // (list for next iteration, list of output elements) 37 | (List(nextElem), stateList) 38 | } 39 | }, 40 | // Cleanup function, we return the last stateList 41 | stateList => Some(stateList)) 42 | .filterNot(each => each.isEmpty) 43 | 44 | val input = Source(List( 45 | Event(1, "A"), 46 | Event(7, "B"), Event(8, "C"), 47 | Event(15, "D"), Event(16, "E"), Event(18, "F"), 48 | Event(25, "G"), Event(26, "H"), Event(26, "I"), Event(28, "J"), 49 | Event(32, "K"), 50 | Event(42, "L"), Event(43, "M") 51 | )) 52 | 53 | val result: Future[Seq[List[Event[String]]]] = input 54 | .via(groupBySessionWindow(maxGap)) 55 | .runWith(Sink.seq) 56 | 57 | result.onComplete { 58 | case Success(sessions) => 59 | sessions.foreach(sessionWindow => println(s"Session window with events: ${sessionWindow.mkString}")) 60 | system.terminate() 61 | case Failure(e) => e.printStackTrace() 62 | system.terminate() 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream/TransformCSV.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.commons.io.FileUtils 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.slf4j.{Logger, LoggerFactory} 6 | 7 | import java.io.File 8 | import java.util.concurrent.TimeUnit 9 | import scala.language.postfixOps 10 | import scala.sys.process.* 11 | 12 | /** 13 | * Inspired by: 14 | * https://discuss.lightbend.com/t/transform-a-csv-file-into-multiple-csv-files-using-akka-stream/3142 15 | * 16 | * Instead of pekko-streams we use linux tools (sort/split) for each step to apply the 17 | * "chainsaw style" transformations to the large csv file. 18 | * With the additional count this does the same as: [[FlightDelayStreaming]]. 19 | * However, the performance is not as good as in FlightDelayStreaming. 20 | * 21 | * Remarks: 22 | * - Instead of putting all steps in a shell script, we want to use Scala [[Process]] for each step 23 | * - Doc Scala ProcessBuilder: https://dotty.epfl.ch/api/scala/sys/process/ProcessBuilder.html 24 | */ 25 | object TransformCSV extends App { 26 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 27 | implicit val system: ActorSystem = ActorSystem() 28 | 29 | val sourceFile = "src/main/resources/2008_subset.csv" 30 | val resultsDir = "results" 31 | val countPlaceholder = "_count_" 32 | 33 | val os = System.getProperty("os.name").toLowerCase 34 | if (os == "mac os x") { 35 | split(sort(sourceFile)) 36 | countLines(resultsDir) 37 | } else { 38 | logger.warn("OS not supported") 39 | } 40 | system.terminate() 41 | 42 | 43 | def sort(sourceFile: String): File = { 44 | val op = "sort" 45 | val tmpSortedFile = File.createTempFile("sorted_tmp", ".csv") 46 | 47 | // Remove csv header and use linux sort on the 9th column (= UniqueCarrier) 48 | val resultSort = exec(op) { 49 | (Process(s"tail -n+2 $sourceFile") #| Process(s"sort -t\",\" -k9,9") #>> tmpSortedFile).! 50 | } 51 | logger.info(s"Exit code '$op': $resultSort") 52 | tmpSortedFile 53 | } 54 | 55 | def split(tmpSortedFile: File): Unit = { 56 | val op = "split" 57 | s"rm -rf $resultsDir".! 58 | s"mkdir -p $resultsDir".! 59 | 60 | // Split into files according to value of 9th column (incl. file closing) 61 | // Note that the $ operator is used in the linux cmd 62 | val bashCmdSplit = s"""awk -F ',' '{out=("$resultsDir""" + """/"$9"-_count_.csv")} out!=prev {close(prev)} {print > out; prev=out}' """ + s"$tmpSortedFile" 63 | val resultSplit = exec(op) { 64 | Seq("bash", "-c", bashCmdSplit).! 65 | } 66 | logger.info(s"Exit code '$op': $resultSplit") 67 | } 68 | 69 | def countLines(results: String): Unit = { 70 | val op = "count" 71 | val bashCmdCountLines = s"""wc -l `find $results -type f`""" 72 | val resultCountLines = exec(op) { 73 | Seq("bash", "-c", bashCmdCountLines).!! 74 | } 75 | logger.info(s"Line count report:\n $resultCountLines") 76 | 77 | val reportCleaned = resultCountLines.split("\\s+").tail.reverse.tail.tail 78 | reportCleaned.sliding(2, 2).foreach { each => 79 | val (path, count) = (each.head, each.last) 80 | logger.info(s"About to rename file: $path with count value: $count") 81 | FileUtils.moveFile(FileUtils.getFile(path), FileUtils.getFile(path.replace(countPlaceholder, count))) 82 | } 83 | } 84 | 85 | private def exec[R](op: String = "")(block: => R): R = { 86 | val t0 = System.nanoTime() 87 | val result = block // call-by-name 88 | val t1 = System.nanoTime() 89 | val elapsedTimeMs = TimeUnit.MILLISECONDS.convert(t1 - t0, TimeUnit.NANOSECONDS) 90 | logger.info(s"Elapsed time to '$op': $elapsedTimeMs ms") 91 | result 92 | } 93 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream/TweetExample.scala: -------------------------------------------------------------------------------- 1 | package sample.stream 2 | 3 | import org.apache.commons.lang3.exception.ExceptionUtils 4 | import org.apache.pekko.NotUsed 5 | import org.apache.pekko.actor.{ActorSystem, Cancellable} 6 | import org.apache.pekko.stream.DelayOverflowStrategy 7 | import org.apache.pekko.stream.scaladsl.{Flow, MergePrioritized, Sink, Source} 8 | import org.slf4j.{Logger, LoggerFactory} 9 | 10 | import java.time.{Instant, ZoneId} 11 | import scala.concurrent.duration.* 12 | import scala.util.{Failure, Success} 13 | 14 | /** 15 | * Adapted tweet example from akka streams tutorial, 16 | * added MergePrioritized Feature 17 | * 18 | * Doc: 19 | * https://doc.akka.io/docs/akka/current/stream/operators/Source/combine.html 20 | * https://softwaremill.com/akka-2.5.4-features 21 | */ 22 | 23 | object TweetExample extends App { 24 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 25 | implicit val system: ActorSystem = ActorSystem() 26 | 27 | import system.dispatcher 28 | 29 | final case class Author(handle: String) 30 | 31 | final case class Hashtag(name: String) 32 | 33 | final case class Tweet(author: Author, timestamp: Long, body: String) { 34 | def hashtags: Set[Hashtag] = 35 | body.split(" ").collect { case t if t.startsWith("#") => Hashtag(t) }.toSet 36 | 37 | override def toString: String = { 38 | val localDateTime = Instant.ofEpochMilli(timestamp).atZone(ZoneId.systemDefault()).toLocalDateTime 39 | s"$localDateTime - ${author.handle} tweeted: ${body.take(5)}..." 40 | } 41 | } 42 | 43 | val akkaTag = Hashtag("#akka") 44 | 45 | val tweetsLowPrio: Source[Tweet, Cancellable] = Source.tick(1.second, 200.millis, NotUsed).map(_ => Tweet(Author("LowPrio"), System.currentTimeMillis, "#other #akka aBody")) 46 | val tweetsHighPrio: Source[Tweet, Cancellable] = Source.tick(2.second, 1.second, NotUsed).map(_ => Tweet(Author("HighPrio"), System.currentTimeMillis, "#akka #other aBody")) 47 | val tweetsVeryHighPrio: Source[Tweet, Cancellable] = Source.tick(2.second, 1.second, NotUsed).map(_ => Tweet(Author("VeryHighPrio"), System.currentTimeMillis, "#akka #other aBody")) 48 | 49 | val limitedTweets: Source[Tweet, NotUsed] = Source.combine(tweetsLowPrio, tweetsHighPrio, tweetsVeryHighPrio)(_ => MergePrioritized(List(1, 10, 100))).take(20) 50 | 51 | val processingFlow = Flow[Tweet] 52 | .filter(_.hashtags.contains(akkaTag)) 53 | .wireTap(each => logger.info(s"$each")) 54 | 55 | val slowDownstream = 56 | Flow[Tweet] 57 | .delay(5.seconds, DelayOverflowStrategy.backpressure) 58 | 59 | val processedTweets = 60 | limitedTweets 61 | .via(processingFlow) 62 | .via(slowDownstream) 63 | .runWith(Sink.seq) 64 | 65 | processedTweets.onComplete { 66 | case Success(results) => 67 | logger.info(s"Successfully processed: ${results.size} tweets") 68 | system.terminate() 69 | case Failure(exception) => 70 | logger.info(s"The stream failed with: ${ExceptionUtils.getRootCause(exception)}") 71 | system.terminate() 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor/SimulateWindTurbines.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.pattern.{BackoffOpts, BackoffSupervisor} 5 | import org.apache.pekko.stream.ThrottleMode 6 | import org.apache.pekko.stream.scaladsl.{Sink, Source} 7 | 8 | import scala.concurrent.duration.* 9 | 10 | /** 11 | * Sample Implementation of: 12 | * http://blog.colinbreck.com/integrating-akka-streams-and-akka-actors-part-ii 13 | * 14 | * Starts n [[WindTurbineSimulator]], which generate [[WindTurbineData]] 15 | * Uses a [[BackoffSupervisor]] as level of indirection 16 | * 17 | * The server is started with [[WindTurbineServer]] 18 | */ 19 | object SimulateWindTurbines extends App { 20 | implicit val system: ActorSystem = ActorSystem() 21 | 22 | val endpoint = "ws://127.0.0.1:8080" 23 | val numberOfTurbines = 5 24 | Source(1 to numberOfTurbines) 25 | .throttle( 26 | elements = 100, //number of elements to be taken from bucket 27 | per = 1.second, 28 | maximumBurst = 100, //capacity of bucket 29 | mode = ThrottleMode.shaping 30 | ) 31 | .map { _ => 32 | val id = java.util.UUID.randomUUID.toString 33 | 34 | val supervisor = BackoffSupervisor.props( 35 | BackoffOpts.onFailure( 36 | WindTurbineSimulator.props(id, endpoint), 37 | childName = id, 38 | minBackoff = 1.second, 39 | maxBackoff = 30.seconds, 40 | randomFactor = 0.2 41 | )) 42 | 43 | system.actorOf(supervisor, name = s"$id-backoff-supervisor") 44 | } 45 | .runWith(Sink.ignore) 46 | } 47 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor/Total.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor 2 | 3 | import org.apache.pekko.Done 4 | import org.apache.pekko.actor.Actor 5 | import sample.stream_actor.Total.Increment 6 | 7 | import java.text.SimpleDateFormat 8 | import java.util.{Date, TimeZone} 9 | 10 | object Total { 11 | case class Increment(value: Long, avg: Double, id: String) 12 | } 13 | 14 | class Total extends Actor { 15 | var total: Long = 0 16 | 17 | override def receive: Receive = { 18 | case Increment(value, avg, id) => 19 | println(s"Received: $value new measurements from turbine with id: $id - Avg wind speed is: $avg") 20 | total = total + value 21 | 22 | val date = new Date() 23 | val df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") 24 | df.setTimeZone(TimeZone.getTimeZone("Europe/Zurich")) 25 | 26 | println(s"${df.format(date) } - Current total of all measurements: $total") 27 | sender() ! Done 28 | } 29 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor/WebSocketClient.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.{ActorRef, ActorSystem} 5 | import org.apache.pekko.http.scaladsl.Http 6 | import org.apache.pekko.http.scaladsl.model.StatusCodes 7 | import org.apache.pekko.http.scaladsl.model.ws.* 8 | import org.apache.pekko.stream.scaladsl.{Flow, GraphDSL, Keep, Sink, Source} 9 | import org.apache.pekko.stream.{FlowShape, Graph, SourceShape} 10 | import sample.stream_actor.WindTurbineSimulator.* 11 | 12 | import scala.concurrent.duration.* 13 | import scala.concurrent.{ExecutionContext, Future} 14 | import scala.util.{Failure, Success} 15 | 16 | object WebSocketClient { 17 | def apply(id: String, endpoint: String, windTurbineSimulator: ActorRef) 18 | (implicit 19 | system: ActorSystem, 20 | executionContext: ExecutionContext): WebSocketClient = { 21 | new WebSocketClient(id, endpoint, windTurbineSimulator)(system, executionContext) 22 | } 23 | } 24 | 25 | class WebSocketClient(id: String, endpoint: String, windTurbineSimulator: ActorRef) 26 | (implicit 27 | system: ActorSystem, 28 | executionContext: ExecutionContext) { 29 | 30 | 31 | val webSocketFlow: Flow[Message, Message, Future[WebSocketUpgradeResponse]] = { 32 | val websocketUri = s"$endpoint/measurements/$id" 33 | Http().webSocketClientFlow(WebSocketRequest(websocketUri)) 34 | } 35 | 36 | val outgoing: Graph[SourceShape[TextMessage.Strict], NotUsed] = GraphDSL.create() { implicit builder => 37 | val data = WindTurbineData(id) 38 | 39 | val flow = builder.add { 40 | Source.tick(1.second, 100.millis,()) //valve for the WindTurbineData frequency 41 | .map(_ => TextMessage(data.getNext)) 42 | } 43 | 44 | SourceShape(flow.out) 45 | } 46 | 47 | val incoming: Graph[FlowShape[Message, Unit], NotUsed] = GraphDSL.create() { implicit builder => 48 | val flow = builder.add { 49 | Flow[Message] 50 | .collect { 51 | case TextMessage.Strict(text) => 52 | Future.successful(text) 53 | case TextMessage.Streamed(textStream) => 54 | textStream.runFold("")(_ + _) 55 | .flatMap(Future.successful) 56 | } 57 | .mapAsync(1)(identity) 58 | .map(each => println(s"Client received msg: $each")) 59 | } 60 | 61 | FlowShape(flow.in, flow.out) 62 | } 63 | 64 | val (upgradeResponse, closed) = Source.fromGraph(outgoing) 65 | .viaMat(webSocketFlow)(Keep.right) // keep the materialized Future[WebSocketUpgradeResponse] 66 | .via(incoming) 67 | .toMat(Sink.ignore)(Keep.both) // also keep the Future[Done] 68 | .run() 69 | 70 | 71 | val connected: Future[Unit] = 72 | upgradeResponse.map { upgrade => 73 | upgrade.response.status match { 74 | case StatusCodes.SwitchingProtocols => windTurbineSimulator ! Upgraded 75 | case statusCode => windTurbineSimulator ! FailedUpgrade(statusCode) 76 | } 77 | } 78 | 79 | connected.onComplete { 80 | case Success(_) => windTurbineSimulator ! Connected 81 | case Failure(ex) => windTurbineSimulator ! ConnectionFailure(ex) 82 | } 83 | 84 | closed.map { _ => 85 | windTurbineSimulator ! Terminated 86 | } 87 | closed.onComplete { 88 | case Success(_) => windTurbineSimulator ! Connected 89 | case Failure(ex) => windTurbineSimulator ! ConnectionFailure(ex) 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor/WindTurbineData.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor 2 | 3 | import scala.util.Random 4 | 5 | case class MeasurementsContainer(id: String = "", timestamp: Long = 0, measurements: Measurements = Measurements()) 6 | 7 | case class Measurements(power: Double = 0, rotor_speed : Double = 0, wind_speed: Double = 0) 8 | 9 | object WindTurbineData { 10 | def apply(id: String) = new WindTurbineData(id) 11 | } 12 | 13 | class WindTurbineData(id: String) { 14 | val random: Random.type = Random 15 | 16 | def getNext: String = { 17 | val timestamp = System.currentTimeMillis / 1000 18 | val power: String = f"${random.nextDouble() * 10}%.2f".replace(",", ".") 19 | val rotorSpeed = f"${random.nextDouble() * 10}%.2f".replace(",", ".") 20 | val windSpeed = f"${random.nextDouble() * 100}%.2f".replace(",", ".") 21 | 22 | s"""{ 23 | | "id": "$id", 24 | | "timestamp": $timestamp, 25 | | "measurements": { 26 | | "power": $power, 27 | | "rotor_speed": $rotorSpeed, 28 | | "wind_speed": $windSpeed 29 | | } 30 | |}""".stripMargin 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor/WindTurbineSimulator.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor 2 | 3 | import org.apache.pekko.actor.{Actor, ActorLogging, ActorSystem, Props} 4 | import org.apache.pekko.http.scaladsl.model.StatusCode 5 | import sample.stream_actor.WindTurbineSimulator.* 6 | 7 | import scala.concurrent.ExecutionContextExecutor 8 | 9 | case class WindTurbineSimulatorException(id: String) extends RuntimeException 10 | 11 | /** 12 | * WindTurbineSimulator starts the [[WebSocketClient]] and coordinates 13 | * issues during: 14 | * - startup 15 | * - running 16 | */ 17 | object WindTurbineSimulator { 18 | def props(id: String, endpoint: String): Props = 19 | Props(new WindTurbineSimulator(id, endpoint)) 20 | 21 | final case object Upgraded 22 | final case object Connected 23 | final case object Terminated 24 | final case class ConnectionFailure(ex: Throwable) 25 | final case class FailedUpgrade(statusCode: StatusCode) 26 | } 27 | 28 | class WindTurbineSimulator(id: String, endpoint: String) 29 | extends Actor with ActorLogging { 30 | implicit private val system: ActorSystem = context.system 31 | implicit private val executionContext: ExecutionContextExecutor = system.dispatcher 32 | 33 | val webSocketClient: WebSocketClient = WebSocketClient(id, endpoint, self) 34 | 35 | override def receive: Receive = startup //initial state 36 | 37 | private def startup: Receive = { 38 | case Upgraded => 39 | log.info(s"$id : WebSocket upgraded") 40 | case FailedUpgrade(statusCode) => 41 | log.error(s"$id : Failed to upgrade WebSocket connection: $statusCode") 42 | throw WindTurbineSimulatorException(id) 43 | case ConnectionFailure(ex) => 44 | log.error(s"$id : Failed to establish WebSocket connection: $ex") 45 | throw WindTurbineSimulatorException(id) 46 | case Connected => 47 | log.info(s"$id : WebSocket connected") 48 | context.become(running) 49 | } 50 | 51 | private def running: Receive = { 52 | case Terminated => 53 | log.error(s"$id : WebSocket connection terminated") 54 | throw WindTurbineSimulatorException(id) 55 | case ConnectionFailure(ex) => 56 | log.error(s"$id : ConnectionFailure occurred: $ex") 57 | throw WindTurbineSimulatorException(id) 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor/typed/CustomCache.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor.typed 2 | 3 | import org.apache.pekko.actor.typed.scaladsl.Behaviors 4 | import org.apache.pekko.actor.typed.{ActorRef, Behavior} 5 | 6 | case class DeviceId(id: String) 7 | 8 | object CustomCache { 9 | sealed trait CacheRequests 10 | final case class Get(requestId: String, replyTo: ActorRef[CacheResponses]) extends CacheRequests 11 | final case class Devices(devices: List[DeviceId]) extends CacheRequests 12 | final case class AddDevices(devices: List[DeviceId]) extends CacheRequests 13 | 14 | sealed trait CacheResponses 15 | final case object EmptyCache extends CacheResponses 16 | final case class CachedDevices(devices: List[DeviceId]) extends CacheResponses 17 | 18 | val empty: Behavior[CacheRequests] = 19 | Behaviors.receive[CacheRequests] { (context, message) => 20 | message match { 21 | case Get(requestId, replyTo) => 22 | context.log.info(s"Empty cache request for requestId $requestId") 23 | replyTo ! EmptyCache 24 | Behaviors.same 25 | case Devices(devices) => 26 | context.log.info(s"Initializing cache with: ${devices.size} devices") 27 | cached(devices) 28 | case AddDevices(devices) => 29 | context.log.info(s"Initializing cache with: ${devices.size} devices") 30 | cached(devices) 31 | } 32 | } 33 | 34 | private def cached(devices: List[DeviceId]): Behavior[CacheRequests] = 35 | Behaviors.receive { (context, message) => 36 | message match { 37 | case Get(requestId, replyTo) => 38 | context.log.info(s"Cache request for requestId $requestId") 39 | replyTo ! CachedDevices(devices) 40 | Behaviors.same 41 | case Devices(updatedDevices) => 42 | context.log.info(s"Updating cache with: ${updatedDevices.size} devices") 43 | cached(updatedDevices) 44 | case AddDevices(updatedDevices) => 45 | context.log.info(s"Adding: ${updatedDevices.size} device(s)") 46 | cached(devices = devices ++ updatedDevices) 47 | } 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor/typed/CustomCacheRunner.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor.typed 2 | 3 | import org.apache.pekko.actor.typed.ActorSystem 4 | import org.apache.pekko.actor.typed.scaladsl.AskPattern.* 5 | import org.apache.pekko.stream.ThrottleMode 6 | import org.apache.pekko.stream.scaladsl.{Sink, Source} 7 | import org.apache.pekko.util.Timeout 8 | import sample.stream_actor.typed.CustomCache.{AddDevices, CacheRequests, CacheResponses, CachedDevices} 9 | 10 | import scala.concurrent.duration.* 11 | import scala.concurrent.{ExecutionContextExecutor, Future} 12 | 13 | /** Use typed actor [[CustomCache]] to show shared state: 14 | * - Request-Response with ask from outside (= a stream) 15 | * - Request with tell from outside (= a stream) 16 | * 17 | * Inspired by: 18 | * - https://blog.colinbreck.com/rethinking-streaming-workloads-with-akka-streams-part-iii 19 | * 20 | * Doc ask: 21 | * - https://doc.akka.io/docs/akka/current/typed/interaction-patterns.html#request-response-with-ask-from-outside-an-actor 22 | */ 23 | object CustomCacheRunner extends App { 24 | // the system is also the top level actor ref 25 | implicit val cache: ActorSystem[CacheRequests] = ActorSystem[CacheRequests](CustomCache.empty, "CustomCache") 26 | implicit val ec: ExecutionContextExecutor = cache.executionContext 27 | implicit val timeout: Timeout = 5.seconds 28 | 29 | //Request-Response with ask 30 | Source 31 | .tick(initialDelay = 0.seconds, interval = 2.seconds, tick = ()) 32 | .mapAsync(parallelism = 1)(_ => cache.ref.ask(ref => CustomCache.Get("42", ref))) 33 | .map((each: CacheResponses) => 34 | each match { 35 | case cachedDevices: CachedDevices => cache.log.info(s"Current amount of cached devices: ${cachedDevices.devices.size}") 36 | case _ => cache.log.info("No devices") 37 | }) 38 | .recover { 39 | case ex => cache.log.error("Failed to read cached devices: ", ex) 40 | } 41 | .runWith(Sink.ignore) 42 | 43 | //Request with tell 44 | Source(LazyList.continually(java.util.UUID.randomUUID.toString).take(100)) 45 | .throttle(10, 1.second, 10, ThrottleMode.shaping) 46 | .mapAsync(parallelism = 10)(each => Future(cache ! AddDevices(List(DeviceId(each))))) 47 | .runWith(Sink.ignore) 48 | } 49 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor_simple/EncapsulateStreamWithActor.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor_simple 2 | 3 | import org.apache.pekko.actor.{ActorSystem, Props} 4 | 5 | import scala.concurrent.ExecutionContextExecutor 6 | import scala.concurrent.duration.* 7 | 8 | /** 9 | * Inspired by: 10 | * http://blog.colinbreck.com/integrating-akka-streams-and-akka-actors-part-ii 11 | * 12 | */ 13 | object EncapsulateStreamWithActor extends App { 14 | implicit val system: ActorSystem = ActorSystem() 15 | implicit val executionContext: ExecutionContextExecutor = system.dispatcher 16 | 17 | val actorRef = system.actorOf(Props(classOf[PrintMoreNumbers])) 18 | system.scheduler.scheduleOnce(10.seconds) { 19 | actorRef ! "stop" 20 | } 21 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream_actor_simple/PrintMoreNumbers.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_actor_simple 2 | 3 | import org.apache.pekko.actor.{Actor, ActorSystem} 4 | import org.apache.pekko.stream.scaladsl.{Keep, Sink, Source} 5 | import org.apache.pekko.stream.{KillSwitches, UniqueKillSwitch} 6 | 7 | import scala.concurrent.ExecutionContextExecutor 8 | import scala.concurrent.duration.* 9 | 10 | class PrintMoreNumbers extends Actor { 11 | implicit val system: ActorSystem = context.system 12 | implicit val executionContext: ExecutionContextExecutor = context.system.dispatcher 13 | 14 | private val (killSwitch: UniqueKillSwitch, done) = 15 | Source.tick(0.seconds, 1.second, 1) 16 | .scan(0)(_ + _) 17 | .map(_.toString) 18 | .viaMat(KillSwitches.single)(Keep.right) 19 | .toMat(Sink.foreach(println))(Keep.both) 20 | .run() 21 | 22 | done.map(_ => self ! "done") 23 | 24 | override def receive: Receive = { 25 | //When the actor is stopped, it will also stop the stream 26 | case "stop" => 27 | println("Stopping...") 28 | killSwitch.shutdown() 29 | case "done" => 30 | println("Done") 31 | context.stop(self) 32 | context.system.terminate() 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_divert/AlsoTo.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_divert 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.event.{Logging, LoggingAdapter} 5 | import org.apache.pekko.stream.Attributes 6 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 7 | 8 | /** 9 | * Shows the async nature of the alsoTo operator. 10 | * Uses the provided akka streams [[akka.event.LoggingAdapter]] (backed by slf4j/logback) 11 | * 12 | * Inspired by: 13 | * https://blog.softwaremill.com/akka-streams-pitfalls-to-avoid-part-2-f93e60746c58 14 | * https://doc.akka.io/docs/akka/current/stream/stream-cookbook.html?language=scala#logging-in-streams 15 | * 16 | */ 17 | 18 | object AlsoTo extends App { 19 | implicit val system: ActorSystem = ActorSystem() 20 | implicit val adapter: LoggingAdapter = Logging(system, this.getClass) 21 | 22 | import system.dispatcher 23 | 24 | val source = Source(1 to 10) 25 | 26 | val sink = Sink.foreach((value: Int) => adapter.log(Logging.InfoLevel, s" --> Element: $value reached sink")) 27 | 28 | def sinkBlocking = Sink.foreach { (value: Int) => 29 | Thread.sleep(1000) 30 | adapter.log(Logging.InfoLevel, s" --> Element: $value logged in alsoTo sinkBlocking by ${Thread.currentThread().getName}") 31 | } 32 | 33 | val flow = Flow[Int] 34 | .log("before alsoTo") 35 | .alsoTo(sinkBlocking) 36 | .log("after alsoTo") 37 | .withAttributes( 38 | Attributes.logLevels( 39 | onElement = Logging.InfoLevel, 40 | onFinish = Logging.InfoLevel, 41 | onFailure = Logging.DebugLevel 42 | )) 43 | 44 | val done = source.via(flow).runWith(sink) 45 | done.onComplete(_ => system.terminate()) 46 | } 47 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_divert/DivertTo.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_divert 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 6 | 7 | import scala.concurrent.Future 8 | import scala.util.{Failure, Success} 9 | 10 | /** 11 | * Inspired by: 12 | * Colin Breck talk Scala Days NY 2018 13 | * 14 | * Concepts: 15 | * - treat errors as data by using the Either type 16 | * - divert invalid elements at the end (instead of filtering/dropping earlier) 17 | * - keep order of elements downstream 18 | * 19 | * Trade-off of this approach: Needs pattern matching on all downstream operations 20 | * 21 | * See also: 22 | * https://bszwej.medium.com/akka-streams-error-handling-7ff9cc01bc12 23 | */ 24 | object DivertTo extends App { 25 | implicit val system: ActorSystem = ActorSystem() 26 | 27 | import system.dispatcher 28 | 29 | val source = Source(1 to 10) 30 | 31 | val sink = Sink.foreach[Either[Valid[Int], Invalid[Int]]](each => println(s"Reached sink: ${each.swap.getOrElse(0)}")) 32 | 33 | private val errorSink = Flow[Invalid[Int]] 34 | .map(each => println(s"Reached errorSink: $each")) 35 | .to(Sink.ignore) 36 | 37 | val flow: Flow[Int, Either[Valid[Int], Invalid[Int]], NotUsed] = Flow[Int] 38 | .map { x => 39 | if (x % 2 == 0) Left(Valid(x)) 40 | else Right(Invalid(x, Some(new Exception("Is odd")))) 41 | } 42 | .map { 43 | case left@Left(_) => businessLogicOn(left) 44 | case right@Right(_) => right 45 | } 46 | .map { 47 | case left@Left(_) => left 48 | case right@Right(_) => right 49 | } 50 | // Divert invalid elements 51 | // contramap: apply "getOrElse" to each incoming upstream element *before* it is passed to the errorSink 52 | .divertTo(errorSink.contramap(_.getOrElse(Invalid(0, Some(new Exception("N/A"))))), _.isRight) 53 | 54 | private def businessLogicOn(left: Left[Valid[Int], Invalid[Int]]) = { 55 | if (left.value.payload > 5) left 56 | else Right(Invalid(left.value.payload, Some(new Exception("Is smaller than 5")))) 57 | } 58 | 59 | val done = source.via(flow).runWith(sink) 60 | terminateWhen(done) 61 | 62 | 63 | def terminateWhen(done: Future[?]): Unit = { 64 | done.onComplete { 65 | case Success(_) => 66 | println("Flow Success. About to terminate...") 67 | system.terminate() 68 | case Failure(e) => 69 | println(s"Flow Failure: $e. About to terminate...") 70 | system.terminate() 71 | } 72 | } 73 | } 74 | 75 | case class Valid[T](payload: T) 76 | 77 | case class Invalid[T](payload: T, cause: Option[Throwable]) 78 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/Blacklist.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.* 5 | import org.apache.pekko.stream.scaladsl.{Keep, Sink, Source} 6 | import org.apache.pekko.stream.stage.* 7 | 8 | import scala.concurrent.duration.* 9 | import scala.language.reflectiveCalls 10 | 11 | /** 12 | * Source -> Flow(Blacklist) -> Sink 13 | * Inject shared state (eg Blacklist) from outside the flow execution 14 | * 15 | * Implementation doc: 16 | * https://doc.akka.io/docs/akka/current/stream/stream-customize.html#custom-materialized-values 17 | * 18 | * Similar to [[ParametrizedFlow]] 19 | * 20 | * Sample Implementation of discussion: 21 | * https://discuss.lightbend.com/t/the-idiomatic-way-to-manage-shared-state-with-akka-streams/2552 22 | */ 23 | 24 | object Blacklist extends App { 25 | implicit val system: ActorSystem = ActorSystem() 26 | 27 | val initBlacklist = Set.empty[String] 28 | 29 | val service: StateService[Set[String]] = 30 | Source.repeat("yes") 31 | .throttle(1, 1.second, 10, ThrottleMode.shaping) 32 | .viaMat(new ZipWithState(initBlacklist))(Keep.right) 33 | .filterNot { case (blacklist: Set[String], elem: String) => blacklist(elem) } 34 | .to(Sink.foreach(each => println(each._2))) 35 | .run() 36 | 37 | println("Starting with empty blacklist on a list of 'yes' elements -> elements are passing") 38 | 39 | Thread.sleep(2000) 40 | println("Inject new blacklist with value: 'yes' -> elements are filtered") 41 | service.update(Set("yes")) 42 | 43 | Thread.sleep(5000) 44 | println("Inject new blacklist with value: 'no' -> elements are passing again") 45 | service.update(Set("no")) 46 | } 47 | 48 | 49 | trait StateService[A] { 50 | def update(state: A): Unit 51 | } 52 | 53 | class StateServiceCallback[A](callback: AsyncCallback[A]) extends StateService[A] { 54 | override def update(state: A): Unit = callback.invoke(state) 55 | } 56 | 57 | class ZipWithState[S, I](initState: S) extends GraphStageWithMaterializedValue[FlowShape[I, (S, I)], StateService[S]] { 58 | val in: Inlet[I] = Inlet[I]("ZipWithState.in") 59 | val out: Outlet[(S, I)] = Outlet[(S, I)]("ZipWithState.out") 60 | 61 | override val shape: FlowShape[I, (S, I)] = FlowShape.of(in, out) 62 | 63 | override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, StateService[S]) = { 64 | val logic = new GraphStageLogic(shape) { 65 | private[this] var state: S = initState 66 | val updateStateCallback: AsyncCallback[S] = 67 | getAsyncCallback[S] { 68 | state = _ 69 | } 70 | 71 | setHandler(in, new InHandler { 72 | override def onPush(): Unit = { 73 | push(out, (state, grab(in))) 74 | } 75 | }) 76 | 77 | setHandler(out, new OutHandler { 78 | override def onPull(): Unit = { 79 | pull(in) 80 | } 81 | }) 82 | } 83 | 84 | (logic, new StateServiceCallback(logic.updateStateCallback)) 85 | } 86 | } -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/Dedupe.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.{Sink, Source} 5 | import org.slf4j.{Logger, LoggerFactory} 6 | 7 | import java.util.concurrent.ThreadLocalRandom 8 | import scala.concurrent.Future 9 | import scala.util.{Failure, Success} 10 | 11 | /** 12 | * Drop identical (consecutive or non-consecutive) elements in an 13 | * unbounded stream using the (re-implemented) squbs `Deduplicate` GraphStage: 14 | * https://squbs.readthedocs.io/en/latest/deduplicate 15 | * 16 | * More general than: [[DeduplicateConsecutiveElements]] 17 | * 18 | * Similar example implemented with Apache Flink: 19 | * https://github.com/pbernet/flink-scala-3/blob/main/src/main/scala/com/ververica/Example_05_DataStream_Deduplicate.scala 20 | */ 21 | object Dedupe extends App { 22 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 23 | implicit val system: ActorSystem = ActorSystem() 24 | 25 | import system.dispatcher 26 | 27 | val maxRandomNumber = 50 28 | // use take() for testing 29 | val sourceOfRndElements = Source.fromIterator(() => Iterator.continually(Element(ThreadLocalRandom.current().nextInt(maxRandomNumber), "payload"))).take(100) 30 | 31 | val done = sourceOfRndElements 32 | .wireTap(each => logger.info(s"Before: $each")) 33 | // When duplicateCount is reached: 34 | // Remove element from internal registry/cache of already seen elements to prevent the registry growing unboundedly 35 | .via(Deduplicate((el: Element) => el.id, duplicateCount = 2)) 36 | .wireTap(each => logger.info(s"After: $each")) 37 | .runWith(Sink.ignore) 38 | 39 | terminateWhen(done) 40 | 41 | def terminateWhen(done: Future[?]): Unit = { 42 | done.onComplete { 43 | case Success(_) => 44 | logger.info("Flow Success. About to terminate...") 45 | system.terminate() 46 | case Failure(e) => 47 | logger.error(s"Flow Failure: $e. About to terminate...") 48 | system.terminate() 49 | } 50 | } 51 | 52 | case class Element(id: Int, payload: String) { 53 | override def toString = s"$id" 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/Deduplicate.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.event.Logging 4 | import org.apache.pekko.stream.* 5 | import org.apache.pekko.stream.ActorAttributes.SupervisionStrategy 6 | import org.apache.pekko.stream.Supervision.Decider 7 | import org.apache.pekko.stream.stage.{GraphStage, GraphStageLogic, InHandler, OutHandler} 8 | 9 | import scala.compat.java8.FunctionConverters.* 10 | import scala.util.control.NonFatal 11 | 12 | object Deduplicate { 13 | 14 | def apply[T, U](key: T => U, duplicateCount: Long) = 15 | new Deduplicate[T, U](key, duplicateCount, new java.util.HashMap[U, MutableLong]()) 16 | 17 | def apply[T, U](key: T => U, duplicateCount: Long, registry: java.util.Map[U, MutableLong]) = 18 | new Deduplicate[T, U](key, duplicateCount, registry) 19 | 20 | def apply[T](duplicateCount: Long = Long.MaxValue, 21 | registry: java.util.Map[T, MutableLong] = new java.util.HashMap[T, MutableLong]()): Deduplicate[T, T] = 22 | Deduplicate(t => t, duplicateCount, registry) 23 | } 24 | 25 | /** 26 | * Only pass on those elements that have not been seen so far. 27 | * 28 | * '''Emits when''' the element is not a duplicate 29 | * 30 | * '''Backpressures when''' the element is not a duplicate and downstream backpressures 31 | * 32 | * '''Completes when''' upstream completes 33 | * 34 | * '''Cancels when''' downstream cancels 35 | * 36 | * Ripped and migrated because of squbs EOL: 37 | * https://github.com/paypal/squbs/blob/master/squbs-ext/src/main/scala/org/squbs/streams/Deduplicate.scala 38 | * 39 | */ 40 | final class Deduplicate[T, U](key: T => U, duplicateCount: Long = Long.MaxValue, 41 | registry: java.util.Map[U, MutableLong] = new java.util.HashMap[U, MutableLong]()) 42 | extends GraphStage[FlowShape[T, T]] { 43 | 44 | require(duplicateCount >= 2) 45 | 46 | val in: Inlet[T] = Inlet[T](Logging.simpleName(this) + ".in") 47 | val out: Outlet[T] = Outlet[T](Logging.simpleName(this) + ".out") 48 | override val shape: FlowShape[T, T] = FlowShape(in, out) 49 | 50 | override def toString: String = "Deduplicate" 51 | 52 | override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = 53 | new GraphStageLogic(shape) with OutHandler with InHandler { 54 | def decider: Decider = inheritedAttributes.get[SupervisionStrategy].map(_.decider).getOrElse(Supervision.stoppingDecider) 55 | 56 | override def onPush(): Unit = { 57 | try { 58 | val elem = grab(in) 59 | val counter = registry.merge(key(elem), MutableLong(1), asJavaBiFunction((old, _) => { 60 | pull(in) 61 | if (old.increment() == duplicateCount) null else old 62 | })) 63 | if (counter != null && counter.value == 1) { 64 | push(out, elem) 65 | } 66 | } catch { 67 | case NonFatal(ex) => decider(ex) match { 68 | case Supervision.Stop => failStage(ex) 69 | case _ => pull(in) 70 | } 71 | } 72 | } 73 | 74 | override def onPull(): Unit = pull(in) 75 | 76 | setHandlers(in, out, this) 77 | } 78 | } 79 | 80 | /** 81 | * [[MutableLong]] is used to avoid boxing/unboxing and also 82 | * to avoid [[java.util.Map#put]] operation to increment the counters in the registry. 83 | * 84 | * @param value 85 | */ 86 | case class MutableLong(var value: Long = 0L) { 87 | def increment(): Long = { 88 | value += 1 89 | value 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/DeduplicateConsecutiveElements.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 5 | 6 | /** 7 | * Inspired by: 8 | * https://github.com/akka/akka/issues/19395 9 | * 10 | * Deduplicate consecutive elements in a stream using the sliding operator 11 | * 12 | * See also: [[Dedupe]] and [[LocalFileCacheCaffeine]] 13 | */ 14 | object DeduplicateConsecutiveElements extends App { 15 | implicit val system: ActorSystem = ActorSystem() 16 | 17 | val source = Source(List(1, 1, 1, 2, 2, 1, 2, 2, 2, 3, 4, 4, 5, 6)) 18 | 19 | val flow = Flow[Int] 20 | .sliding(2, 1) 21 | .mapConcat { case prev +: current +: _ => 22 | if (prev == current) Nil 23 | else List(current) 24 | } 25 | 26 | // prepend this value to the source to avoid losing the first value 27 | val ignoredValue = Int.MinValue 28 | val prependedSource = Source.single(ignoredValue).concat(source) 29 | prependedSource.via(flow).runWith(Sink.foreach(println)) 30 | } 31 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/DeferredStreamCreation.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.NotUsed 4 | import org.apache.pekko.actor.ActorSystem 5 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 6 | import org.slf4j.{Logger, LoggerFactory} 7 | 8 | import scala.concurrent.Future 9 | import scala.util.{Failure, Success} 10 | 11 | /** 12 | * Doc: 13 | * https://doc.akka.io/docs/akka/current/stream/operators/Flow/futureFlow.html? 14 | * https://doc.akka.io/docs/akka/current/stream/operators/Source-or-Flow/prefixAndTail.html 15 | * 16 | * Deferred stream creation of tail elements based on the first element 17 | * 18 | * Similar to: [[HandleFirstElementSpecially]] 19 | */ 20 | object DeferredStreamCreation extends App { 21 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 22 | implicit val system: ActorSystem = ActorSystem() 23 | 24 | import system.dispatcher 25 | 26 | val source = Source(List(1, 2, 3, 4, 5)) 27 | val printSink = Sink.foreach[String](each => println(s"Reached sink: $each")) 28 | 29 | def processingFlow(id: Int): Future[Flow[Int, String, NotUsed]] = { 30 | println("About to process tail elements...") 31 | Thread.sleep(2000) 32 | Future(Flow[Int].map(n => s"head element: $id, tail element: $n")) 33 | } 34 | 35 | val doneDelayed = 36 | source.prefixAndTail(1).flatMapConcat { 37 | case (Seq(id), tailSource) => 38 | // process all tail elements once the first element is here 39 | tailSource.via(Flow.futureFlow(processingFlow(id))) 40 | }.runWith(printSink) 41 | 42 | terminateWhen(doneDelayed) 43 | 44 | def terminateWhen(done: Future[?]): Unit = { 45 | done.onComplete { 46 | case Success(_) => 47 | println("Flow Success. About to terminate...") 48 | system.terminate() 49 | case Failure(e) => 50 | println(s"Flow Failure: $e. About to terminate...") 51 | system.terminate() 52 | } 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/FileLister.java: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state; 2 | 3 | import org.slf4j.Logger; 4 | import org.slf4j.LoggerFactory; 5 | 6 | import java.io.File; 7 | import java.io.IOException; 8 | import java.nio.file.Files; 9 | import java.nio.file.Path; 10 | import java.nio.file.attribute.FileTime; 11 | import java.util.Comparator; 12 | import java.util.List; 13 | import java.util.concurrent.atomic.AtomicInteger; 14 | import java.util.stream.Collectors; 15 | import java.util.stream.Stream; 16 | 17 | /** 18 | * Read files ordered by lastModified, so new files are loaded at the end 19 | * We can not use the alpakka-file connector here, because an unbounded stream does not support ordering 20 | */ 21 | public class FileLister { 22 | private static final Logger LOGGER = LoggerFactory.getLogger(FileLister.class); 23 | 24 | public List run(File directory) { 25 | AtomicInteger filesCounter = new AtomicInteger(0); 26 | List resultList = null; 27 | 28 | try (Stream walk = Files.walk(directory.toPath(), 2)) { 29 | resultList = walk 30 | .filter(Files::isRegularFile) 31 | .filter(path -> path.toString().endsWith(".zip")) 32 | .sorted(Comparator.comparing(zipFile -> { 33 | try { 34 | return Files.getLastModifiedTime(zipFile); 35 | } catch (IOException e) { 36 | LOGGER.warn("Cannot access last modified time of '{}'. It will be assigned as the " + 37 | "most recent file to minimize the risk that it will be evicted earlier than it should.", zipFile, e); 38 | return FileTime.fromMillis(System.currentTimeMillis()); 39 | } 40 | })) 41 | .map(each -> { 42 | filesCounter.incrementAndGet(); 43 | return each; 44 | }) 45 | .collect(Collectors.toList()); 46 | 47 | LOGGER.info("Loaded {} files from directory {}", filesCounter.get(), directory); 48 | } catch (IOException e) { 49 | LOGGER.error("Error while loading file paths from directory {}. Please check file system permissions", directory, e); 50 | } 51 | return resultList; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/HandleFirstElementSpecially.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.{Flow, Sink, Source} 5 | import org.slf4j.{Logger, LoggerFactory} 6 | 7 | import scala.concurrent.Future 8 | import scala.util.{Failure, Success} 9 | 10 | 11 | /** 12 | * Inspired by: 13 | * https://stackoverflow.com/questions/40743047/handle-akka-streams-first-element-specially 14 | * Doc: 15 | * https://doc.akka.io/docs/akka/current/stream/operators/Source-or-Flow/prefixAndTail.html 16 | * 17 | * Similar to: [[DeferredStreamCreation]] 18 | * 19 | */ 20 | object HandleFirstElementSpecially extends App { 21 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 22 | implicit val system: ActorSystem = ActorSystem() 23 | 24 | import system.dispatcher 25 | 26 | val source = Source(List(1, 2, 3, 4, 5)) 27 | val first = Flow[Int].map(i => s"Processed first: $i") 28 | val rest = Flow[Int].map(i => s"Processed rest: $i") 29 | 30 | val printSink = Sink.foreach[String](each => println(s"Reached sink: $each")) 31 | 32 | val done = source.prefixAndTail(1).flatMapConcat { case (head, tail) => 33 | // `head` is a Seq of prefix element(s), processed via `first` flow 34 | // `tail` is a Seq of tail elements, processed via `rest` flow 35 | // process head and tail elements with separate flows and concat results 36 | Source(head).via(first).concat(tail.via(rest)) 37 | }.runWith(printSink) 38 | 39 | terminateWhen(done) 40 | 41 | def terminateWhen(done: Future[?]): Unit = { 42 | done.onComplete { 43 | case Success(_) => 44 | println("Flow Success. About to terminate...") 45 | system.terminate() 46 | case Failure(e) => 47 | println(s"Flow Failure: $e. About to terminate...") 48 | system.terminate() 49 | } 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/ParametrizedFlow.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.Done 4 | import org.apache.pekko.actor.{ActorSystem, Cancellable} 5 | import org.apache.pekko.stream.scaladsl.{Flow, GraphDSL, Keep, Sink, Source, SourceQueueWithComplete, Zip} 6 | import org.apache.pekko.stream.{FlowShape, OverflowStrategy} 7 | 8 | import scala.collection.immutable 9 | import scala.concurrent.Future 10 | import scala.concurrent.duration.* 11 | import scala.util.{Failure, Success} 12 | 13 | 14 | /** 15 | * Source -> Flow(parameter) -> Sink 16 | * Inject parameter from outside the flow execution 17 | * 18 | * The basic idea is to Zip this parameter with the periodic (extrapolated) flow values 19 | * and then apply a user function 20 | * 21 | * Implementation is done with GraphDSL, Doc: 22 | * https://doc.akka.io/docs/akka/current/stream/stream-graphs.html 23 | * 24 | * Similar to [[Blacklist]] 25 | * 26 | * Inspired by: 27 | * https://discuss.lightbend.com/t/how-to-configure-flow-on-the-fly/6554/2 28 | * 29 | */ 30 | object ParametrizedFlow extends App { 31 | val service = ParameterizedFlowService 32 | 33 | Thread.sleep(5000) 34 | service.update(1.0) 35 | 36 | Thread.sleep(2000) 37 | service.update(1.5) 38 | Thread.sleep(2000) 39 | service.cancel() 40 | Thread.sleep(2000) 41 | 42 | println(service.result()) 43 | } 44 | 45 | object ParameterizedFlowService { 46 | implicit val system: ActorSystem = ActorSystem() 47 | 48 | import system.dispatcher 49 | 50 | def update(element: Double): Unit = flow._1._2.offer(element) 51 | 52 | def cancel(): Boolean = flow._1._1.cancel() 53 | 54 | def result(): Future[Seq[Double]] = flow._2 55 | 56 | val fun: (Int, Double) => Double = (flowValue: Int, paramValue: Double) => flowValue * paramValue 57 | val flow: ((Cancellable, SourceQueueWithComplete[Double]), Future[immutable.Seq[Double]]) = 58 | Source.tick(0.seconds, 500.millis, 10) 59 | .viaMat(createParamFlow(1, OverflowStrategy.dropBuffer, 0.5)(fun))(Keep.both) 60 | .wireTap(x => println(x)) 61 | .toMat(Sink.seq)(Keep.both) 62 | .run() 63 | 64 | val done: Future[Done] = flow._1._2.watchCompletion() 65 | terminateWhen(done) 66 | 67 | private def createParamFlow[A, P, O](bufferSize: Int, overflowStrategy: OverflowStrategy, initialParam: P)(fun: (A, P) => O) = 68 | Flow.fromGraph(GraphDSL.createGraph(Source.queue[P](bufferSize, overflowStrategy)) { implicit builder => 69 | queue => 70 | import GraphDSL.Implicits.* 71 | val zip = builder.add(Zip[A, P]()) 72 | // Interesting use of the extrapolate operator 73 | // based on https://doc.akka.io/docs/akka/current/stream/stream-rate.html#understanding-extrapolate-and-expand 74 | val extra = builder.add(Flow[P].extrapolate(Iterator.continually(_), Some(initialParam))) 75 | val map = builder.add(Flow[(A, P)].map(r => fun(r._1, r._2))) 76 | 77 | queue ~> extra ~> zip.in1 78 | zip.out ~> map 79 | FlowShape(zip.in0, map.out) 80 | }) 81 | 82 | private def terminateWhen(done: Future[?]): Unit = { 83 | done.onComplete { 84 | case Success(_) => 85 | println("Flow Success. About to terminate...") 86 | system.terminate() 87 | case Failure(e) => 88 | println(s"Flow Failure: $e. About to terminate...") 89 | system.terminate() 90 | } 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/SplitAfter.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.{Sink, Source} 5 | import org.slf4j.{Logger, LoggerFactory} 6 | 7 | import java.time.{Instant, LocalDateTime, ZoneOffset} 8 | import scala.collection.immutable.* 9 | import scala.concurrent.Future 10 | import scala.concurrent.duration.* 11 | import scala.util.{Failure, Success} 12 | 13 | /** 14 | * Split time series data into sub-streams per second 15 | * Thus we have a `session window` using processing time 16 | * Similar to: [[SplitWhen]] 17 | * Similar streams operator: groupedWithin 18 | * 19 | * Inspired by: 20 | * https://doc.akka.io/docs/akka/current/stream/operators/Source-or-Flow/splitAfter.html 21 | * 22 | * Note that this implementation can be materialized many times because the 23 | * stateful decision is done in statefulMapConcat, see discussion: 24 | * https://discuss.lightbend.com/t/state-inside-of-flow-operators/5717 25 | */ 26 | object SplitAfter extends App { 27 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 28 | implicit val system: ActorSystem = ActorSystem() 29 | 30 | import system.dispatcher 31 | 32 | private def hasSecondChanged: () => Seq[(Int, Instant)] => Iterable[(Instant, Boolean)] = { 33 | () => { 34 | slidingElements => { 35 | if (slidingElements.size == 2) { 36 | val current = slidingElements.head 37 | val next = slidingElements.tail.head 38 | val currentBucket = LocalDateTime.ofInstant(current._2, ZoneOffset.UTC).withNano(0) 39 | val nextBucket = LocalDateTime.ofInstant(next._2, ZoneOffset.UTC).withNano(0) 40 | List((current._2, currentBucket != nextBucket)) 41 | } else { 42 | val current = slidingElements.head 43 | List((current._2, false)) 44 | } 45 | } 46 | } 47 | } 48 | 49 | val done = Source(1 to 100) 50 | .throttle(1, 100.millis) 51 | .map(elem => (elem, Instant.now())) 52 | .sliding(2) // allows to compare this element with the next element 53 | .statefulMapConcat(hasSecondChanged) // stateful decision 54 | .splitAfter(_._2) // split when second has changed 55 | .map(_._1) // proceed with payload 56 | .fold(0)((acc, _) => acc + 1) // sum 57 | .mergeSubstreams 58 | .runWith(Sink.foreach(each => println(s"Elements in group: $each"))) 59 | 60 | terminateWhen(done) 61 | 62 | 63 | def terminateWhen(done: Future[?]): Unit = { 64 | done.onComplete { 65 | case Success(_) => 66 | println("Flow Success. About to terminate...") 67 | system.terminate() 68 | case Failure(e) => 69 | println(s"Flow Failure: $e. About to terminate...") 70 | system.terminate() 71 | } 72 | } 73 | } 74 | 75 | -------------------------------------------------------------------------------- /src/main/scala/sample/stream_shared_state/SplitAfterPrefix.scala: -------------------------------------------------------------------------------- 1 | package sample.stream_shared_state 2 | 3 | import org.apache.pekko.actor.ActorSystem 4 | import org.apache.pekko.stream.scaladsl.{Flow, Keep, Sink, Source} 5 | import org.slf4j.{Logger, LoggerFactory} 6 | 7 | import scala.concurrent.Future 8 | 9 | /** 10 | * Chunk a stream of strings and 11 | * for each line process tail elements based on the first element 12 | * 13 | * Inspired by: 14 | * https://discuss.lightbend.com/t/combine-prefixandtail-1-with-sink-lazysink-for-subflow-created-by-splitafter/8623 15 | * 16 | * Similar to: [[HandleFirstElementSpecially]] 17 | * Similar to: [[DeferredStreamCreation]] 18 | * Similar to: [[SplitAfter]] 19 | */ 20 | object SplitAfterPrefix extends App { 21 | val logger: Logger = LoggerFactory.getLogger(this.getClass) 22 | implicit val system: ActorSystem = ActorSystem() 23 | import system.dispatcher 24 | 25 | val chunkTerminator = "STOP" 26 | val input = Seq( 27 | "A", "say", "a", "word", chunkTerminator, 28 | "B", "be", "ready", chunkTerminator 29 | ) 30 | 31 | def handleLine(prefix: Seq[String], stream: Source[String, Any]): Source[Any, Any] = { 32 | prefix.head match { 33 | case "A" => 34 | stream 35 | .map(_.toUpperCase()) 36 | .runFold("")(_ + " " + _) 37 | .onComplete(res => logger.info(s"Result for $prefix: $res")) 38 | case "B" => 39 | stream 40 | .map(_.toLowerCase) 41 | .runFold("")(_ + " " + _) 42 | .onComplete(res => logger.info(s"Result for $prefix: $res")) 43 | } 44 | Source.empty 45 | } 46 | 47 | val handleChunk: Sink[String, Future[Any]] = 48 | Flow[String] 49 | .prefixAndTail(1) 50 | .flatMapConcat((handleLine _).tupled) // getting only a single element (of type Tuple) 51 | .toMat(Sink.ignore)(Keep.right) 52 | 53 | val done = Source(input) 54 | .splitAfter(_ == chunkTerminator) 55 | .to(handleChunk) 56 | .run() 57 | 58 | Thread.sleep(1000) 59 | system.terminate() 60 | } -------------------------------------------------------------------------------- /src/main/scala/tools/CompletionsUtil.java: -------------------------------------------------------------------------------- 1 | package tools; 2 | 3 | import org.slf4j.Logger; 4 | import org.slf4j.LoggerFactory; 5 | 6 | /** 7 | * Utility class for common functionality used by different LLM completion implementations 8 | */ 9 | public class CompletionsUtil { 10 | private static final Logger LOGGER = LoggerFactory.getLogger(CompletionsUtil.class); 11 | 12 | /** 13 | * Default system message for translation tasks 14 | */ 15 | public static final String DEFAULT_TRANSLATOR_SYSTEM_MESSAGE = "You are a translator"; 16 | 17 | /** 18 | * Creates a translation prompt 19 | * 20 | * @param text Text to translate 21 | * @param sourceLanguage Source language 22 | * @param targetLanguage Target language 23 | * @return Formatted translation prompt 24 | */ 25 | public static String createTranslationPrompt(String text, String sourceLanguage, String targetLanguage) { 26 | return String.format("Translate the following subtitle text from %s to %s: %s", 27 | sourceLanguage, targetLanguage, text); 28 | } 29 | 30 | /** 31 | * Creates a movie context prompt 32 | * 33 | * @param movieTitle The title of the movie 34 | * @param movieReleaseYear The release year of the movie 35 | * @return Formatted movie context prompt 36 | */ 37 | public static String createMovieContextPrompt(String movieTitle, int movieReleaseYear) { 38 | return String.format( 39 | "Find movie metadata about the movie '%s' released in %d\n" + 40 | "If you don't know this movie or if the title or release year is ambiguous: Respond with \"N/A\", do not apologize." + 41 | "Structure of the response:\n" + 42 | "Plot summary: [Max 100 words]" + 43 | "Locations: [Max 3 key locations]" + 44 | "Key characters: [Max 5 main character names, no actor names]" + 45 | "Themes: [Max 3 main themes]" + 46 | "Notable scenes: [Max 3 brief descriptions of memorable scenes]", 47 | movieTitle, movieReleaseYear); 48 | } 49 | 50 | /** 51 | * Logs completion results 52 | * 53 | * @param result The completion result 54 | * @param tokenCount The token count 55 | * @param context Additional context information (optional) 56 | */ 57 | public static void logCompletionResult(String result, int tokenCount, String context) { 58 | if (context != null && !context.isEmpty()) { 59 | LOGGER.info("{}: {}", context, result); 60 | } else { 61 | LOGGER.info("Result: {}", result); 62 | } 63 | LOGGER.info("Total tokens: {}", tokenCount); 64 | } 65 | } -------------------------------------------------------------------------------- /src/scalafix/resources/META-INF/services/scalafix.v1.Rule: -------------------------------------------------------------------------------- 1 | fix.PekkoScalafix -------------------------------------------------------------------------------- /src/scalafix/scala/fix/PekkoScalafix.scala: -------------------------------------------------------------------------------- 1 | package fix 2 | 3 | import scalafix.v1._ 4 | 5 | import scala.meta._ 6 | 7 | class PekkoScalafix extends SemanticRule("PekkoScalafix") { 8 | 9 | override def fix(implicit doc: SemanticDocument): Patch = { 10 | // println("Tree.syntax: " + doc.tree.syntax) 11 | // println("Tree.structure: " + doc.tree.structure) 12 | // println("Tree.structureLabeled: " + doc.tree.structureLabeled) 13 | doc.tree.collect { 14 | case i@Importer(ref, _) if ref.toString.startsWith("akka.stream.alpakka") => 15 | Patch.replaceTree(i, i.toString() 16 | .replaceFirst("akka.stream.alpakka", "org.apache.pekko.stream.connectors") 17 | .replaceFirst("Akka", "Pekko") 18 | ) 19 | case i@Importer(ref, _) if ref.toString.startsWith("akka") => 20 | Patch.replaceTree(i, i.toString() 21 | .replaceFirst("akka", "org.apache.pekko") 22 | .replaceFirst("Akka", "Pekko") 23 | ) 24 | case n: Type.Name if n.value.startsWith("Akka") => 25 | Patch.replaceTree(n, n.toString().replaceFirst("Akka", "Pekko")) 26 | case n: Term.Name if n.value.contains("Akka") => 27 | Patch.replaceTree(n, n.toString().replaceAll("Akka", "Pekko")) 28 | }.asPatch 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /src/test/scala/ReverseProxySimulation.scala: -------------------------------------------------------------------------------- 1 | 2 | import io.gatling.core.Predef.* 3 | import io.gatling.http.Predef.* 4 | 5 | import scala.concurrent.duration.* 6 | 7 | /** 8 | * Start [[akkahttp.ReverseProxy]] 9 | * Run this simulation from cmd shell: 10 | * sbt 'Gatling/testOnly ReverseProxySimulation' 11 | * or from sbt shell: 12 | * Gatling/testOnly ReverseProxySimulation 13 | */ 14 | class ReverseProxySimulation extends Simulation { 15 | val baseUrl = "http://127.0.0.1:8080" 16 | 17 | val httpProtocol = http 18 | .baseUrl(baseUrl) 19 | .acceptHeader("application/json") 20 | .userAgentHeader("Gatling") 21 | 22 | val scn = scenario("GatlingLocalClient") 23 | .exec(session => session.set("correlationId", 1)) 24 | .repeat(10) { 25 | exec( 26 | http("Local Mode Request") 27 | .get("/") 28 | .header("Host", "local") 29 | .header("X-Correlation-ID", session => s"load-${session.userId}-${session("correlationId").as[Int]}") 30 | .check(status.is(200)) 31 | .check(status.saveAs("responseStatus")) 32 | .check(header("X-Correlation-ID").saveAs("responseCorrelationId")) 33 | ) 34 | .exec(session => { 35 | println(s"Got: ${session.status} response with HTTP status: ${session("responseStatus").as[String]} for id: ${session("responseCorrelationId").as[String]}") 36 | session 37 | }) 38 | .exec(session => session.set("correlationId", session("correlationId").as[Int] + 1)) 39 | } 40 | 41 | // Adjust to scale load 42 | val loadFactorMorning = 0.01 43 | val loadFactorMidday = 0.02 44 | val loadFactorEvening = 0.03 45 | 46 | val morningPeak = scenario("Morning Peak") 47 | .exec(scn) 48 | .inject( 49 | nothingFor(5.seconds), // initial quiet period 50 | rampUsers((20 * loadFactorMorning).toInt).during(10.seconds), // ramp up 51 | constantUsersPerSec(50 * loadFactorMorning).during(20.seconds), // peak load 52 | rampUsersPerSec(50 * loadFactorMorning).to(10 * loadFactorMorning).during(10.seconds), // ramp down 53 | constantUsersPerSec(10 * loadFactorMorning).during(10.seconds), // tail off 54 | nothingFor(30.seconds) // cool down period 55 | ) 56 | 57 | val middayPeak = scenario("Midday Peak") 58 | .exec(scn) 59 | .inject( 60 | nothingFor(5.seconds), 61 | rampUsers((20 * loadFactorMidday).toInt).during(10.seconds), 62 | constantUsersPerSec(50 * loadFactorMidday).during(20.seconds), 63 | rampUsersPerSec(50 * loadFactorMidday).to(10 * loadFactorMidday).during(10.seconds), 64 | constantUsersPerSec(10 * loadFactorMidday).during(10.seconds), 65 | nothingFor(30.seconds) 66 | ) 67 | 68 | val eveningPeak = scenario("Evening Peak") 69 | .exec(scn) 70 | .inject( 71 | nothingFor(5.seconds), 72 | rampUsers((20 * loadFactorEvening).toInt).during(10.seconds), 73 | constantUsersPerSec(50 * loadFactorEvening).during(20.seconds), 74 | rampUsersPerSec(50 * loadFactorEvening).to(10 * loadFactorEvening).during(10.seconds), 75 | constantUsersPerSec(10 * loadFactorEvening).during(10.seconds), 76 | nothingFor(30.seconds) 77 | ) 78 | 79 | setUp( 80 | morningPeak.andThen(middayPeak).andThen(eveningPeak) 81 | ).protocols(httpProtocol) 82 | } -------------------------------------------------------------------------------- /src/test/scala/alpakka/dynamodb/DynamoDBEchoIT.java: -------------------------------------------------------------------------------- 1 | package alpakka.dynamodb; 2 | 3 | import org.junit.jupiter.api.BeforeAll; 4 | import org.junit.jupiter.api.Test; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | import org.testcontainers.containers.localstack.LocalStackContainer; 8 | import org.testcontainers.containers.wait.strategy.Wait; 9 | import org.testcontainers.junit.jupiter.Container; 10 | import org.testcontainers.junit.jupiter.Testcontainers; 11 | import org.testcontainers.utility.DockerImageName; 12 | import org.testcontainers.utility.MountableFile; 13 | import scala.jdk.javaapi.FutureConverters; 14 | 15 | import java.util.concurrent.CompletionStage; 16 | import java.util.concurrent.ExecutionException; 17 | 18 | import static org.assertj.core.api.Assertions.assertThat; 19 | import static org.testcontainers.containers.localstack.LocalStackContainer.Service.DYNAMODB; 20 | 21 | /** 22 | * Setup/run {@link alpakka.dynamodb.DynamoDBEcho} on localStack container 23 | *

24 | * Running this example against AWS: 25 | * Looks as if there is a way to delete a DB instance via the SDK: 26 | * https://docs.aws.amazon.com/code-library/latest/ug/rds_example_rds_DeleteDBInstance_section.html 27 | * However, getting the `dbInstanceIdentifier` via SDK is not straightforward 28 | * Therefore, we only run against localStack for now in order to avoid dangling resources 29 | *

30 | * Doc: 31 | * https://pekko.apache.org/docs/pekko-connectors/current/dynamodb.html#aws-dynamodb 32 | * https://community.aws/content/2dxWQAZsdc3dk5uCILAmNqEME2e/testing-dynamodb-interactions-in-spring-boot-using-localstack-and-testcontainers?lang=en 33 | */ 34 | @Testcontainers 35 | public class DynamoDBEchoIT { 36 | private static final Logger LOGGER = LoggerFactory.getLogger(DynamoDBEchoIT.class); 37 | 38 | @Container 39 | public static LocalStackContainer localStack = new LocalStackContainer(DockerImageName.parse("localstack/localstack:3.3")) 40 | .withCopyFileToContainer(MountableFile.forClasspathResource("/localstack/init_dynamodb.sh", 700), "/etc/localstack/init/ready.d/init_dynamodb.sh") 41 | .withServices(DYNAMODB) 42 | .waitingFor(Wait.forLogMessage(".*Executed init_dynamodb.sh.*", 1)); 43 | 44 | 45 | @BeforeAll 46 | public static void beforeAll() { 47 | localStack.start(); 48 | LOGGER.info("LocalStack container started on host address: {}", localStack.getEndpoint()); 49 | } 50 | 51 | @Test 52 | public void testLocal() throws ExecutionException, InterruptedException { 53 | DynamoDBEcho dynamoDBEcho = new DynamoDBEcho(localStack.getEndpointOverride(DYNAMODB), localStack.getAccessKey(), localStack.getSecretKey(), localStack.getRegion()); 54 | int noOfItemsEven = 10; 55 | 56 | CompletionStage result = FutureConverters.asJava(dynamoDBEcho.run(noOfItemsEven)); 57 | assertThat(result.toCompletableFuture().get()).isEqualTo(noOfItemsEven / 2); 58 | } 59 | } 60 | 61 | -------------------------------------------------------------------------------- /src/test/scala/alpakka/s3/MinioContainer.java: -------------------------------------------------------------------------------- 1 | package alpakka.s3; 2 | 3 | import org.testcontainers.containers.GenericContainer; 4 | import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; 5 | 6 | import java.time.Duration; 7 | 8 | // See also: 9 | // https://github.com/akka/alpakka/blob/main/s3/src/test/scala/akka/stream/alpakka/s3/MinioContainer.scala 10 | public class MinioContainer extends GenericContainer { 11 | 12 | private static final int DEFAULT_PORT = 9000; 13 | private static final String DEFAULT_IMAGE = "quay.io/minio/minio"; 14 | private static final String DEFAULT_TAG = "latest"; 15 | 16 | private static final String MINIO_ACCESS_KEY = "MINIO_ACCESS_KEY"; 17 | private static final String MINIO_SECRET_KEY = "MINIO_SECRET_KEY"; 18 | 19 | private static final String DEFAULT_STORAGE_DIRECTORY = "/data"; 20 | private static final String HEALTH_ENDPOINT = "/minio/health/ready"; 21 | 22 | public MinioContainer(CredentialsProvider credentials) { 23 | this(DEFAULT_IMAGE + ":" + DEFAULT_TAG, credentials); 24 | } 25 | 26 | public MinioContainer(String image, CredentialsProvider credentials) { 27 | super(image == null ? DEFAULT_IMAGE + ":" + DEFAULT_TAG : image); 28 | addExposedPort(DEFAULT_PORT); 29 | if (credentials != null) { 30 | withEnv(MINIO_ACCESS_KEY, credentials.getAccessKey()); 31 | withEnv(MINIO_SECRET_KEY, credentials.getSecretKey()); 32 | } 33 | withCommand("server", DEFAULT_STORAGE_DIRECTORY); 34 | setWaitStrategy(new HttpWaitStrategy() 35 | .forPort(DEFAULT_PORT) 36 | .forPath(HEALTH_ENDPOINT) 37 | .withStartupTimeout(Duration.ofMinutes(2))); 38 | } 39 | 40 | public String getHostAddress() { 41 | return getHost() + ":" + getMappedPort(DEFAULT_PORT); 42 | } 43 | 44 | 45 | public static class CredentialsProvider { 46 | private final String accessKey; 47 | private final String secretKey; 48 | 49 | public CredentialsProvider(String accessKey, String secretKey) { 50 | this.accessKey = accessKey; 51 | this.secretKey = secretKey; 52 | } 53 | 54 | public String getAccessKey() { 55 | return accessKey; 56 | } 57 | 58 | public String getSecretKey() { 59 | return secretKey; 60 | } 61 | } 62 | } -------------------------------------------------------------------------------- /src/test/scala/alpakka/s3/S3EchoMinioIT.java: -------------------------------------------------------------------------------- 1 | package alpakka.s3; 2 | 3 | import org.junit.jupiter.api.BeforeAll; 4 | import org.junit.jupiter.api.Test; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | import org.testcontainers.junit.jupiter.Testcontainers; 8 | import scala.jdk.javaapi.FutureConverters; 9 | 10 | import java.io.IOException; 11 | import java.nio.file.Files; 12 | import java.util.concurrent.CompletionStage; 13 | import java.util.concurrent.ExecutionException; 14 | 15 | import static org.assertj.core.api.Assertions.assertThat; 16 | 17 | @Testcontainers 18 | public class S3EchoMinioIT { 19 | private static final Logger LOGGER = LoggerFactory.getLogger(S3EchoMinioIT.class); 20 | 21 | // Credentials for client access 22 | private static final String ACCESS_KEY = "minio"; 23 | private static final String SECRET_KEY = "minio123"; 24 | 25 | @org.testcontainers.junit.jupiter.Container 26 | private static final MinioContainer minioContainer = new MinioContainer( 27 | new MinioContainer.CredentialsProvider(ACCESS_KEY, SECRET_KEY)); 28 | 29 | @BeforeAll 30 | public static void beforeAll() { 31 | LOGGER.info("Minio container started on host address: {}", minioContainer.getHostAddress()); 32 | } 33 | 34 | @Test 35 | public void testLocal() throws IOException, ExecutionException, InterruptedException { 36 | S3Echo echo = new S3Echo(minioContainer.getHostAddress(), ACCESS_KEY, SECRET_KEY); 37 | 38 | CompletionStage result = FutureConverters.asJava(echo.run()); 39 | 40 | // Number of files in bucket: n uploaded files + 1 zip 41 | assertThat(result.toCompletableFuture().get()).isEqualTo(11); 42 | 43 | assertThat(Files.list(echo.localTmpDir()).count()).isEqualTo(10); 44 | } 45 | } -------------------------------------------------------------------------------- /src/test/scala/alpakka/slick/PostgresClusterEmbedded.java: -------------------------------------------------------------------------------- 1 | package alpakka.slick; 2 | 3 | import io.zonky.test.db.postgres.embedded.EmbeddedPostgres; 4 | import org.junit.jupiter.api.extension.AfterAllCallback; 5 | import org.junit.jupiter.api.extension.BeforeAllCallback; 6 | import org.junit.jupiter.api.extension.ExtensionContext; 7 | 8 | import java.io.IOException; 9 | import java.sql.Connection; 10 | import java.sql.SQLException; 11 | import java.sql.Statement; 12 | 13 | /** 14 | * Wrapper around embedded-postgres. 15 | *

16 | * Integrated into unit tests with @RegisterExtension 17 | */ 18 | public class PostgresClusterEmbedded implements BeforeAllCallback, AfterAllCallback { 19 | private EmbeddedPostgres postgres; 20 | 21 | @Override 22 | public void beforeAll(ExtensionContext context) throws IOException { 23 | postgres = 24 | EmbeddedPostgres.builder() 25 | // Explicitly clean-up the data directory to always start with a clean setup 26 | .setCleanDataDirectory(true) 27 | .setPort(5432) 28 | // Set the write-ahead log level to "logical" to make sure that enough 29 | // information is written to the write-ahead log 30 | // because this is required for CDC with Debezium 31 | .setServerConfig("wal_level", "logical") 32 | .start(); 33 | } 34 | 35 | @Override 36 | public void afterAll(ExtensionContext context) throws IOException { 37 | postgres.close(); 38 | } 39 | 40 | public void executeStatement(String statement) throws SQLException { 41 | Connection conn = postgres.getPostgresDatabase().getConnection(); 42 | try (Statement stmt = conn.createStatement()) { 43 | stmt.executeUpdate(statement); 44 | } catch (SQLException e) { 45 | throw new RuntimeException(e); 46 | } 47 | } 48 | 49 | public String getJdbcUrl() { 50 | return postgres.getJdbcUrl("test", "test"); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/test/scala/alpakka/slick/SlickEmbeddedIT.java: -------------------------------------------------------------------------------- 1 | package alpakka.slick; 2 | 3 | import org.junit.jupiter.api.AfterEach; 4 | import org.junit.jupiter.api.BeforeEach; 5 | import org.junit.jupiter.api.Test; 6 | import org.junit.jupiter.api.extension.RegisterExtension; 7 | import org.slf4j.Logger; 8 | import org.slf4j.LoggerFactory; 9 | import scala.jdk.javaapi.FutureConverters; 10 | 11 | import java.sql.SQLException; 12 | import java.util.concurrent.TimeUnit; 13 | 14 | import static org.assertj.core.api.Assertions.assertThat; 15 | 16 | /** 17 | * Same as SlickIT, but with local embedded Postgres DB 18 | */ 19 | class SlickEmbeddedIT { 20 | private static final Logger LOGGER = LoggerFactory.getLogger(SlickEmbeddedIT.class); 21 | private static SlickRunner SLICK_RUNNER; 22 | 23 | @RegisterExtension 24 | public static final PostgresClusterEmbedded POSTGRES = new PostgresClusterEmbedded(); 25 | 26 | private static final String DB_NAME = "test"; 27 | private static final String USER_NAME = "test"; 28 | private static final String PASSWORD = "test"; 29 | 30 | @BeforeEach 31 | void setUp() throws SQLException { 32 | POSTGRES.executeStatement("CREATE DATABASE " + DB_NAME); 33 | POSTGRES.executeStatement("CREATE USER " + USER_NAME + " with encrypted PASSWORD '" + PASSWORD + "'"); 34 | POSTGRES.executeStatement("grant all privileges on database " + DB_NAME + " to " + USER_NAME); 35 | 36 | SLICK_RUNNER = SlickRunner.apply(POSTGRES.getJdbcUrl()); 37 | SLICK_RUNNER.createTableOnSession(); 38 | LOGGER.info("DB: {} created at URL: {}, USER_NAME: {}, PASSWORD: {}", DB_NAME, POSTGRES.getJdbcUrl(), USER_NAME, PASSWORD); 39 | } 40 | 41 | @AfterEach 42 | void teardown() throws SQLException { 43 | SLICK_RUNNER.dropTableOnSession(); 44 | SLICK_RUNNER.terminate(); 45 | POSTGRES.executeStatement("DROP DATABASE " + DB_NAME); 46 | POSTGRES.executeStatement("DROP USER " + USER_NAME); 47 | } 48 | 49 | @Test 50 | void populateAndReadUsers() { 51 | int noOfUsers = 100; 52 | SLICK_RUNNER.populateSync(noOfUsers); 53 | assertThat(SLICK_RUNNER.readUsersSync().size()).isEqualTo(noOfUsers); 54 | } 55 | 56 | @Test 57 | void populateAndReadUsersPaged() { 58 | int noOfUsers = 20000; 59 | SLICK_RUNNER.populateSync(noOfUsers); 60 | 61 | // Done via counter to avoid Scala->Java result collection conversion "type trouble" 62 | assertThat(FutureConverters.asJava(SLICK_RUNNER.processUsersPaged())).succeedsWithin(5, TimeUnit.SECONDS); 63 | assertThat(SLICK_RUNNER.counter().get()).isEqualTo(noOfUsers); 64 | } 65 | 66 | @Test 67 | void populateAndCountUsers() { 68 | int noOfUsers = 100; 69 | SLICK_RUNNER.populateSync(noOfUsers); 70 | assertThat(SLICK_RUNNER.getTotal()).isEqualTo(noOfUsers); 71 | } 72 | } -------------------------------------------------------------------------------- /src/test/scala/alpakka/sqs/SqsEchoIT.java: -------------------------------------------------------------------------------- 1 | package alpakka.sqs; 2 | 3 | import org.junit.jupiter.api.BeforeAll; 4 | import org.junit.jupiter.api.Test; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | import org.testcontainers.containers.localstack.LocalStackContainer; 8 | import org.testcontainers.junit.jupiter.Container; 9 | import org.testcontainers.junit.jupiter.Testcontainers; 10 | import org.testcontainers.utility.DockerImageName; 11 | import org.testcontainers.utility.MountableFile; 12 | 13 | import java.io.IOException; 14 | 15 | import static org.assertj.core.api.Assertions.assertThat; 16 | import static org.testcontainers.containers.localstack.LocalStackContainer.Service.SQS; 17 | 18 | /** 19 | * Setup/run {@link alpakka.sqs.SqsEcho} on localStack container 20 | */ 21 | @Testcontainers 22 | public class SqsEchoIT { 23 | private static final Logger LOGGER = LoggerFactory.getLogger(SqsEchoIT.class); 24 | 25 | @Container 26 | public static LocalStackContainer localStack = new LocalStackContainer(DockerImageName.parse("localstack/localstack:3.3")) 27 | .withServices(SQS) 28 | // https://docs.localstack.cloud/user-guide/aws/sqs/#queue-urls 29 | .withEnv("SQS_ENDPOINT_STRATEGY", "domain") 30 | // Redundant to createQueue(), left to show that this is a way to init a queue 31 | .withCopyFileToContainer(MountableFile.forClasspathResource("/localstack/init_sqs.sh", 700), "/etc/localstack/init/ready.d/init_sqs.sh"); 32 | 33 | @BeforeAll 34 | public static void beforeAll() throws InterruptedException, IOException { 35 | LOGGER.info("LocalStack container started on host address: {}", localStack.getEndpoint()); 36 | LOGGER.info("Waiting 10 seconds for queue setup to complete..."); 37 | Thread.sleep(10000); 38 | 39 | org.testcontainers.containers.Container.ExecResult result = localStack.execInContainer("awslocal", "sqs", "list-queues"); 40 | LOGGER.debug("Result exit code: {}", result.getExitCode()); 41 | LOGGER.info("Check queues on container: {}", result.getStdout()); 42 | } 43 | 44 | @Test 45 | public void testLocal() { 46 | SqsEcho sqsEcho = new SqsEcho(localStack.getEndpointOverride(SQS), localStack.getAccessKey(), localStack.getSecretKey(), localStack.getRegion()); 47 | Integer result = sqsEcho.run(); 48 | assertThat(result).isEqualTo(10); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /src/test/scala/tools/SrtParserSpec.scala: -------------------------------------------------------------------------------- 1 | package tools 2 | 3 | import org.scalatest.funsuite.AnyFunSuite 4 | import org.scalatest.matchers.should.Matchers 5 | 6 | import java.io.PrintWriter 7 | import java.nio.file.Files 8 | 9 | class SrtParserSpec extends AnyFunSuite with Matchers { 10 | 11 | test("parse valid srt file") { 12 | val parser = new SrtParser("src/main/resources/EN_challenges.srt") 13 | val blocks = parser.runSync() 14 | 15 | blocks.nonEmpty shouldBe true 16 | blocks.head.lines.nonEmpty shouldBe true 17 | } 18 | 19 | test("convert subtitle block to formatted output") { 20 | val block = SubtitleBlock( 21 | start = 1000, 22 | end = 4000, 23 | lines = Seq("First line", "Second line") 24 | ) 25 | val formatted = block.formatOutBlock(1) 26 | 27 | formatted should include("00:00:01,000 --> 00:00:04,000") 28 | formatted should include("First line") 29 | formatted should include("Second line") 30 | } 31 | 32 | test("parse subtitle block with multiple lines") { 33 | val block = SubtitleBlock( 34 | start = 0, 35 | end = 2000, 36 | lines = Seq("Line 1", "Line 2", "Line 3") 37 | ) 38 | block.allLines shouldBe "Line 1 Line 2 Line 3" 39 | } 40 | 41 | test("format time correctly") { 42 | val block = SubtitleBlock( 43 | start = 3661000, // 1 hour, 1 minute, 1 second 44 | end = 3662000, // 1 hour, 1 minute, 2 seconds 45 | lines = Seq("Test") 46 | ) 47 | val formatted = block.formatOutBlock(1) 48 | 49 | formatted should include("01:01:01,000 --> 01:01:02,000") 50 | } 51 | 52 | 53 | test("handle empty subtitle file") { 54 | val tempFile = Files.createTempFile("empty", ".srt") 55 | tempFile.toFile.deleteOnExit() 56 | val parser = new SrtParser(tempFile.toString) 57 | val blocks = parser.runSync() 58 | 59 | blocks shouldBe empty 60 | } 61 | 62 | test("handle invalid srt file format") { 63 | val tempFile = Files.createTempFile("invalid", ".srt") 64 | tempFile.toFile.deleteOnExit() 65 | val writer = new PrintWriter(tempFile.toFile) 66 | writer.write( 67 | """ 68 | |Invalid Format 69 | |No timestamps here 70 | |Just random text 71 | |Without proper structure 72 | |---> wrong separator 73 | """.stripMargin) 74 | writer.close() 75 | 76 | val parser = new SrtParser(tempFile.toString) 77 | 78 | assertThrows[java.time.format.DateTimeParseException] { 79 | parser.runSync() 80 | } 81 | } 82 | } -------------------------------------------------------------------------------- /src/test/scala/util/LogFileScanner.scala: -------------------------------------------------------------------------------- 1 | package util 2 | 3 | 4 | import org.apache.pekko.NotUsed 5 | import org.apache.pekko.actor.ActorSystem 6 | import org.apache.pekko.stream.KillSwitches 7 | import org.apache.pekko.stream.connectors.file.scaladsl.FileTailSource 8 | import org.apache.pekko.stream.scaladsl.{Keep, Sink, Source} 9 | 10 | import java.nio.file.{FileSystem, FileSystems, Path, Paths} 11 | import java.util.Scanner 12 | import scala.concurrent.Await 13 | import scala.concurrent.duration.* 14 | 15 | class LogFileScanner(localLogFilePath: String = "logs/application.log") { 16 | implicit val system: ActorSystem = ActorSystem() 17 | 18 | private val fs: FileSystem = FileSystems.getDefault 19 | 20 | // The patterns are case-sensitive 21 | def run(scanDelaySeconds: Int = 0, scanForSeconds: Int = 5, searchAfterPattern: String, pattern: String): List[String] = { 22 | val path: Path = fs.getPath(localLogFilePath) 23 | val pollingInterval = 250.millis 24 | val maxLineSize: Int = 100 * 1024 25 | 26 | // Wait for the components to produce log messages 27 | Thread.sleep(scanDelaySeconds * 1000) 28 | println(String.format("About to start LogFileScanner - Searching file: %s for pattern: '%s', consider only lines after: '%s'", path, pattern, searchAfterPattern)) 29 | 30 | val lineSeparator = detectLineSeparator(Paths.get(localLogFilePath)) 31 | val lines: Source[String, NotUsed] = FileTailSource.lines(path, maxLineSize, pollingInterval, lineSeparator) 32 | 33 | val (killSwitch, resultFut) = 34 | lines 35 | .dropWhile(line => !line.contains(searchAfterPattern)) 36 | //.wireTap(line => println("Process line: " + line)) 37 | .filter(line => line.contains(pattern)) 38 | //.wireTap(line => println("Found pattern in line: " + line)) 39 | .viaMat(KillSwitches.single)(Keep.right) 40 | .toMat(Sink.seq)(Keep.both) 41 | .run() 42 | 43 | // Time to scan in 'tail -f' mode 44 | Thread.sleep(scanForSeconds * 1000) 45 | killSwitch.shutdown() 46 | 47 | val resultList = Await.result(resultFut, 1.seconds) 48 | println(s"Occurrences found: ${resultList.length}") 49 | resultList.toList 50 | } 51 | 52 | /** 53 | * To process files that were created on different operating systems. 54 | * 55 | * @param path to log file 56 | * @return the line separator for the file 57 | */ 58 | def detectLineSeparator(path: Path): String = { 59 | 60 | // Assumption: Small buffer (1024 chars) is sufficient to read 1st line 61 | val scanner = new Scanner(path) 62 | try { 63 | scanner.useDelimiter("\\z") 64 | val firstLine = scanner.next() 65 | 66 | if (firstLine.matches("(?s).*(\\r\\n).*")) { 67 | println(s"Detected line.separator for: Windows") 68 | "\r\n" 69 | } 70 | else if (firstLine.matches("(?s).*(\\n).*")) { 71 | println(s"Detected line.separator for: Unix/Linux") 72 | "\n" 73 | } 74 | else if (firstLine.matches("(?s).*(\\r).*")) { 75 | println(s"Detected line.separator for: Legacy mac os 9") 76 | "\r" 77 | } 78 | else { 79 | println(s"Unable to detected line.separator, fallback to separator for: ${System.getProperty("os.name")}") 80 | System.lineSeparator() 81 | } 82 | } finally { 83 | scanner.close() 84 | } 85 | } 86 | } 87 | --------------------------------------------------------------------------------