├── .gitignore ├── README.md └── ScalaKafkaStreamsDemo ├── RunPipeline.ps1 ├── build.sbt └── src ├── main ├── resources │ └── streamsApplication.conf └── scala │ ├── StraightThroughTopology.scala │ ├── WordCountTopology.scala │ ├── entities │ ├── AkkaHttpEntitiesJsonFormats.scala │ ├── Contributor.scala │ ├── HostStoreInfo.scala │ └── Rating.scala │ ├── interactive │ └── queries │ │ ├── MetadataService.scala │ │ └── ratings │ │ ├── RatingRestService.scala │ │ ├── RatingStreamProcessingTopology.scala │ │ ├── RatingStreamProcessingTopologyApp.scala │ │ └── RatingsProducerApp.scala │ ├── joining │ ├── InnerJoinTopology.scala │ ├── LeftJoinTopology.scala │ └── OuterJoinTopology.scala │ ├── processorapi │ └── interop │ │ ├── ContributorPrintingSupplier.scala │ │ ├── ContributorTranformSupplier.scala │ │ ├── ProcessorApiProcessSupplierTopology.scala │ │ └── ProcessorApiTransformValuesTopology.scala │ ├── serialization │ ├── CustomSerdesTopology.scala │ └── JsonSerdes.scala │ ├── stateful │ └── transformations │ │ └── aggregating │ │ ├── AggregateTopology.scala │ │ ├── CountTopology.scala │ │ └── ReduceTopology.scala │ ├── stateless │ └── transformations │ │ ├── BranchTopology.scala │ │ ├── FilterTopology.scala │ │ ├── FlatMapTopology.scala │ │ ├── ForEachTopology.scala │ │ ├── GroupByTopology.scala │ │ ├── MapTopology.scala │ │ ├── PeekTopology.scala │ │ ├── SelectKeyTopology.scala │ │ └── ThroughCustomPartitionerTopology.scala │ ├── utils │ ├── Retry.scala │ ├── Settings.scala │ └── StateStores.scala │ └── windowing │ ├── HoppingTimeWordCountTopology.scala │ └── TumblingWordCountTopology.scala └── test └── scala ├── StaightThoughTests.scala ├── interactivequeries └── ratings │ └── RatingsStreamProcessingTopologyTests.scala ├── joining ├── InnerJoinTopologyTests.scala ├── LeftJoinTopologyTests.scala └── OuterJoinTopologyTests.scala ├── processorapi └── interop │ ├── ProcessorApiProcessSupplierTopologyTests.scala │ └── ProcessorApiTransformValuesTopologyTests.scala ├── serialization └── CustomSerdesTopologyTests.scala ├── stateful └── transformations │ └── aggregating │ ├── AggregateTopologyTests.scala │ ├── CountTopologyTests.scala │ └── ReduceTopologyTests.scala └── stateless └── transformations ├── BranchTopologyTests.scala ├── FilterTopologyTests.scala ├── FlatMapTopologyTests.scala ├── ForEachTopologyTests.scala ├── GroupByTopologyTests.scala ├── MapTopologyTests.scala ├── PeekTopologyTests.scala ├── SelectKeyTopologyTests.scala └── ThroughCustomPartitionerTopologyTests.scala /.gitignore: -------------------------------------------------------------------------------- 1 | # sbt 2 | # (may want to keep parts of 'project') 3 | bin/ 4 | project/ 5 | target/ 6 | build/ 7 | 8 | # eclipse 9 | build 10 | .classpath 11 | .project 12 | .settings 13 | .worksheet 14 | 15 | # intellij idea 16 | *.log 17 | *.iml 18 | *.ipr 19 | *.iws 20 | .idea 21 | 22 | # mac 23 | .DS_Store 24 | 25 | # other? 26 | .history 27 | .scala_dependencies 28 | .cache 29 | .cache-main 30 | 31 | #general 32 | *.class -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # KafkaStreamsDemo 2 | Set of small KafkaStreams demos/tests written in Scala 2.12, using Kafka 2.1.1 3 | 4 | 57 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/RunPipeline.ps1: -------------------------------------------------------------------------------- 1 | $global:kafkaWindowsBatFolder = "C:\Apache\confluent-5.2.1-2.12\bin\windows\" 2 | $global:kafkaAndZooLoggingFolder = "C:\temp\" 3 | $global:kafkaAndZooTmpFolder = "C:\tmp\" 4 | 5 | 6 | $global:kafkaTopics = 7 | "rating-submit-topic" 8 | 9 | $global:ProcessesToKill = @() 10 | 11 | 12 | 13 | function RunPipeLine() 14 | { 15 | WriteHeader "STOPPING PREVIOUS SERVICES" 16 | StopZookeeper 17 | StopKafka 18 | 19 | $path = $kafkaAndZooLoggingFolder + "kafka-logs" 20 | Remove-Item -Recurse -Force $path 21 | 22 | $path = $kafkaAndZooLoggingFolder + "zookeeper" 23 | Remove-Item -Recurse -Force $path 24 | 25 | $path = $kafkaAndZooLoggingFolder 26 | Remove-Item -Recurse -Force $path 27 | 28 | 29 | Start-Sleep -s 20 30 | 31 | WriteHeader "STARTING NEW SERVICE INSTANCES" 32 | StartZookeeper 33 | 34 | Start-Sleep -s 20 35 | StartKafka 36 | 37 | Start-Sleep -s 20 38 | 39 | CreateKafkaTopics 40 | 41 | Start-Sleep -s 20 42 | 43 | 44 | WaitForKeyPress 45 | 46 | WriteHeader "KILLING PROCESSES CREATED BY SCRIPT" 47 | KillProcesses 48 | } 49 | 50 | function WriteHeader($text) 51 | { 52 | Write-Host "========================================`r`n" 53 | Write-Host "$text`r`n" 54 | Write-Host "========================================`r`n" 55 | } 56 | 57 | 58 | function StopZookeeper() { 59 | # C:\Apache\confluent-5.2.1-2.12\bin\windows\zookeeper-server-stop.bat 60 | $zookeeperCommandLine = $global:kafkaWindowsBatFolder + "zookeeper-server-stop.bat" 61 | Write-Host "> Zookeeper Command Line : $zookeeperCommandLine`r`n" 62 | $global:ProcessesToKill += start-process $zookeeperCommandLine -WindowStyle Normal -PassThru 63 | } 64 | 65 | function StopKafka() { 66 | # C:\Apache\confluent-5.2.1-2.12\bin\windows\kafka-server-stop.bat 67 | $kafkaServerCommandLine = $global:kafkaWindowsBatFolder + "kafka-server-stop.bat" 68 | Write-Host "> Kafka Server Command Line : $kafkaServerCommandLine`r`n" 69 | $global:ProcessesToKill += start-process $kafkaServerCommandLine -WindowStyle Normal -PassThru 70 | } 71 | 72 | function StartZookeeper() { 73 | # C:\Apache\confluent-5.2.1-2.12\bin\windows\zookeeper-server-start.bat C:\Apache\confluent-5.2.1-2.12\bin\windows\..\..\etc\kafka\zookeeper.properties 74 | $zookeeperCommandLine = $global:kafkaWindowsBatFolder + "zookeeper-server-start.bat" 75 | $arguments = $global:kafkaWindowsBatFolder + "..\..\etc\kafka\zookeeper.properties" 76 | Write-Host "> Zookeeper Command Line : $zookeeperCommandLine args: $arguments `r`n" 77 | $global:ProcessesToKill += start-process $zookeeperCommandLine $arguments -WindowStyle Normal -PassThru 78 | } 79 | 80 | function StartKafka() { 81 | # C:\Apache\confluent-5.2.1-2.12\bin\windows\kafka-server-start.bat C:\Apache\confluent-5.2.1-2.12\bin\windows\..\..\etc\kafka\server.properties 82 | $kafkaServerCommandLine = $global:kafkaWindowsBatFolder + "kafka-server-start.bat" 83 | $arguments = $global:kafkaWindowsBatFolder + "..\..\etc\kafka\server.properties" 84 | Write-Host "> Kafka Server Command Line : $kafkaServerCommandLine args: $arguments `r`n" 85 | $global:ProcessesToKill += start-process $kafkaServerCommandLine $arguments -WindowStyle Normal -PassThru 86 | } 87 | 88 | function CreateKafkaTopics() 89 | { 90 | # C:\Apache\confluent-5.2.1-2.12\bin\windows\kafka-topics.bat --zookeeper localhost:2181 --delete --topic rating-submit-topic 91 | 92 | # C:\Apache\confluent-5.2.1-2.12\bin\windows\kafka-topics.bat --zookeeper localhost:2181 --create --replication-factor 1 --partitions 1 --topic rating-submit-topic 93 | Foreach ($topic in $global:kafkaTopics ) 94 | { 95 | $kafkaCommandLine = $global:kafkaWindowsBatFolder + "kafka-topics.bat" 96 | $arguments = "--zookeeper localhost:2181 --create --replication-factor 1 --partitions 1 --topic $topic" 97 | Write-Host "> Create Kafka Topic Command Line : $kafkaCommandLine args: $arguments `r`n" 98 | $global:ProcessesToKill += start-process $kafkaCommandLine $arguments -WindowStyle Normal -PassThru 99 | } 100 | } 101 | 102 | function WaitForKeyPress 103 | { 104 | Write-Host -NoNewLine "Press any key to continue....`r`n" 105 | [Console]::ReadKey() 106 | } 107 | 108 | 109 | function KillProcesses() 110 | { 111 | Foreach ($processToKill in $global:ProcessesToKill ) 112 | { 113 | $name = $processToKill | Get-ChildItem -Name 114 | Write-Host "Killing Process : $name `r`n" 115 | $processToKill | Stop-Process -Force 116 | } 117 | } 118 | 119 | 120 | # Kick of the entire pipeline 121 | RunPipeLine -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/build.sbt: -------------------------------------------------------------------------------- 1 | name := "ScalaKafkaStreamsDemo" 2 | 3 | version := "1.0" 4 | 5 | scalaVersion := "2.12.1" 6 | 7 | libraryDependencies += "javax.ws.rs" % "javax.ws.rs-api" % "2.1.1" artifacts(Artifact("javax.ws.rs-api", "jar", "jar")) 8 | libraryDependencies += "org.apache.kafka" %% "kafka" % "2.1.0" 9 | libraryDependencies += "org.apache.kafka" % "kafka-clients" % "2.1.0" 10 | libraryDependencies += "org.apache.kafka" % "kafka-streams" % "2.1.0" 11 | libraryDependencies += "org.apache.kafka" %% "kafka-streams-scala" % "2.1.0" 12 | libraryDependencies += "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.9.8" 13 | libraryDependencies += "com.typesafe" % "config" % "1.3.4" 14 | libraryDependencies += "com.typesafe.akka" %% "akka-http-spray-json" % "10.0.9" 15 | libraryDependencies += "com.typesafe.akka" %% "akka-http" % "10.0.9" 16 | libraryDependencies += "org.rocksdb" % "rocksdbjni" % "5.6.1" 17 | libraryDependencies += "org.apache.commons" % "commons-lang3" % "3.6" 18 | libraryDependencies += "ch.qos.logback" % "logback-classic" % "1.1.3" 19 | 20 | //TEST 21 | libraryDependencies += "org.scalatest" %% "scalatest" % "3.0.5" % Test 22 | libraryDependencies += "org.apache.kafka" % "kafka-streams-test-utils" % "2.1.0" % Test 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/resources/streamsApplication.conf: -------------------------------------------------------------------------------- 1 | kafka { 2 | bootStrapServers = "localhost:9092" 3 | zooKeepers = "zookeeper:2181" 4 | schemaRegistryUrl = "http://localhost:8081" 5 | partition = 0, 6 | restApiDefaultHostName = "localhost", 7 | restApiDefaultPort = "8080" 8 | 9 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/StraightThroughTopology.scala: -------------------------------------------------------------------------------- 1 | import java.time.Duration 2 | 3 | import org.apache.kafka.streams.Topology 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream._ 9 | import org.apache.kafka.streams.KafkaStreams 10 | import utils.Settings 11 | 12 | 13 | class StraightThroughTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "straight-through-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[String, String] = 35 | builder.stream[String, String]("InputTopic") 36 | textLines.mapValues(v => v).to("OutputTopic") 37 | builder.build() 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/WordCountTopology.scala: -------------------------------------------------------------------------------- 1 | import java.time.Duration 2 | import java.util 3 | import java.util.Properties 4 | 5 | import org.apache.kafka.streams.{KafkaStreams, Topology} 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream._ 9 | import utils.Settings 10 | 11 | 12 | class WordCountTopology extends App { 13 | 14 | import Serdes._ 15 | 16 | val props: Properties = Settings.createBasicStreamProperties( 17 | "straight-through-application","localhost:9092") 18 | 19 | run() 20 | 21 | private def run(): Unit = { 22 | val topology = wordCountToplogyWithConfiguredStore() 23 | val streams: KafkaStreams = new KafkaStreams(topology, props) 24 | streams.start() 25 | sys.ShutdownHookThread { 26 | streams.close(Duration.ofSeconds(10)) 27 | } 28 | } 29 | 30 | def wordCountToplogy() : Topology = { 31 | 32 | import org.apache.kafka.streams.state.Stores 33 | val wordCountStoreName = "wordCountStore" 34 | val wordCountStoreSupplied = Stores.inMemoryKeyValueStore(wordCountStoreName) 35 | 36 | val builder = new StreamsBuilder() 37 | val textLines: KStream[String, String] = builder.stream("TextLinesTopic") 38 | val wordCounts = textLines.flatMapValues(x => x.toLowerCase.split("\\W+")) 39 | .groupBy((key, word) => word) 40 | .count()(Materialized.as(wordCountStoreSupplied)) 41 | wordCounts.toStream.to("WordsWithCountsTopic") 42 | builder.build() 43 | } 44 | 45 | def wordCountToplogyWithConfiguredStore() : Topology = { 46 | 47 | import org.apache.kafka.streams.state.Stores 48 | val wordCountStoreName = "wordCountStore" 49 | 50 | val logConfig = new util.HashMap[String, String] 51 | logConfig.put("retention.ms", "172800000") 52 | logConfig.put("retention.bytes", "10000000000") 53 | logConfig.put("cleanup.policy", "compact,delete") 54 | val wordCountStoreSupplier = Stores.inMemoryKeyValueStore(wordCountStoreName) 55 | val wordCountStoreBuilder = Stores.keyValueStoreBuilder(wordCountStoreSupplier, 56 | Serdes.String, Serdes.Long) 57 | .withLoggingEnabled(logConfig) 58 | .withCachingEnabled() 59 | 60 | 61 | val builder = new StreamsBuilder() 62 | 63 | builder.addStateStore(wordCountStoreBuilder) 64 | 65 | //NOTE : You may find you need to play with these Config values in order 66 | //to get the stateful operation to work correctly/how you want it to 67 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 68 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 69 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 70 | //By playing around with these values you should be able to find the values that work for you 71 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 72 | //cache flushing may occur 73 | val textLines: KStream[String, String] = builder.stream("TextLinesTopic") 74 | val wordCounts = textLines.flatMapValues(x => x.toLowerCase.split("\\W+")) 75 | .groupBy((key, word) => word) 76 | .count()(Materialized.as(wordCountStoreName)) 77 | 78 | 79 | wordCounts.toStream.to("WordsWithCountsTopic") 80 | builder.build() 81 | } 82 | 83 | } 84 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/entities/AkkaHttpEntitiesJsonFormats.scala: -------------------------------------------------------------------------------- 1 | package entities 2 | 3 | import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport._ 4 | import spray.json.DefaultJsonProtocol._ 5 | 6 | object AkkaHttpEntitiesJsonFormats { 7 | implicit val RatingFormat = jsonFormat3(Rating) 8 | implicit val HostStoreInfoFormat = jsonFormat3(HostStoreInfo) 9 | } 10 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/entities/Contributor.scala: -------------------------------------------------------------------------------- 1 | package entities 2 | 3 | case class Contributor(email: String, ranking: Float, lastUpdatedTime: Long) { 4 | 5 | def updatedWithinLastMillis(currentTime: Long, limit: Long): Boolean = { 6 | currentTime - this.lastUpdatedTime <= limit 7 | } 8 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/entities/HostStoreInfo.scala: -------------------------------------------------------------------------------- 1 | package entities 2 | 3 | case class HostStoreInfo(host: String, port: Int, storeNames: List[String]) 4 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/entities/Rating.scala: -------------------------------------------------------------------------------- 1 | package entities 2 | 3 | case class Rating(fromEmail: String, toEmail: String, score: Float) -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/interactive/queries/MetadataService.scala: -------------------------------------------------------------------------------- 1 | package interactive.queries 2 | 3 | import org.apache.kafka.streams.KafkaStreams 4 | import org.apache.kafka.streams.state.StreamsMetadata 5 | import java.util.stream.Collectors 6 | import entities.HostStoreInfo 7 | import org.apache.kafka.common.serialization.Serializer 8 | import org.apache.kafka.connect.errors.NotFoundException 9 | import scala.collection.JavaConverters._ 10 | 11 | 12 | /** 13 | * Looks up StreamsMetadata from KafkaStreams 14 | */ 15 | class MetadataService(val streams: KafkaStreams) { 16 | 17 | 18 | /** 19 | * Get the metadata for all of the instances of this Kafka Streams application 20 | * 21 | * @return List of { @link HostStoreInfo} 22 | */ 23 | def streamsMetadata() : List[HostStoreInfo] = { 24 | 25 | // Get metadata for all of the instances of this Kafka Streams application 26 | val metadata = streams.allMetadata 27 | return mapInstancesToHostStoreInfo(metadata) 28 | } 29 | 30 | 31 | /** 32 | * Get the metadata for all instances of this Kafka Streams application that currently 33 | * has the provided store. 34 | * 35 | * @param store The store to locate 36 | * @return List of { @link HostStoreInfo} 37 | */ 38 | def streamsMetadataForStore(store: String) : List[HostStoreInfo] = { 39 | 40 | // Get metadata for all of the instances of this Kafka Streams application hosting the store 41 | val metadata = streams.allMetadataForStore(store) 42 | return mapInstancesToHostStoreInfo(metadata) 43 | } 44 | 45 | 46 | /** 47 | * Find the metadata for the instance of this Kafka Streams Application that has the given 48 | * store and would have the given key if it exists. 49 | * 50 | * @param store Store to find 51 | * @param key The key to find 52 | * @return { @link HostStoreInfo} 53 | */ 54 | def streamsMetadataForStoreAndKey[T](store: String, key: T, serializer: Serializer[T]) : HostStoreInfo = { 55 | // Get metadata for the instances of this Kafka Streams application hosting the store and 56 | // potentially the value for key 57 | val metadata = streams.metadataForKey(store, key, serializer) 58 | if (metadata == null) 59 | throw new NotFoundException( 60 | s"No metadata could be found for store : ${store}, and key type : ${key.getClass.getName}") 61 | 62 | HostStoreInfo(metadata.host, metadata.port, metadata.stateStoreNames.asScala.toList) 63 | } 64 | 65 | 66 | def mapInstancesToHostStoreInfo(metadatas : java.util.Collection[StreamsMetadata]) : List[HostStoreInfo] = { 67 | metadatas.stream.map[HostStoreInfo](metadata => 68 | HostStoreInfo( 69 | metadata.host(), 70 | metadata.port, 71 | metadata.stateStoreNames.asScala.toList)) 72 | .collect(Collectors.toList()) 73 | .asScala.toList 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/interactive/queries/ratings/RatingRestService.scala: -------------------------------------------------------------------------------- 1 | package interactive.queries.ratings 2 | 3 | import org.apache.kafka.streams.KafkaStreams 4 | import org.apache.kafka.streams.state.HostInfo 5 | import akka.actor.ActorSystem 6 | import akka.http.scaladsl.Http 7 | import akka.http.scaladsl.model._ 8 | import akka.http.scaladsl.server.Directives._ 9 | import akka.stream.ActorMaterializer 10 | import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport._ 11 | import spray.json.DefaultJsonProtocol._ 12 | import entities.AkkaHttpEntitiesJsonFormats._ 13 | import entities._ 14 | import utils.StateStores 15 | import akka.http.scaladsl.marshalling.ToResponseMarshallable 16 | import org.apache.kafka.common.serialization.Serdes 17 | import scala.concurrent.{Await, ExecutionContext, Future} 18 | import akka.http.scaladsl.unmarshalling.Unmarshal 19 | import interactive.queries.MetadataService 20 | import spray.json._ 21 | import scala.util.{Failure, Success} 22 | import org.apache.kafka.streams.state.QueryableStoreTypes 23 | import scala.concurrent.duration._ 24 | 25 | 26 | object RestService { 27 | val DEFAULT_REST_ENDPOINT_HOSTNAME = "localhost" 28 | } 29 | 30 | class RatingRestService(val streams: KafkaStreams, val hostInfo: HostInfo) { 31 | 32 | val metadataService = new MetadataService(streams) 33 | var bindingFuture: Future[Http.ServerBinding] = null 34 | 35 | implicit val system = ActorSystem("rating-system") 36 | implicit val materializer = ActorMaterializer() 37 | implicit val executionContext = system.dispatcher 38 | 39 | var isStateStoredReady: Boolean = false 40 | 41 | 42 | def setReady(isReady : Boolean): Unit = { 43 | isStateStoredReady = isReady 44 | } 45 | 46 | 47 | def start() : Unit = { 48 | val emailRegexPattern = """\w+""".r 49 | val storeNameRegexPattern = """\w+""".r 50 | 51 | val route = 52 | path("ratingByEmail") { 53 | get { 54 | parameters('email.as[String]) { (email) => 55 | 56 | if(!isStateStoredReady) { 57 | complete(HttpResponse(StatusCodes.InternalServerError, entity = "state stored not queryable, possible due to re-balancing")) 58 | } 59 | 60 | try { 61 | 62 | val host = metadataService.streamsMetadataForStoreAndKey[String]( 63 | StateStores.RATINGS_BY_EMAIL_STORE, 64 | email, 65 | Serdes.String().serializer() 66 | ) 67 | 68 | //store is hosted on another process, REST Call 69 | if(!thisHost(host)) { 70 | onComplete(fetchRemoteRatingByEmail(host, email)) { 71 | case Success(value) => complete(value) 72 | case Failure(ex) => complete(HttpResponse(StatusCodes.InternalServerError, entity = ex.getMessage)) 73 | } 74 | } 75 | else { 76 | onComplete(fetchLocalRatingByEmail(email)) { 77 | case Success(value) => complete(value) 78 | case Failure(ex) => complete(HttpResponse(StatusCodes.InternalServerError, entity = ex.getMessage)) 79 | } 80 | } 81 | } 82 | catch { 83 | case (ex: Exception) => { 84 | complete(HttpResponse(StatusCodes.InternalServerError, entity = ex.getMessage)) 85 | } 86 | } 87 | } 88 | } 89 | } ~ 90 | path("instances") { 91 | get { 92 | if(!isStateStoredReady) { 93 | complete(HttpResponse(StatusCodes.InternalServerError, entity = "state stored not queryable, possible due to re-balancing")) 94 | } 95 | complete(ToResponseMarshallable.apply(metadataService.streamsMetadata)) 96 | } 97 | }~ 98 | path("instances" / storeNameRegexPattern) { storeName => 99 | get { 100 | if(!isStateStoredReady) { 101 | complete(HttpResponse(StatusCodes.InternalServerError, entity = "state stored not queryable, possible due to re-balancing")) 102 | } 103 | complete(ToResponseMarshallable.apply(metadataService.streamsMetadataForStore(storeName))) 104 | } 105 | } 106 | 107 | bindingFuture = Http().bindAndHandle(route, hostInfo.host, hostInfo.port) 108 | println(s"Server online at http://${hostInfo.host}:${hostInfo.port}/\n") 109 | 110 | Runtime.getRuntime.addShutdownHook(new Thread(() => { 111 | bindingFuture 112 | .flatMap(_.unbind()) // trigger unbinding from the port 113 | .onComplete(_ => system.terminate()) // and shutdown when done 114 | })) 115 | } 116 | 117 | 118 | def fetchRemoteRatingByEmail(host:HostStoreInfo, email: String) : Future[List[Rating]] = { 119 | 120 | val requestPath = s"http://${hostInfo.host}:${hostInfo.port}/ratingByEmail?email=${email}" 121 | println(s"Client attempting to fetch from online at ${requestPath}") 122 | 123 | val responseFuture: Future[List[Rating]] = { 124 | Http().singleRequest(HttpRequest(uri = requestPath)) 125 | .flatMap(response => Unmarshal(response.entity).to[List[Rating]]) 126 | } 127 | 128 | responseFuture 129 | } 130 | 131 | def fetchLocalRatingByEmail(email: String) : Future[List[Rating]] = { 132 | 133 | val ec = ExecutionContext.global 134 | 135 | println(s"client fetchLocalRatingByEmail email=${email}") 136 | 137 | val host = metadataService.streamsMetadataForStoreAndKey[String]( 138 | StateStores.RATINGS_BY_EMAIL_STORE, 139 | email, 140 | Serdes.String().serializer() 141 | ) 142 | 143 | val f = StateStores.waitUntilStoreIsQueryable( 144 | StateStores.RATINGS_BY_EMAIL_STORE, 145 | QueryableStoreTypes.keyValueStore[String,List[Rating]](), 146 | streams 147 | ).map(_.get(email))(ec) 148 | 149 | val mapped = f.map(rating => { 150 | if (rating == null) 151 | List[Rating]() 152 | else 153 | rating 154 | }) 155 | 156 | mapped 157 | } 158 | 159 | def stop() : Unit = { 160 | bindingFuture 161 | .flatMap(_.unbind()) // trigger unbinding from the port 162 | .onComplete(_ => system.terminate()) // and shutdown when done 163 | } 164 | 165 | def thisHost(hostStoreInfo: HostStoreInfo) : Boolean = { 166 | hostStoreInfo.host.equals(hostInfo.host()) && 167 | hostStoreInfo.port == hostInfo.port 168 | } 169 | } 170 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/interactive/queries/ratings/RatingStreamProcessingTopology.scala: -------------------------------------------------------------------------------- 1 | package interactive.queries.ratings 2 | 3 | 4 | import java.util 5 | 6 | import entities.Rating 7 | import org.apache.kafka.streams.scala.{Serdes, _} 8 | import org.apache.kafka.streams.scala.kstream._ 9 | import org.apache.kafka.streams.Topology 10 | import org.apache.kafka.streams.state.KeyValueStore 11 | import serialization.JSONSerde 12 | import utils.StateStores 13 | 14 | 15 | class RatingStreamProcessingTopology { 16 | 17 | def createTopolgy(): Topology = { 18 | 19 | implicit val stringSerde = Serdes.String 20 | implicit val ratingSerde = new JSONSerde[Rating] 21 | implicit val listRatingSerde = new JSONSerde[List[Rating]] 22 | implicit val consumed = kstream.Consumed.`with`(stringSerde, ratingSerde) 23 | implicit val materializer = Materialized.`with`(stringSerde, listRatingSerde) 24 | implicit val grouped = Grouped.`with`(stringSerde, ratingSerde) 25 | 26 | val builder: StreamsBuilder = new StreamsBuilder 27 | val ratings: KStream[String, Rating] = 28 | builder.stream[String, Rating]("rating-submit-topic") 29 | 30 | 31 | import org.apache.kafka.streams.state.Stores 32 | 33 | val logConfig = new util.HashMap[String, String] 34 | logConfig.put("retention.ms", "172800000") 35 | logConfig.put("retention.bytes", "10000000000") 36 | logConfig.put("cleanup.policy", "compact,delete") 37 | val ratingByEmailStoreName = StateStores.RATINGS_BY_EMAIL_STORE 38 | val ratingByEmailStoreSupplied = Stores.inMemoryKeyValueStore(ratingByEmailStoreName) 39 | val ratingByEmailStoreBuilder = Stores.keyValueStoreBuilder(ratingByEmailStoreSupplied, 40 | Serdes.String, listRatingSerde) 41 | .withLoggingEnabled(logConfig) 42 | .withCachingEnabled() 43 | 44 | val builtStore = ratingByEmailStoreBuilder.build() 45 | 46 | //When aggregating a grouped stream, you must provide an initializer (e.g., aggValue = 0) 47 | //and an “adder” aggregator (e.g., aggValue + curValue). When aggregating a grouped table, 48 | //you must provide a “subtractor” aggregator (think: aggValue - oldValue). 49 | val groupedBy = ratings.groupByKey 50 | 51 | //aggrgate by (user email -> their ratings) 52 | val ratingTable : KTable[String, List[Rating]] = groupedBy 53 | .aggregate[List[Rating]](List[Rating]())((aggKey, newValue, aggValue) => { 54 | newValue :: aggValue 55 | })(Materialized.as[String, List[Rating]](ratingByEmailStoreSupplied)) 56 | 57 | ratingTable.mapValues((k,v) => { 58 | val theKey = k 59 | val theValue = v 60 | v 61 | }) 62 | 63 | 64 | builder.build() 65 | } 66 | } 67 | 68 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/interactive/queries/ratings/RatingStreamProcessingTopologyApp.scala: -------------------------------------------------------------------------------- 1 | package interactive.queries.ratings 2 | 3 | import java.util.Properties 4 | 5 | import entities.Rating 6 | import org.apache.kafka.common.serialization.Serdes 7 | import org.apache.kafka.streams.errors.BrokerNotFoundException 8 | import org.apache.kafka.streams.state.{HostInfo, QueryableStoreTypes} 9 | import org.apache.kafka.streams.KafkaStreams 10 | import utils.{Retry, Settings, StateStores} 11 | 12 | import scala.concurrent.duration._ 13 | import scala.concurrent.ExecutionContext 14 | import scala.util.Success 15 | import java.util.concurrent.CountDownLatch 16 | 17 | object RatingStreamProcessingTopologyApp extends App { 18 | 19 | import Serdes._ 20 | 21 | 22 | implicit val ec = ExecutionContext.global 23 | val doneSignal = new CountDownLatch(1) 24 | 25 | run() 26 | 27 | private def run(): Unit = { 28 | 29 | val restEndpoint: HostInfo = new HostInfo(Settings.restApiDefaultHostName, Settings.restApiDefaultPort) 30 | System.out.println(s"Connecting to Kafka cluster via bootstrap servers ${Settings.bootStrapServers}") 31 | System.out.println(s"REST endpoint at http://${restEndpoint.host}:${restEndpoint.port}") 32 | 33 | val props: Properties = Settings.createRatingStreamsProperties() 34 | val topology = new RatingStreamProcessingTopology().createTopolgy() 35 | val streams: KafkaStreams = new KafkaStreams(topology,props) 36 | 37 | val restService = new RatingRestService(streams, restEndpoint) 38 | 39 | //Can only add this in State == CREATED 40 | streams.setUncaughtExceptionHandler(( thread :Thread, throwable : Throwable) => { 41 | println(s"============> ${throwable.getMessage}") 42 | shutDown(streams,restService) 43 | 44 | }) 45 | 46 | streams.setStateListener((newState, oldState) => { 47 | if (newState == KafkaStreams.State.RUNNING && oldState == KafkaStreams.State.REBALANCING) { 48 | restService.setReady(true) 49 | } else if (newState != KafkaStreams.State.RUNNING) { 50 | restService.setReady(false) 51 | } 52 | }) 53 | 54 | restService.start() 55 | 56 | Runtime.getRuntime.addShutdownHook(new Thread(() => { 57 | shutDown(streams,restService) 58 | })) 59 | 60 | println("Starting KafkaStream") 61 | 62 | // Always (and unconditionally) clean local state prior to starting the processing topology. 63 | // We opt for this unconditional call here because this will make it easier for you to 64 | // play around with the example when resetting the application for doing a re-run 65 | // (via the Application Reset Tool, 66 | // http://docs.confluent.io/current/streams/developer-guide.html#application-reset-tool). 67 | // 68 | // The drawback of cleaning up local state prior is that your app must rebuilt its local 69 | // state from scratch, which will take time and will require reading all the state-relevant 70 | // data from the Kafka cluster over the network. 71 | // Thus in a production scenario you typically do not want to clean up always as we do 72 | // here but rather only when it is truly needed, i.e., only under certain conditions 73 | // (e.g., the presence of a command line flag for your app). 74 | // See `ApplicationResetExample.java` for a production-like example. 75 | streams.cleanUp 76 | streams.start 77 | 78 | doneSignal.await 79 | () 80 | } 81 | 82 | 83 | private def shutDown(streams: KafkaStreams, restService: RatingRestService): Unit = { 84 | doneSignal.countDown 85 | streams.close() 86 | restService.stop 87 | } 88 | } 89 | 90 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/interactive/queries/ratings/RatingsProducerApp.scala: -------------------------------------------------------------------------------- 1 | package interactive.queries.ratings 2 | 3 | import java.util.concurrent.TimeUnit 4 | 5 | import entities.Rating 6 | import serialization.JSONSerde 7 | 8 | import scala.util.Random 9 | import org.apache.kafka.clients.producer.ProducerRecord 10 | import org.apache.kafka.clients.producer.KafkaProducer 11 | import org.apache.kafka.common.serialization.Serdes 12 | import utils.Settings 13 | import org.apache.kafka.clients.producer.ProducerConfig 14 | 15 | object RatingsProducerApp extends App { 16 | 17 | run() 18 | 19 | private def run(): Unit = { 20 | 21 | val jSONSerde = new JSONSerde[Rating] 22 | val random = new Random 23 | val producerProps = Settings.createBasicProducerProperties 24 | val ratingsList = List( 25 | Rating("jarden@here.com","sacha@here.com", 1.5f), 26 | Rating("miro@here.com","mary@here.com", 1.5f), 27 | Rating("anne@here.com","margeret@here.com", 3.5f), 28 | Rating("frank@here.com","bert@here.com", 2.5f), 29 | Rating("morgan@here.com","ruth@here.com", 1.5f)) 30 | 31 | System.out.println("Connecting to Kafka cluster via bootstrap servers " + 32 | s"${producerProps.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)}") 33 | 34 | // send a random string from List event every 100 milliseconds 35 | val ratingProducer = new KafkaProducer[String, Array[Byte]]( 36 | producerProps, Serdes.String.serializer, Serdes.ByteArray.serializer) 37 | 38 | while (true) { 39 | //for (i <- 0 to 10) { 40 | //val rating = ratingsList(random.nextInt(ratingsList.size)) 41 | val rating = ratingsList(0) 42 | val ratingBytes = jSONSerde.serializer().serialize("", rating) 43 | System.out.println(s"Writing rating ${rating} to input topic rating-submit-topic") 44 | ratingProducer.send(new ProducerRecord[String, Array[Byte]]( 45 | "rating-submit-topic", rating.toEmail, ratingBytes)) 46 | Thread.sleep(500) 47 | } 48 | 49 | Runtime.getRuntime.addShutdownHook(new Thread(() => { 50 | ratingProducer.close(10, TimeUnit.SECONDS) 51 | })) 52 | } 53 | } 54 | 55 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/joining/InnerJoinTopology.scala: -------------------------------------------------------------------------------- 1 | package joining 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | import java.util.concurrent.TimeUnit 6 | 7 | import org.apache.kafka.streams.kstream.JoinWindows 8 | import org.apache.kafka.streams.scala.ImplicitConversions._ 9 | import org.apache.kafka.streams.scala._ 10 | import org.apache.kafka.streams.scala.kstream._ 11 | import org.apache.kafka.streams.{KafkaStreams, Topology} 12 | import utils.Settings 13 | 14 | import scala.util.Try 15 | 16 | 17 | class InnerJoinTopology extends App { 18 | 19 | import Serdes._ 20 | 21 | val props: Properties = Settings.createBasicStreamProperties( 22 | "inner-join-application", "localhost:9092") 23 | 24 | run() 25 | 26 | private def run(): Unit = { 27 | val topology = createTopolgy() 28 | val streams: KafkaStreams = new KafkaStreams(topology, props) 29 | streams.start() 30 | sys.ShutdownHookThread { 31 | streams.close(Duration.ofSeconds(10)) 32 | } 33 | } 34 | 35 | def createTopolgy(): Topology = { 36 | 37 | val builder: StreamsBuilder = new StreamsBuilder 38 | val left: KStream[Int, String] = 39 | builder.stream[Int, String]("LeftTopic") 40 | val right: KStream[Int, String] = 41 | builder.stream[Int, String]("RightTopic") 42 | 43 | def tryToInt( s: String ) = Try(s.toInt).toOption 44 | 45 | //do the join 46 | val joinedStream = left.join(right)( (l: String, r: String) => { 47 | val result = (tryToInt(l), tryToInt(r)) 48 | result match { 49 | case (Some(a), Some(b)) => a + b 50 | case (None, Some(b)) => b 51 | case (Some(a), None) => a 52 | case (None, None) => 0 53 | } 54 | } , JoinWindows.of(Duration.ofSeconds(2))) 55 | 56 | 57 | val peeked = joinedStream.peek((k,v)=> { 58 | 59 | val theK = k 60 | val theV = v 61 | }) 62 | 63 | peeked.to("innerJoinOutput") 64 | 65 | builder.build() 66 | } 67 | } 68 | 69 | 70 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/joining/LeftJoinTopology.scala: -------------------------------------------------------------------------------- 1 | package joining 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.kstream.JoinWindows 7 | import org.apache.kafka.streams.scala.ImplicitConversions._ 8 | import org.apache.kafka.streams.scala._ 9 | import org.apache.kafka.streams.scala.kstream._ 10 | import org.apache.kafka.streams.{KafkaStreams, Topology} 11 | import utils.Settings 12 | 13 | import scala.util.Try 14 | 15 | 16 | class LeftJoinTopology extends App { 17 | 18 | import Serdes._ 19 | 20 | val props: Properties = Settings.createBasicStreamProperties( 21 | "left-join-application", "localhost:9092") 22 | 23 | run() 24 | 25 | private def run(): Unit = { 26 | val topology = createTopolgy() 27 | val streams: KafkaStreams = new KafkaStreams(topology, props) 28 | streams.start() 29 | sys.ShutdownHookThread { 30 | streams.close(Duration.ofSeconds(10)) 31 | } 32 | } 33 | 34 | def createTopolgy(): Topology = { 35 | 36 | val builder: StreamsBuilder = new StreamsBuilder 37 | val left: KStream[Int, String] = 38 | builder.stream[Int, String]("LeftTopic") 39 | val right: KStream[Int, String] = 40 | builder.stream[Int, String]("RightTopic") 41 | 42 | //do the join 43 | val joinedStream = left.leftJoin(right)( (l: String, r: String) => { 44 | val result = s"Left='${l}', Right='${r}'" 45 | result 46 | } , JoinWindows.of(Duration.ofSeconds(2))) 47 | 48 | val peeked = joinedStream.peek((k,v)=> { 49 | 50 | val theK = k 51 | val theV = v 52 | }) 53 | 54 | peeked.to("leftJoinOutput") 55 | 56 | builder.build() 57 | } 58 | } 59 | 60 | 61 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/joining/OuterJoinTopology.scala: -------------------------------------------------------------------------------- 1 | package joining 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.kstream.JoinWindows 7 | import org.apache.kafka.streams.scala.ImplicitConversions._ 8 | import org.apache.kafka.streams.scala._ 9 | import org.apache.kafka.streams.scala.kstream._ 10 | import org.apache.kafka.streams.{KafkaStreams, Topology} 11 | import utils.Settings 12 | 13 | 14 | class OuterJoinTopology extends App { 15 | 16 | import Serdes._ 17 | 18 | val props: Properties = Settings.createBasicStreamProperties( 19 | "outer-join-application", "localhost:9092") 20 | 21 | run() 22 | 23 | private def run(): Unit = { 24 | val topology = createTopolgy() 25 | val streams: KafkaStreams = new KafkaStreams(topology, props) 26 | streams.start() 27 | sys.ShutdownHookThread { 28 | streams.close(Duration.ofSeconds(10)) 29 | } 30 | } 31 | 32 | def createTopolgy(): Topology = { 33 | 34 | val builder: StreamsBuilder = new StreamsBuilder 35 | val left: KStream[Int, String] = 36 | builder.stream[Int, String]("LeftTopic") 37 | val right: KStream[Int, String] = 38 | builder.stream[Int, String]("RightTopic") 39 | 40 | //do the join 41 | val joinedStream = left.outerJoin(right)( (l: String, r: String) => { 42 | val result = s"Left='${l}', Right='${r}'" 43 | result 44 | } , JoinWindows.of(Duration.ofSeconds(2))) 45 | 46 | val peeked = joinedStream.peek((k,v)=> { 47 | 48 | val theK = k 49 | val theV = v 50 | }) 51 | 52 | peeked.to("outerJoinOutput") 53 | 54 | builder.build() 55 | } 56 | } 57 | 58 | 59 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/processorapi/interop/ContributorPrintingSupplier.scala: -------------------------------------------------------------------------------- 1 | package processorapi.interop 2 | 3 | import java.io.PrintWriter 4 | 5 | import entities.Contributor 6 | import org.apache.kafka.streams.processor.{Processor, ProcessorSupplier} 7 | 8 | class ContributorPrintingSupplier(val pw: PrintWriter) extends ProcessorSupplier[String, Long] { 9 | override def get(): Processor[String, Long] = new Processor[String,Long] { 10 | 11 | import org.apache.kafka.streams.processor.ProcessorContext 12 | import org.apache.kafka.streams.state.KeyValueStore 13 | 14 | private var context:ProcessorContext = null 15 | private var contributorStore:KeyValueStore[String, Long] = null 16 | 17 | override def init(context: ProcessorContext): Unit = { 18 | import org.apache.kafka.streams.state.KeyValueStore 19 | this.context = context 20 | this.contributorStore = context.getStateStore("processContributorStore") 21 | .asInstanceOf[KeyValueStore[String, Long]] 22 | } 23 | 24 | override def process(key: String, value: Long): Unit = { 25 | pw.write(s"key ${key} has been seen ${value} times\r\n") 26 | pw.flush() 27 | } 28 | 29 | override def close(): Unit = { 30 | if(contributorStore != null) { 31 | contributorStore.close() 32 | } 33 | } 34 | } 35 | } 36 | 37 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/processorapi/interop/ContributorTranformSupplier.scala: -------------------------------------------------------------------------------- 1 | package processorapi.interop 2 | 3 | import java.time.Duration 4 | 5 | import entities.Contributor 6 | import org.apache.kafka.streams.kstream.{ValueTransformer, ValueTransformerSupplier} 7 | import org.apache.kafka.streams.processor.{PunctuationType, Punctuator} 8 | 9 | class ContributorTranformSupplier extends ValueTransformerSupplier[Contributor, Contributor] { 10 | override def get(): ValueTransformer[Contributor, Contributor] = new ValueTransformer[Contributor, Contributor] { 11 | 12 | import org.apache.kafka.streams.processor.ProcessorContext 13 | import org.apache.kafka.streams.state.KeyValueStore 14 | 15 | private var context:ProcessorContext = null 16 | private var contributorStore:KeyValueStore[String, Contributor] = null 17 | 18 | override def init(context: ProcessorContext): Unit = { 19 | import org.apache.kafka.streams.state.KeyValueStore 20 | this.context = context 21 | this.contributorStore = context.getStateStore("contributorStore") 22 | .asInstanceOf[KeyValueStore[String, Contributor]] 23 | 24 | //to punctuate you would do something like this 25 | // context.schedule(Duration.ofSeconds(1), 26 | // PunctuationType.WALL_CLOCK_TIME, new Punctuator { 27 | // override def punctuate(timestamp: Long): Unit = { 28 | // 29 | // val it = contributorStore.all 30 | // val currentTime = System.currentTimeMillis 31 | // while (it.hasNext) { 32 | // val contributorValue = it.next.value 33 | // if (contributorValue.updatedWithinLastMillis(currentTime, 11000)) 34 | // context.forward(contributorValue.email,contributorValue) 35 | // } 36 | // } 37 | // }) 38 | } 39 | 40 | override def transform(value: Contributor): Contributor = { 41 | 42 | var finalContributor:Contributor = null 43 | try { 44 | val contributor = contributorStore.get(value.email) 45 | if(contributor == null) { 46 | contributorStore.putIfAbsent(value.email, value) 47 | finalContributor = value 48 | } 49 | else { 50 | val newContributor = contributor.copy( 51 | ranking = contributor.ranking + 1, 52 | lastUpdatedTime = System.currentTimeMillis 53 | ) 54 | contributorStore.put(value.email,newContributor) 55 | finalContributor = newContributor 56 | } 57 | 58 | finalContributor 59 | } 60 | catch { 61 | case e:NullPointerException => { 62 | contributorStore.putIfAbsent(value.email, value) 63 | value 64 | } 65 | case e:Exception => { 66 | value 67 | } 68 | } 69 | } 70 | 71 | override def close(): Unit = { 72 | if(contributorStore != null) { 73 | contributorStore.close() 74 | } 75 | } 76 | } 77 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/processorapi/interop/ProcessorApiProcessSupplierTopology.scala: -------------------------------------------------------------------------------- 1 | package processorapi.interop 2 | 3 | import java.io.PrintWriter 4 | import java.time.Duration 5 | import java.util 6 | import java.util.Properties 7 | 8 | import entities.Contributor 9 | import org.apache.kafka.common.serialization.Serdes 10 | import org.apache.kafka.streams.scala.kstream._ 11 | import org.apache.kafka.streams.scala.{ByteArrayKeyValueStore, StreamsBuilder, kstream} 12 | import org.apache.kafka.streams.{KafkaStreams, Topology} 13 | import serialization.JSONSerde 14 | import utils.Settings 15 | 16 | 17 | class ProcessorApiProcessSupplierTopology(val pw: PrintWriter) extends App { 18 | 19 | val props: Properties = Settings.createBasicStreamProperties( 20 | "processor-api-process-supplier-application", "localhost:9092") 21 | 22 | run() 23 | 24 | private def run(): Unit = { 25 | val topology = createTopolgy() 26 | val streams: KafkaStreams = new KafkaStreams(topology, props) 27 | streams.start() 28 | sys.ShutdownHookThread { 29 | streams.close(Duration.ofSeconds(10)) 30 | } 31 | } 32 | 33 | def stop() : Unit = { 34 | pw.close() 35 | } 36 | 37 | def createTopolgy(): Topology = { 38 | 39 | implicit val stringSerde = Serdes.String 40 | implicit val contributorSerde = new JSONSerde[Contributor] 41 | implicit val consumed = kstream.Consumed.`with`(stringSerde, contributorSerde) 42 | implicit val grouped = Grouped.`with`(stringSerde, contributorSerde) 43 | 44 | import org.apache.kafka.streams.state.Stores 45 | val contributorStoreName = "processContributorStore" 46 | 47 | val logConfig = new util.HashMap[String, String] 48 | logConfig.put("retention.ms", "172800000") 49 | logConfig.put("retention.bytes", "10000000000") 50 | logConfig.put("cleanup.policy", "compact,delete") 51 | val contributorStoreSupplier = Stores.inMemoryKeyValueStore(contributorStoreName) 52 | val contributorStoreBuilder = Stores.keyValueStoreBuilder(contributorStoreSupplier, 53 | Serdes.String, Serdes.Long()) 54 | .withLoggingEnabled(logConfig) 55 | .withCachingEnabled() 56 | 57 | implicit val materializer = 58 | Materialized.as(contributorStoreSupplier)(Serdes.String, Serdes.Long()) 59 | .asInstanceOf[Materialized[String, Long,ByteArrayKeyValueStore ]] 60 | 61 | val builder: StreamsBuilder = new StreamsBuilder 62 | val contribs: KStream[String, Contributor] = 63 | builder.stream[String, Contributor]("ProcessorApiProcessorSupplierInputTopic") 64 | 65 | contribs 66 | .groupByKey 67 | .count()(materializer) 68 | .toStream 69 | .process(() => new ContributorPrintingSupplier(pw).get(), contributorStoreName) 70 | 71 | 72 | 73 | builder.build() 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/processorapi/interop/ProcessorApiTransformValuesTopology.scala: -------------------------------------------------------------------------------- 1 | package processorapi.interop 2 | 3 | import java.time.Duration 4 | import java.util 5 | import java.util.Properties 6 | 7 | import entities.Contributor 8 | import org.apache.kafka.common.serialization.Serdes 9 | import org.apache.kafka.streams.scala.{StreamsBuilder, kstream} 10 | import org.apache.kafka.streams.scala.kstream._ 11 | import org.apache.kafka.streams.{KafkaStreams, Topology} 12 | import serialization.JSONSerde 13 | import utils.Settings 14 | 15 | 16 | class ProcessorApiTransformValuesTopology extends App { 17 | 18 | val props: Properties = Settings.createBasicStreamProperties( 19 | "processor-api-transform-values-application", "localhost:9092") 20 | 21 | run() 22 | 23 | private def run(): Unit = { 24 | val topology = createTopolgy() 25 | val streams: KafkaStreams = new KafkaStreams(topology, props) 26 | streams.start() 27 | sys.ShutdownHookThread { 28 | streams.close(Duration.ofSeconds(10)) 29 | } 30 | } 31 | 32 | def createTopolgy(): Topology = { 33 | 34 | implicit val stringSerde = Serdes.String 35 | implicit val contributorSerde = new JSONSerde[Contributor] 36 | implicit val consumed = kstream.Consumed.`with`(stringSerde, contributorSerde) 37 | implicit val materializer = Materialized.`with`(stringSerde, contributorSerde) 38 | 39 | import org.apache.kafka.streams.state.Stores 40 | val contributorStoreName = "contributorStore" 41 | 42 | val logConfig = new util.HashMap[String, String] 43 | logConfig.put("retention.ms", "172800000") 44 | logConfig.put("retention.bytes", "10000000000") 45 | logConfig.put("cleanup.policy", "compact,delete") 46 | val contributorStoreSupplier = Stores.inMemoryKeyValueStore(contributorStoreName) 47 | val contributorStoreBuilder = Stores.keyValueStoreBuilder(contributorStoreSupplier, Serdes.String, contributorSerde) 48 | .withLoggingEnabled(logConfig) 49 | .withCachingEnabled() 50 | 51 | 52 | val builder: StreamsBuilder = new StreamsBuilder 53 | val contribs: KStream[String, Contributor] = 54 | builder.stream[String, Contributor]("ProcessorApiTransformValuesInputTopic") 55 | 56 | builder.addStateStore(contributorStoreBuilder) 57 | 58 | contribs 59 | .transformValues(new ContributorTranformSupplier, contributorStoreName) 60 | .to("ProcessorApiTransformValuesOutputTopic")(Produced.`with`(stringSerde, contributorSerde)) 61 | 62 | builder.build() 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/serialization/CustomSerdesTopology.scala: -------------------------------------------------------------------------------- 1 | package serialization 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import entities.Rating 7 | import org.apache.kafka.common.serialization.Serdes 8 | import org.apache.kafka.streams.scala.ImplicitConversions._ 9 | import org.apache.kafka.streams.scala._ 10 | import org.apache.kafka.streams.scala.kstream._ 11 | import org.apache.kafka.streams.{KafkaStreams, Topology} 12 | import utils.Settings 13 | 14 | 15 | class CustomSerdesTopology extends App { 16 | 17 | import Serdes._ 18 | 19 | val props: Properties = Settings.createBasicStreamProperties( 20 | "custom-serdes-application", "localhost:9092") 21 | 22 | run() 23 | 24 | private def run(): Unit = { 25 | val topology = createTopolgy() 26 | val streams: KafkaStreams = new KafkaStreams(topology, props) 27 | streams.start() 28 | sys.ShutdownHookThread { 29 | streams.close(Duration.ofSeconds(10)) 30 | } 31 | } 32 | 33 | def createTopolgy(): Topology = { 34 | 35 | implicit val stringSerde = Serdes.String 36 | implicit val ratingSerde = new JSONSerde[Rating] 37 | implicit val listRatingSerde = new JSONSerde[List[Rating]] 38 | implicit val consumed = kstream.Consumed.`with`(stringSerde, ratingSerde) 39 | implicit val materializer = Materialized.`with`(stringSerde, listRatingSerde) 40 | implicit val grouped = Grouped.`with`(stringSerde, ratingSerde) 41 | 42 | val builder: StreamsBuilder = new StreamsBuilder 43 | val ratings: KStream[String, Rating] = 44 | builder.stream[String, Rating]("CustomSerdesInputTopic") 45 | 46 | //When aggregating a grouped stream, you must provide an initializer (e.g., aggValue = 0) 47 | //and an “adder” aggregator (e.g., aggValue + curValue). When aggregating a grouped table, 48 | //you must provide a “subtractor” aggregator (think: aggValue - oldValue). 49 | val groupedBy = ratings.groupByKey 50 | val aggregatedTable = 51 | groupedBy 52 | .aggregate[List[Rating]](List[Rating]())((aggKey, newValue, aggValue) => newValue :: aggValue) 53 | 54 | var finalStream = aggregatedTable.toStream 55 | finalStream.peek((key, values) => { 56 | 57 | val theKey = key 58 | val theValues = values 59 | 60 | }) 61 | 62 | 63 | finalStream.to("CustomSerdesOutputTopic")(Produced.`with`(stringSerde, listRatingSerde)) 64 | 65 | builder.build() 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/serialization/JsonSerdes.scala: -------------------------------------------------------------------------------- 1 | import java.lang.reflect.{ParameterizedType, Type} 2 | import java.util 3 | import com.fasterxml.jackson.annotation.JsonInclude 4 | import com.fasterxml.jackson.core.JsonParseException 5 | import com.fasterxml.jackson.core.`type`.TypeReference 6 | import com.fasterxml.jackson.databind.ObjectMapper 7 | import com.fasterxml.jackson.databind.exc.{UnrecognizedPropertyException => UPE} 8 | import com.fasterxml.jackson.module.scala.DefaultScalaModule 9 | import org.apache.kafka.common.serialization.{Deserializer, Serde, Serializer} 10 | 11 | 12 | package serialization { 13 | 14 | object Json { 15 | 16 | type ParseException = JsonParseException 17 | type UnrecognizedPropertyException = UPE 18 | 19 | private val mapper = new ObjectMapper() 20 | mapper.registerModule(DefaultScalaModule) 21 | mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL) 22 | 23 | private def typeReference[T: Manifest] = new TypeReference[T] { 24 | override def getType = typeFromManifest(manifest[T]) 25 | } 26 | 27 | private def typeFromManifest(m: Manifest[_]): Type = { 28 | if (m.typeArguments.isEmpty) { 29 | m.runtimeClass 30 | } 31 | else new ParameterizedType { 32 | def getRawType = m.runtimeClass 33 | 34 | def getActualTypeArguments = m.typeArguments.map(typeFromManifest).toArray 35 | 36 | def getOwnerType = null 37 | } 38 | } 39 | 40 | object ByteArray { 41 | def encode(value: Any): Array[Byte] = mapper.writeValueAsBytes(value) 42 | 43 | def decode[T: Manifest](value: Array[Byte]): T = 44 | mapper.readValue(value, typeReference[T]) 45 | } 46 | 47 | } 48 | 49 | /** 50 | * JSON serializer for JSON serde 51 | * 52 | * @tparam T 53 | */ 54 | class JSONSerializer[T] extends Serializer[T] { 55 | override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = () 56 | 57 | override def serialize(topic: String, data: T): Array[Byte] = 58 | Json.ByteArray.encode(data) 59 | 60 | override def close(): Unit = () 61 | } 62 | 63 | /** 64 | * JSON deserializer for JSON serde 65 | * 66 | * @tparam T 67 | */ 68 | class JSONDeserializer[T >: Null <: Any : Manifest] extends Deserializer[T] { 69 | override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = () 70 | 71 | override def close(): Unit = () 72 | 73 | override def deserialize(topic: String, data: Array[Byte]): T = { 74 | if (data == null) { 75 | return null 76 | } else { 77 | Json.ByteArray.decode[T](data) 78 | } 79 | } 80 | } 81 | 82 | /** 83 | * JSON serde for local state serialization 84 | * 85 | * @tparam T 86 | */ 87 | class JSONSerde[T >: Null <: Any : Manifest] extends Serde[T] { 88 | override def deserializer(): Deserializer[T] = new JSONDeserializer[T] 89 | 90 | override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = () 91 | 92 | override def close(): Unit = () 93 | 94 | override def serializer(): Serializer[T] = new JSONSerializer[T] 95 | } 96 | 97 | } 98 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateful/transformations/aggregating/AggregateTopology.scala: -------------------------------------------------------------------------------- 1 | package stateful.transformations.aggregating 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream.{Materialized, _} 9 | import org.apache.kafka.streams.{KafkaStreams, Topology} 10 | import utils.Settings 11 | 12 | 13 | class AggregateTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "stateless-aggregate-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[Int, Int] = 35 | builder.stream[Int, Int]("AggregateKeyInputTopic") 36 | 37 | implicit val matererlized: Materialized[Int, Long, ByteArrayKeyValueStore] 38 | = Materialized.as("aggregated-stream-store") 39 | 40 | //Rolling aggregation. Aggregates the values of (non-windowed) records by the grouped key. 41 | //Aggregating is a generalization of reduce and allows, for example, the aggregate value to 42 | //have a different type than the input values. (KGroupedStream details, KGroupedTable details) 43 | // 44 | //When aggregating a grouped stream, you must provide an initializer (e.g., aggValue = 0) 45 | //and an “adder” aggregator (e.g., aggValue + curValue). When aggregating a grouped table, 46 | //you must provide a “subtractor” aggregator (think: aggValue - oldValue). 47 | val groupedBy = textLines.groupByKey 48 | val aggregatedTable = 49 | groupedBy 50 | .aggregate[Long](0L)((aggKey, newValue, aggValue) => aggValue + newValue)(matererlized) 51 | aggregatedTable 52 | .toStream 53 | .peek((k,v) => 54 | { 55 | val theKey = k 56 | val theValue =v 57 | }) 58 | .to("aggregateOutputTopic") 59 | builder.build() 60 | } 61 | } 62 | 63 | 64 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateful/transformations/aggregating/CountTopology.scala: -------------------------------------------------------------------------------- 1 | package stateful.transformations.aggregating 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream.{Materialized, _} 9 | import org.apache.kafka.streams.{KafkaStreams, Topology} 10 | import utils.Settings 11 | 12 | 13 | class CountTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "stateless-count-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[String, String] = 35 | builder.stream[String, String]("CountInputTopic") 36 | 37 | 38 | //lets create a named wordCountStore state store 39 | //The "persistentKeyValueStore" is one of the pre-canned state store types 40 | //and as such logging is enabled, so the ChangeLog topic to persist state 41 | import org.apache.kafka.streams.state.Stores 42 | val wordCountStoreName = "wordCountStore" 43 | val wordCountStoreSupplied = Stores.persistentKeyValueStore(wordCountStoreName) 44 | 45 | val wordCounts = textLines 46 | .flatMapValues(x => x.toLowerCase.split("\\W+")) 47 | .groupBy((key, word) => word) 48 | .count()(Materialized.as(wordCountStoreSupplied)) 49 | wordCounts 50 | .toStream 51 | .peek((k,v) => 52 | { 53 | val theKey = k 54 | val theValue =v 55 | }) 56 | .to("WordsWithCountsOutputTopic") 57 | 58 | builder.build() 59 | } 60 | } 61 | 62 | 63 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateful/transformations/aggregating/ReduceTopology.scala: -------------------------------------------------------------------------------- 1 | package stateful.transformations.aggregating 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream.{Materialized, _} 9 | import org.apache.kafka.streams.{KafkaStreams, Topology} 10 | import utils.Settings 11 | 12 | 13 | class ReduceTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "stateless-reduce-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[String, String] = 35 | builder.stream[String, String]("ReduceInputTopic") 36 | 37 | 38 | //lets create a named reduceStore state store 39 | //The "persistentKeyValueStore" is one of the pre-canned state store types 40 | //and as such logging is enabled, so the ChangeLog topic to persist state 41 | import org.apache.kafka.streams.state.Stores 42 | val reduceStoreName = "reduceStore" 43 | val reduceStoreSupplied = Stores.persistentKeyValueStore(reduceStoreName) 44 | 45 | //by using this dummy topic we can go straight from a KSTream to kTable 46 | textLines.to("Dummy-ReduceInputTopic") 47 | val userProfiles : KTable[String, String] = builder.table("Dummy-ReduceInputTopic") 48 | 49 | 50 | val groupedTable = userProfiles.groupBy((user, region) => (region, user.length())) 51 | val reduced: KTable[String, Integer] = 52 | groupedTable.reduce( 53 | (aggValue : Int, newValue: Int) => aggValue + newValue, 54 | (aggValue: Int, oldValue: Int) => aggValue - oldValue) 55 | .mapValues(v => new Integer(v)) 56 | 57 | val finalStream = reduced.toStream 58 | finalStream.to("ReduceOutputTopic") 59 | 60 | builder.build() 61 | } 62 | } 63 | 64 | 65 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/BranchTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.{KafkaStreams, Topology} 7 | import org.apache.kafka.streams.scala.ImplicitConversions._ 8 | import org.apache.kafka.streams.scala._ 9 | import org.apache.kafka.streams.scala.kstream._ 10 | import utils.Settings 11 | 12 | 13 | class BranchTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "stateless-branch-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[String, String] = 35 | builder.stream[String, String]("InputTopic") 36 | 37 | val predicates : List[(String, String) => Boolean] = List( 38 | (k,v) => v.startsWith("Odd"), 39 | (k,v) => v.startsWith("Even") 40 | ) 41 | 42 | //Branch (or split) a KStream based on the supplied predicates into one or more KStream instances. 43 | //Predicates are evaluated in order. A record is placed to one and only one output stream on 44 | //the first match: if the n-th predicate evaluates to true, the record is placed to n-th stream. 45 | //If no predicate matches, the the record is dropped. 46 | //Branching is useful, for example, to route records to different downstream topics. 47 | val branches : Array[KStream[String, String]] = textLines.branch(predicates:_*) 48 | branches(0).to("OddTopic") 49 | branches(1).to("EvenTopic") 50 | builder.build() 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/FilterTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream._ 9 | import org.apache.kafka.streams.{KafkaStreams, Topology} 10 | import utils.Settings 11 | 12 | 13 | class FilterTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "stateless-filter-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[String, Long] = 35 | builder.stream[String, Long]("InputTopic") 36 | 37 | //Evaluates a boolean function for each element and retains those 38 | //for which the function returns true 39 | val above5 = textLines.filter((k,v) => v > 5) 40 | above5.to("Above5OutputTopic") 41 | 42 | //Evaluates a boolean function for each element and drops those 43 | //for which the function returns true. 44 | val belowOrEqualTo5 = textLines.filterNot((k,v) => v > 5) 45 | belowOrEqualTo5.to("BelowOrEqualTo5OutputTopic") 46 | 47 | builder.build() 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/FlatMapTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream._ 9 | import org.apache.kafka.streams.{KafkaStreams, KeyValue, Topology} 10 | import utils.Settings 11 | 12 | 13 | class FlatMapTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "stateless-flatMap-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[Int, Int] = 35 | builder.stream[Int, Int]("InputTopic") 36 | 37 | //Takes one record and produces zero, one, or more records. 38 | //You can modify the record keys and values, including their types 39 | val flatMapped = textLines.flatMap((k,v) => { 40 | List( 41 | (k + 1, v + 2), 42 | (k + 3, v + 4) 43 | ) 44 | }) 45 | flatMapped.to("flatMappedOutputTopic") 46 | 47 | 48 | //Takes one record and produces zero, one, or more records, while retaining the key of the original record. 49 | //You can modify the record values and the value type. 50 | //flatMapValues is preferable to flatMap because it will not cause data re-partitioning. 51 | //However, you cannot modify the key or key type like flatMap does 52 | val flatMappedValues = textLines.flatMapValues((k,v) => { 53 | List( 54 | v + 10, 55 | v + 20 56 | ) 57 | }) 58 | flatMappedValues.to("flatMappedValuesOutputTopic") 59 | 60 | builder.build() 61 | } 62 | } 63 | 64 | 65 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/ForEachTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io.{File, PrintWriter} 4 | import java.time.Duration 5 | import java.util.Properties 6 | 7 | import org.apache.kafka.streams.scala.ImplicitConversions._ 8 | import org.apache.kafka.streams.scala._ 9 | import org.apache.kafka.streams.scala.kstream._ 10 | import org.apache.kafka.streams.{KafkaStreams, Topology} 11 | import utils.Settings 12 | 13 | 14 | class ForEachTopology(val pw: PrintWriter) extends App { 15 | 16 | import Serdes._ 17 | 18 | val props: Properties = Settings.createBasicStreamProperties( 19 | "stateless-foreach-application", "localhost:9092") 20 | 21 | run() 22 | 23 | def stop() : Unit = { 24 | pw.close() 25 | } 26 | 27 | private def run(): Unit = { 28 | 29 | 30 | val topology = createTopolgy() 31 | val streams: KafkaStreams = new KafkaStreams(topology, props) 32 | streams.start() 33 | sys.ShutdownHookThread { 34 | streams.close(Duration.ofSeconds(10)) 35 | } 36 | } 37 | 38 | def createTopolgy(): Topology = { 39 | 40 | val builder: StreamsBuilder = new StreamsBuilder 41 | val textLines: KStream[Int, Int] = 42 | builder.stream[Int, Int]("InputTopic") 43 | 44 | //Terminal operation. Performs a stateless action on each record. 45 | 46 | //You would use foreach to cause side effects based on the input data (similar to peek) 47 | //and then stop further processing of the input data (unlike peek, which is not a terminal operation). 48 | //Note on processing guarantees: Any side effects of an action (such as writing to 49 | //external systems) are not trackable by Kafka, which means they will typically not 50 | //benefit from Kafka’s processing guarantees. 51 | val flatMapped = textLines.foreach((k,v)=> { 52 | pw.write(s"Saw input value line '$v'\r\n") 53 | pw.flush() 54 | }) 55 | 56 | builder.build() 57 | } 58 | } 59 | 60 | 61 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/GroupByTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.kstream.Serialized 7 | import org.apache.kafka.streams.scala.ImplicitConversions._ 8 | import org.apache.kafka.streams.scala._ 9 | import org.apache.kafka.streams.scala.kstream._ 10 | import org.apache.kafka.streams.{KafkaStreams, Topology} 11 | import utils.Settings 12 | 13 | 14 | class GroupByTopology extends App { 15 | 16 | import Serdes._ 17 | 18 | val props: Properties = Settings.createBasicStreamProperties( 19 | "stateless-groupBy-application", "localhost:9092") 20 | 21 | run() 22 | 23 | private def run(): Unit = { 24 | val topology = createTopolgy() 25 | val streams: KafkaStreams = new KafkaStreams(topology, props) 26 | streams.start() 27 | sys.ShutdownHookThread { 28 | streams.close(Duration.ofSeconds(10)) 29 | } 30 | } 31 | 32 | def createTopolgy(): Topology = { 33 | 34 | val builder: StreamsBuilder = new StreamsBuilder 35 | val groupByKeyTextLines: KStream[Int, String] = 36 | builder.stream[Int, String]("GroupByKeyInputTopic") 37 | val groupByTextLines: KStream[Int, String] = 38 | builder.stream[Int, String]("GroupByInputTopic") 39 | 40 | 41 | //Groups the records by the existing key. (details) 42 | // 43 | //Grouping is a prerequisite for aggregating a stream or a table and ensures that data is 44 | //properly partitioned (“keyed”) for subsequent operations. 45 | // 46 | //When to set explicit SerDes: Variants of groupByKey exist to override the configured default 47 | //SerDes of your application, which you must do if the key and/or value types of the resulting 48 | //KGroupedStream do not match the configured default SerDes. 49 | 50 | //Note 51 | // 52 | //Grouping vs. Windowing: A related operation is windowing, which lets you control how to “sub-group” 53 | //the grouped records of the same key into so-called windows for stateful operations such as windowed 54 | //aggregations or windowed joins. 55 | //Causes data re-partitioning if and only if the stream was marked for re-partitioning. groupByKey is 56 | //preferable to groupBy because it re-partitions data only if the stream was already marked for 57 | //re-partitioning. However, groupByKey does not allow you to modify the key or key type like groupBy does. 58 | 59 | 60 | //NOTE : You may find you need to play with these Config values in order 61 | //to get the stateful operation to work correctly/how you want it to 62 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 63 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 64 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 65 | //By playing around with these values you should be able to find the values that work for you 66 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 67 | //cache flushing may occur 68 | groupByKeyTextLines.groupByKey 69 | .count() 70 | .mapValues(x => x.toString) 71 | .toStream 72 | .to("groupedByKeyOutputTopic") 73 | 74 | 75 | 76 | 77 | //Groups the records by a new key, which may be of a different key type. When grouping a table, you may also specify 78 | //a new value and value type. groupBy is a shorthand for selectKey(...).groupByKey(). 79 | //Grouping is a prerequisite for aggregating a stream or a table and ensures that data is properly partitioned (“keyed”) 80 | //for subsequent operations. 81 | //When to set explicit SerDes: Variants of groupBy exist to override the configured default SerDes of your application, 82 | //which you must do if the key and/or value types of the resulting KGroupedStream or KGroupedTable do not match the 83 | //configured default SerDes. 84 | // 85 | //Note 86 | // 87 | //Grouping vs. Windowing: A related operation is windowing, which lets you control how to “sub-group” the grouped records 88 | //of the same key into so-called windows for stateful operations such as windowed aggregations or windowed joins. 89 | //Always causes data re-partitioning: groupBy always causes data re-partitioning. If possible use groupByKey instead, 90 | //which will re-partition data only if required. 91 | 92 | //NOTE : You may find you need to play with these Config values in order 93 | //to get the stateful operation to work correctly/how you want it to 94 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 95 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 96 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 97 | //By playing around with these values you should be able to find the values that work for you 98 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 99 | //cache flushing may occur 100 | groupByTextLines.flatMapValues(x => x.toLowerCase.split("\\W+")) 101 | .groupBy((key, word) => word) 102 | .count() 103 | .mapValues(x => x.toString) 104 | .toStream 105 | .to("groupedByOutputTopic") 106 | 107 | builder.build() 108 | } 109 | } 110 | 111 | 112 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/MapTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream._ 9 | import org.apache.kafka.streams.{KafkaStreams, Topology} 10 | import utils.Settings 11 | 12 | 13 | class MapTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "stateless-map-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[Int, Int] = 35 | builder.stream[Int, Int]("InputTopic") 36 | 37 | //Takes one record and produces one record. You can modify the record key and value, including their types. 38 | //Marks the stream for data re-partitioning: Applying a grouping or a join after map will result in re-partitioning 39 | //of the records. If possible use mapValues instead, which will not cause data re-partitioning. 40 | val mapped = textLines.map((k,v) => { 41 | (k + 1, v + 2) 42 | }) 43 | mapped.to("mappedOutputTopic") 44 | 45 | 46 | //Takes one record and produces one record, while retaining the key of the original record. 47 | //You can modify the record value and the value type. (KStream details, KTable details) 48 | //mapValues is preferable to map because it will not cause data re-partitioning. However, it 49 | //does not allow you to modify the key or key type like map does. 50 | val mappedValues = textLines.mapValues((k,v) => { 51 | v + 10 52 | }) 53 | mappedValues.to("mappedValuesOutputTopic") 54 | 55 | builder.build() 56 | } 57 | } 58 | 59 | 60 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/PeekTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io.PrintWriter 4 | import java.time.Duration 5 | import java.util.Properties 6 | 7 | import org.apache.kafka.streams.scala.ImplicitConversions._ 8 | import org.apache.kafka.streams.scala._ 9 | import org.apache.kafka.streams.scala.kstream._ 10 | import org.apache.kafka.streams.{KafkaStreams, Topology} 11 | import utils.Settings 12 | 13 | 14 | class PeekTopology(val pw: PrintWriter) extends App { 15 | 16 | import Serdes._ 17 | 18 | val props: Properties = Settings.createBasicStreamProperties( 19 | "stateless-peek-application", "localhost:9092") 20 | 21 | run() 22 | 23 | def stop() : Unit = { 24 | pw.close() 25 | } 26 | 27 | private def run(): Unit = { 28 | 29 | 30 | val topology = createTopolgy() 31 | val streams: KafkaStreams = new KafkaStreams(topology, props) 32 | streams.start() 33 | sys.ShutdownHookThread { 34 | streams.close(Duration.ofSeconds(10)) 35 | } 36 | } 37 | 38 | def createTopolgy(): Topology = { 39 | 40 | val builder: StreamsBuilder = new StreamsBuilder 41 | val textLines: KStream[Int, Int] = 42 | builder.stream[Int, Int]("InputTopic") 43 | 44 | // Performs a stateless action on each record, and returns an unchanged stream. 45 | // 46 | // You would use peek to cause side effects based on the input data (similar to foreach) 47 | // and continue processing the input data (unlike foreach, which is a terminal operation). 48 | // Peek returns the input stream as-is; if you need to modify the input stream, use map or mapValues instead. 49 | // peek is helpful for use cases such as logging or tracking metrics or for debugging and troubleshooting. 50 | // 51 | // Note on processing guarantees: Any side effects of an action (such as writing to external systems) 52 | // are not trackable by Kafka, which means they will typically not benefit from Kafka’s processing guarantees. 53 | val peeked = textLines.peek((k,v)=> { 54 | pw.write(s"Saw input value line '$v'\r\n") 55 | pw.flush() 56 | }) 57 | peeked.to("peekedOutputTopic") 58 | 59 | 60 | builder.build() 61 | } 62 | } 63 | 64 | 65 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/SelectKeyTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.streams.scala.ImplicitConversions._ 7 | import org.apache.kafka.streams.scala._ 8 | import org.apache.kafka.streams.scala.kstream._ 9 | import org.apache.kafka.streams.{KafkaStreams, Topology} 10 | import utils.Settings 11 | 12 | 13 | class SelectKeyTopology extends App { 14 | 15 | import Serdes._ 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "stateless-selectKey-application", "localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = createTopolgy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def createTopolgy(): Topology = { 32 | 33 | val builder: StreamsBuilder = new StreamsBuilder 34 | val textLines: KStream[Int, Int] = 35 | builder.stream[Int, Int]("InputTopic") 36 | 37 | // Assigns a new key – possibly of a new key type – to each record. 38 | // Calling selectKey(mapper) is the same as calling map((key, value) -> mapper(key, value), value). 39 | // Marks the stream for data re-partitioning: Applying a grouping or a join after selectKey will 40 | // result in re-partitioning of the records. 41 | val mapped = textLines.selectKey((k,v) => { 42 | k.toString 43 | }) 44 | mapped.to("selectKeyOutputTopic") 45 | 46 | builder.build() 47 | } 48 | } 49 | 50 | 51 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/stateless/transformations/ThroughCustomPartitionerTopology.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.time.Duration 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization.Serdes 7 | import org.apache.kafka.streams.processor.StreamPartitioner 8 | import org.apache.kafka.streams.scala.ImplicitConversions._ 9 | import org.apache.kafka.streams.scala._ 10 | import org.apache.kafka.streams.scala.kstream._ 11 | import org.apache.kafka.streams.{KafkaStreams, Topology} 12 | import utils.Settings 13 | 14 | class ThroughCustomPartitionerTopology() extends App { 15 | 16 | import Serdes._ 17 | 18 | val props: Properties = Settings.createBasicStreamProperties( 19 | "stateless-custompartitioner-application", "localhost:9092") 20 | 21 | run() 22 | 23 | private def run(): Unit = { 24 | 25 | 26 | val topology = createTopolgy() 27 | val streams: KafkaStreams = new KafkaStreams(topology, props) 28 | streams.start() 29 | sys.ShutdownHookThread { 30 | streams.close(Duration.ofSeconds(10)) 31 | } 32 | } 33 | 34 | def createTopolgy(): Topology = { 35 | 36 | implicit val keySerde = Serdes.Integer 37 | implicit val valueSerde = Serdes.String 38 | 39 | val builder: StreamsBuilder = new StreamsBuilder 40 | implicit val consumed = kstream.Consumed.`with`(keySerde, valueSerde) 41 | 42 | val textLines: KStream[java.lang.Integer, String] = 43 | builder.stream[java.lang.Integer, String]("InputTopic") 44 | 45 | //should use implicit Produced with the through 46 | val repartitoned = textLines.through("InputTopic-RePartitioned") 47 | (Produced.`with`(new AlwaysOneCustomPartitioner())) 48 | 49 | val finalStream = repartitoned 50 | .mapValues((x) => s"${x} Through") 51 | 52 | finalStream.to("OutputTopic") 53 | builder.build() 54 | } 55 | } 56 | 57 | //Custom StreamPartitioner that ALWAYS just returns 1 for Partition 58 | class AlwaysOneCustomPartitioner extends StreamPartitioner[java.lang.Integer, String] { 59 | override def partition(topic: String, key: java.lang.Integer, value: String, numPartitions: Int): Integer = { 60 | //just default it to 1 61 | 1 62 | } 63 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/utils/Retry.scala: -------------------------------------------------------------------------------- 1 | package utils 2 | 3 | import scala.concurrent._ 4 | import scala.concurrent.duration._ 5 | import scala.reflect.ClassTag 6 | import scala.util.{Failure, Success, Try} 7 | import scala.reflect.runtime.universe._ 8 | 9 | object Retry { 10 | 11 | @annotation.tailrec 12 | def whileSeeingExpectedException[A, T : TypeTag](backoff: Duration = 5 seconds)(f: => A): Option[A] = { 13 | 14 | Try(f) match { 15 | case Success(x) => Some(x) 16 | case Failure(e) => 17 | val want = typeToClassTag[T] 18 | val have = ClassTag(e.getClass) 19 | if (want == have) { 20 | Thread.sleep(backoff.toMillis) 21 | println(s"Retrying due to Exception of type ${want.getClass.getName}") 22 | Retry.whileSeeingExpectedException[A,T](backoff)(f) 23 | } else { 24 | println("Un expected Exception type seen, quiting") 25 | None 26 | } 27 | } 28 | } 29 | 30 | def typeToClassTag[T: TypeTag]: ClassTag[T] = { 31 | ClassTag[T]( typeTag[T].mirror.runtimeClass( typeTag[T].tpe ) ) 32 | } 33 | 34 | /** 35 | * exponential back off for retry 36 | */ 37 | def exponentialBackoff(r: Int): Duration = scala.math.pow(2, r).round * 500 milliseconds 38 | 39 | def noIgnore(t: Throwable): Boolean = false 40 | 41 | /** 42 | * retry a particular block that can fail 43 | * 44 | * @param maxRetry how many times to retry before to giveup 45 | * @param deadline how long to retry before giving up; default None 46 | * @param backoff a back-off function that returns a Duration after which to retry. default is an exponential backoff at 100 milliseconds steps 47 | * @param ignoreThrowable if you want to stop retrying on a particular exception 48 | * @param block a block of code to retry 49 | * @param ctx an execution context where to execute the block 50 | * @returns an eventual Future succeeded with the value computed or failed with one of: 51 | * `TooManyRetriesException` if there were too many retries without an exception being caught. Probably impossible if you pass decent parameters 52 | * `DeadlineExceededException` if the retry didn't succeed before the provided deadline 53 | * `TimeoutException` if you provide a deadline and the block takes too long to execute 54 | * `Throwable` the last encountered exception 55 | */ 56 | def retry[T](maxRetry: Int, 57 | deadline: Option[Deadline] = None, 58 | backoff: (Int) => Duration = exponentialBackoff, 59 | ignoreThrowable: Throwable => Boolean = noIgnore)(block: => T)(implicit ctx: ExecutionContext): Future[T] = { 60 | 61 | class TooManyRetriesException extends Exception("too many retries without exception") 62 | class DeadlineExceededException extends Exception("deadline exceded") 63 | 64 | val p = Promise[T] 65 | 66 | def recursiveRetry(retryCnt: Int, exception: Option[Throwable])(f: () => T): Option[T] = { 67 | if (maxRetry == retryCnt 68 | || deadline.isDefined && deadline.get.isOverdue) { 69 | exception match { 70 | case Some(t) => 71 | p failure t 72 | case None if deadline.isDefined && deadline.get.isOverdue => 73 | p failure (new DeadlineExceededException) 74 | case None => 75 | p failure (new TooManyRetriesException) 76 | } 77 | None 78 | } else { 79 | val success = try { 80 | val rez = if (deadline.isDefined) { 81 | Await.result(Future(f()), deadline.get.timeLeft) 82 | } else { 83 | f() 84 | } 85 | Some(rez) 86 | } catch { 87 | case t: Throwable if !ignoreThrowable(t) => 88 | blocking { 89 | val interval = backoff(retryCnt).toMillis 90 | Thread.sleep(interval) 91 | } 92 | recursiveRetry(retryCnt + 1, Some(t))(f) 93 | case t: Throwable => 94 | p failure t 95 | None 96 | } 97 | success match { 98 | case Some(v) => 99 | p success v 100 | Some(v) 101 | case None => None 102 | } 103 | } 104 | } 105 | 106 | def doBlock() = block 107 | 108 | Future { 109 | recursiveRetry(0, None)(doBlock) 110 | } 111 | 112 | p.future 113 | } 114 | 115 | } 116 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/utils/Settings.scala: -------------------------------------------------------------------------------- 1 | package utils 2 | 3 | import java.util.Properties 4 | 5 | import com.typesafe.config.ConfigFactory 6 | import org.apache.kafka.clients.producer.ProducerConfig 7 | import org.apache.kafka.streams.StreamsConfig 8 | import org.apache.kafka.streams.scala.Serdes 9 | 10 | 11 | object Settings { 12 | 13 | val config = ConfigFactory.load("streamsApplication.conf") 14 | 15 | val kafkaConfig = config.getConfig("kafka") 16 | val zooKeepers = kafkaConfig.getString("zooKeepers") 17 | val bootStrapServers = kafkaConfig.getString("bootStrapServers") 18 | val partition = kafkaConfig.getInt("partition") 19 | val restApiDefaultHostName = kafkaConfig.getString("restApiDefaultHostName") 20 | val restApiDefaultPort = kafkaConfig.getInt("restApiDefaultPort") 21 | 22 | 23 | def createBasicProducerProperties(): Properties = { 24 | val props = new Properties() 25 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers) 26 | props.put(ProducerConfig.ACKS_CONFIG, "all") 27 | props 28 | } 29 | 30 | def createRatingStreamsProperties() : Properties = { 31 | val props = new Properties() 32 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers) 33 | props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String.getClass) 34 | props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String.getClass) 35 | props.put(ProducerConfig.RETRIES_CONFIG, 0.asInstanceOf[Object]) 36 | // Records should be flushed every 10 seconds. This is less than the default 37 | // in order to keep this example interactive. 38 | props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 39 | // For illustrative purposes we disable record caches 40 | props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0.asInstanceOf[Object]) 41 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "ratings-application") 42 | props.put(StreamsConfig.CLIENT_ID_CONFIG, "ratings-application-client") 43 | props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String.getClass) 44 | props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String.getClass) 45 | props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, s"${restApiDefaultHostName}:${restApiDefaultPort}") 46 | props.put(StreamsConfig.STATE_DIR_CONFIG, s"C:\\data\\kafka-streams".asInstanceOf[Object]) 47 | props 48 | } 49 | 50 | private def createBasicStreamProperties() : Properties = { 51 | val props = new Properties() 52 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers) 53 | props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String.getClass) 54 | props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String.getClass) 55 | 56 | // Records should be flushed every 10 seconds. This is less than the default 57 | // in order to keep this example interactive. 58 | props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 59 | // For illustrative purposes we disable record caches 60 | props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0.asInstanceOf[Object]) 61 | props.put(StreamsConfig.STATE_DIR_CONFIG, s"C:\\data\\kafka-streams".asInstanceOf[Object]) 62 | props 63 | } 64 | 65 | def createBasicStreamProperties(applicationId: String, specificBootStrapServers: String) : Properties = { 66 | 67 | val props = new Properties() 68 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId) 69 | props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:8080") 70 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, if (specificBootStrapServers.isEmpty) bootStrapServers else specificBootStrapServers) 71 | props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String.getClass) 72 | props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String.getClass) 73 | // Records should be flushed every 10 seconds. This is less than the default 74 | // in order to keep this example interactive. 75 | props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 76 | props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 77 | props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 78 | props.put(StreamsConfig.STATE_DIR_CONFIG, s"C:\\data\\kafka-streams".asInstanceOf[Object]) 79 | props 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/utils/StateStores.scala: -------------------------------------------------------------------------------- 1 | package utils 2 | 3 | import org.apache.kafka.streams.KafkaStreams 4 | import org.apache.kafka.streams.errors.InvalidStateStoreException 5 | import org.apache.kafka.streams.state.{QueryableStoreType, QueryableStoreTypes} 6 | 7 | import scala.concurrent.{ExecutionContext, Future} 8 | 9 | object StateStores { 10 | val RATINGS_BY_EMAIL_STORE = "ratings-by-email-store" 11 | 12 | def waitUntilStoreIsQueryable[T] 13 | ( 14 | storeName: String, 15 | queryableStoreType: QueryableStoreType[T], 16 | streams: KafkaStreams 17 | ) (implicit ec: ExecutionContext): Future[T] = { 18 | 19 | Retry.retry(5) { 20 | Thread.sleep(500) 21 | streams.store(storeName, queryableStoreType) 22 | }(ec) 23 | } 24 | 25 | 26 | private def printStoreMetaData[K, V](streams:KafkaStreams, storeName:String) : Unit = { 27 | 28 | val md = streams.allMetadata() 29 | val mdStore = streams.allMetadataForStore(storeName) 30 | 31 | val maybeStore = StateStores.waitUntilStoreIsQueryableSync( 32 | storeName, 33 | QueryableStoreTypes.keyValueStore[K,V](), 34 | streams) 35 | 36 | maybeStore match { 37 | case Some(store) => { 38 | val range = store.all 39 | val HASNEXT = range.hasNext 40 | while (range.hasNext) { 41 | val next = range.next 42 | System.out.print(s"key: ${next.key} value: ${next.value}") 43 | } 44 | } 45 | case None => { 46 | System.out.print(s"store not ready") 47 | throw new Exception("not ready") 48 | } 49 | } 50 | } 51 | 52 | @throws[InterruptedException] 53 | def waitUntilStoreIsQueryableSync[T]( 54 | storeName: String, 55 | queryableStoreType: QueryableStoreType[T], 56 | streams: KafkaStreams): Option[T] = { 57 | while (true) { 58 | try { 59 | return Some(streams.store(storeName, queryableStoreType)) 60 | } 61 | catch { 62 | case ignored: InvalidStateStoreException => 63 | val state = streams.state 64 | // store not yet ready for querying 65 | Thread.sleep(100) 66 | } 67 | } 68 | None 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/windowing/HoppingTimeWordCountTopology.scala: -------------------------------------------------------------------------------- 1 | package windowing 2 | 3 | import java.time.Duration 4 | import java.util 5 | import java.util.Properties 6 | import org.apache.kafka.common.serialization.Serdes 7 | import org.apache.kafka.streams.kstream.TimeWindows 8 | import org.apache.kafka.streams.scala.ImplicitConversions._ 9 | import org.apache.kafka.streams.scala._ 10 | import org.apache.kafka.streams.scala.kstream._ 11 | import org.apache.kafka.streams.{KafkaStreams, Topology} 12 | import utils.Settings 13 | 14 | 15 | class HoppingTimeWordCountTopology extends App { 16 | 17 | val props: Properties = Settings.createBasicStreamProperties( 18 | "hopping-time-window-wordcount-application","localhost:9092") 19 | 20 | run() 21 | 22 | private def run(): Unit = { 23 | val topology = wordCountToplogy() 24 | val streams: KafkaStreams = new KafkaStreams(topology, props) 25 | streams.start() 26 | sys.ShutdownHookThread { 27 | streams.close(Duration.ofSeconds(10)) 28 | } 29 | } 30 | 31 | def wordCountToplogy() : Topology = { 32 | 33 | import org.apache.kafka.streams.state.Stores 34 | val wordCountStoreName = "wordCountStore" 35 | val wordCountStoreSupplied = Stores.inMemoryKeyValueStore(wordCountStoreName) 36 | 37 | val builder = new StreamsBuilder() 38 | val textLines: KStream[String, String] = builder.stream[String, String]("TextLinesTopic") 39 | val wordCounts = textLines.flatMapValues(x => x.toLowerCase.split("\\W+")) 40 | .groupBy((key, word) => word) 41 | .windowedBy(TimeWindows.of(Duration.ofSeconds(5).plus(Duration.ofMinutes(1)))) 42 | .count() 43 | wordCounts.toStream.to("WordsWithCountsTopic") 44 | builder.build() 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/main/scala/windowing/TumblingWordCountTopology.scala: -------------------------------------------------------------------------------- 1 | package windowing 2 | 3 | import java.time.Duration 4 | import java.util 5 | import java.util.Properties 6 | import org.apache.kafka.common.serialization.Serdes 7 | import org.apache.kafka.streams.kstream.TimeWindows 8 | import org.apache.kafka.streams.scala.ImplicitConversions._ 9 | import org.apache.kafka.streams.scala._ 10 | import org.apache.kafka.streams.scala.kstream._ 11 | import org.apache.kafka.streams.{KafkaStreams, Topology} 12 | import utils.Settings 13 | 14 | 15 | class TumblingWordCountTopology extends App { 16 | 17 | import Serdes._ 18 | 19 | val props: Properties = Settings.createBasicStreamProperties( 20 | "tumbling-window-wordcount-application","localhost:9092") 21 | 22 | run() 23 | 24 | private def run(): Unit = { 25 | val topology = wordCountToplogy() 26 | val streams: KafkaStreams = new KafkaStreams(topology, props) 27 | streams.start() 28 | sys.ShutdownHookThread { 29 | streams.close(Duration.ofSeconds(10)) 30 | } 31 | } 32 | 33 | def wordCountToplogy() : Topology = { 34 | 35 | import org.apache.kafka.streams.state.Stores 36 | val wordCountStoreName = "wordCountStore" 37 | val wordCountStoreSupplied = Stores.inMemoryKeyValueStore(wordCountStoreName) 38 | 39 | val builder = new StreamsBuilder() 40 | val textLines: KStream[String, String] = builder.stream[String, String]("TextLinesTopic") 41 | val wordCounts = textLines.flatMapValues(x => x.toLowerCase.split("\\W+")) 42 | .groupBy((key, word) => word) 43 | .windowedBy(TimeWindows.of(Duration.ofSeconds(5))) 44 | .count() 45 | wordCounts.toStream.to("WordsWithCountsTopic") 46 | builder.build() 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/StaightThoughTests.scala: -------------------------------------------------------------------------------- 1 | import org.scalatest._ 2 | import Matchers._ 3 | import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} 4 | import java.io._ 5 | import java.util.Properties 6 | 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.ConsumerRecordFactory 9 | import org.apache.kafka.streams.test.OutputVerifier 10 | import utils.Settings 11 | 12 | class StaightThoughTests 13 | extends FunSuite 14 | with BeforeAndAfter 15 | with Matchers { 16 | 17 | val props = Settings.createBasicStreamProperties("straight-through-application-tests","localhost:9092") 18 | val stringDeserializer: StringDeserializer = new StringDeserializer 19 | 20 | before { 21 | } 22 | 23 | after { 24 | } 25 | 26 | 27 | test("Should produce correct output") { 28 | 29 | //arrange 30 | val recordFactory: ConsumerRecordFactory[String, String] = 31 | new ConsumerRecordFactory[String, String](new StringSerializer, new StringSerializer) 32 | val straightThroughTopology = new StraightThroughTopology() 33 | val testDriver = new TopologyTestDriver(straightThroughTopology.createTopolgy(), props) 34 | 35 | //act 36 | val consumerRecord = recordFactory.create("InputTopic", "key", "this is the input", 9999L) 37 | testDriver.pipeInput(consumerRecord) 38 | 39 | //assert 40 | OutputVerifier.compareKeyValue(testDriver.readOutput("OutputTopic", stringDeserializer, stringDeserializer), "key", "this is the input") 41 | val result = testDriver.readOutput("OutputTopic", stringDeserializer, stringDeserializer) 42 | assert(result == null) 43 | cleanup(props, testDriver) 44 | } 45 | 46 | 47 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 48 | 49 | try { 50 | //there is a bug on windows which causes this line to throw exception 51 | testDriver.close 52 | } catch { 53 | case e: Exception => { 54 | delete(new File("C:\\data\\kafka-streams")) 55 | } 56 | } 57 | } 58 | 59 | def delete(file: File) { 60 | if (file.isDirectory) 61 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 62 | file.delete 63 | } 64 | 65 | 66 | //++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 67 | // https://jaceklaskowski.gitbooks.io/mastering-kafka-streams/kafka-logging.html 68 | 69 | 70 | // 71 | // 72 | // test("UsingMatcher should be cool") { 73 | // //equality examples 74 | // Array(1, 2) should equal (Array(1, 2)) 75 | // val resultInt = 3 76 | // resultInt should equal (3) // can customize equality 77 | // resultInt should === (3) // can customize equality and enforce type constraints 78 | // resultInt should be (3) // cannot customize equality, so fastest to compile 79 | // resultInt shouldEqual 3 // can customize equality, no parentheses required 80 | // resultInt shouldBe 3 // cannot customize equality, so fastest to compile, no parentheses required 81 | // 82 | // //length examples 83 | // List(1,2) should have length 2 84 | // "cat" should have length 3 85 | // 86 | // //string examples 87 | // val helloWorld = "Hello world" 88 | // helloWorld should startWith ("Hello") 89 | // helloWorld should endWith ("world") 90 | // 91 | // val sevenString ="six seven eight" 92 | // sevenString should include ("seven") 93 | // 94 | // //greater than / less than 95 | // val one = 1 96 | // val zero = 0 97 | // val seven = 7 98 | // one should be < seven 99 | // one should be > zero 100 | // one should be <= seven 101 | // one should be >= zero 102 | // 103 | // //emptiness 104 | // List() shouldBe empty 105 | // List(1,2) should not be empty 106 | // } 107 | 108 | 109 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/interactivequeries/ratings/RatingsStreamProcessingTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package interactivequeries.ratings 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import entities.Rating 7 | import interactive.queries.ratings.RatingStreamProcessingTopology 8 | import org.apache.kafka.common.serialization.{Serdes, _} 9 | import org.apache.kafka.streams.TopologyTestDriver 10 | import org.apache.kafka.streams.state.KeyValueStore 11 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 12 | import org.scalatest._ 13 | import serialization.{JSONDeserializer, JSONSerde} 14 | import utils.{Settings, StateStores} 15 | 16 | import scala.concurrent.{ExecutionContext, Future} 17 | 18 | 19 | class RatingsStreamProcessingTopologyTests 20 | extends FunSuite 21 | with BeforeAndAfter 22 | with Matchers { 23 | 24 | val props = Settings.createBasicStreamProperties("rating-processing-application", "localhost:9092") 25 | val stringDeserializer: StringDeserializer = new StringDeserializer 26 | val ratingLIstDeserializer: JSONDeserializer[List[Rating]] = new JSONDeserializer[List[Rating]] 27 | 28 | before { 29 | } 30 | 31 | after { 32 | } 33 | 34 | 35 | test("Should produce correct output") { 36 | 37 | //arrange 38 | val recordFactory: ConsumerRecordFactory[java.lang.String, Array[Byte]] = 39 | new ConsumerRecordFactory[java.lang.String, Array[Byte]](new StringSerializer, Serdes.ByteArray().serializer()) 40 | val ratingStreamProcessingTopology = new RatingStreamProcessingTopology() 41 | 42 | 43 | val jsonSerde = new JSONSerde[Rating] 44 | 45 | val rating = Rating("jarden@here.com","sacha@here.com", 1.5f) 46 | val ratingBytes = jsonSerde.serializer().serialize("", rating) 47 | 48 | 49 | //NOTE : You may find you need to play with these Config values in order 50 | //to get the stateful operation to work correctly/how you want it to 51 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 52 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 53 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 54 | //By playing around with these values you should be able to find the values that work for you 55 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 56 | //cache flushing may occur 57 | val theTopology = ratingStreamProcessingTopology.createTopolgy() 58 | val testDriver = new TopologyTestDriver(theTopology, props) 59 | 60 | //Use the custom JSONSerde[Rating] 61 | testDriver.pipeInput(recordFactory.create("rating-submit-topic", rating.toEmail, ratingBytes, 9995L)) 62 | 63 | val ratingsByEmailStore: KeyValueStore[String, List[Rating]] = testDriver.getKeyValueStore(StateStores.RATINGS_BY_EMAIL_STORE) 64 | val ratingStored = ratingsByEmailStore.get("sacha@here.com") 65 | 66 | cleanup(props, testDriver) 67 | } 68 | 69 | 70 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 71 | 72 | try { 73 | //there is a bug on windows which causes this line to throw exception 74 | testDriver.close 75 | } catch { 76 | case e: Exception => { 77 | delete(new File("C:\\data\\kafka-streams")) 78 | } 79 | } 80 | } 81 | 82 | def delete(file: File) { 83 | if (file.isDirectory) 84 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 85 | file.delete 86 | } 87 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/joining/InnerJoinTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package joining 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization._ 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | class InnerJoinTopologyTests 13 | extends FunSuite 14 | with BeforeAndAfter 15 | with Matchers { 16 | 17 | val props = Settings.createBasicStreamProperties("inner-join-application", "localhost:9092") 18 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 19 | 20 | before { 21 | } 22 | 23 | after { 24 | } 25 | 26 | 27 | test("Should produce correct output") { 28 | 29 | //arrange 30 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.String] = 31 | new ConsumerRecordFactory[java.lang.Integer, java.lang.String](new IntegerSerializer, new StringSerializer) 32 | val innerJoinTopology = new InnerJoinTopology() 33 | val testDriver = new TopologyTestDriver(innerJoinTopology.createTopolgy(), props) 34 | 35 | //act 36 | testDriver.pipeInput(recordFactory.create("LeftTopic", 1, null, 1900L)) 37 | testDriver.pipeInput(recordFactory.create("RightTopic", 1, null, 1901L)) 38 | 39 | testDriver.pipeInput(recordFactory.create("LeftTopic", 1, "1", 2902L)) 40 | testDriver.pipeInput(recordFactory.create("RightTopic", 1, "2",2903L )) 41 | 42 | OutputVerifier.compareValue(testDriver.readOutput("innerJoinOutput", integerDeserializer, integerDeserializer), 43 | 3.asInstanceOf[Integer]) 44 | 45 | 46 | //push these out past 2 seconds (which is what Topology Join Window duration is) 47 | Thread.sleep(2500) 48 | 49 | testDriver.pipeInput(recordFactory.create("LeftTopic", 1, "2", 5916L)) 50 | testDriver.pipeInput(recordFactory.create("RightTopic", 1, "4", 5917L)) 51 | OutputVerifier.compareValue(testDriver.readOutput("innerJoinOutput", integerDeserializer, integerDeserializer), 52 | 6.asInstanceOf[Integer]) 53 | 54 | val result1 = testDriver.readOutput("innerJoinOutput", integerDeserializer, integerDeserializer) 55 | assert(result1 == null) 56 | 57 | cleanup(props, testDriver) 58 | } 59 | 60 | 61 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 62 | 63 | try { 64 | //there is a bug on windows which causes this line to throw exception 65 | testDriver.close 66 | } catch { 67 | case e: Exception => { 68 | delete(new File("C:\\data\\kafka-streams")) 69 | } 70 | } 71 | } 72 | 73 | def delete(file: File) { 74 | if (file.isDirectory) 75 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 76 | file.delete 77 | } 78 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/joining/LeftJoinTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package joining 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization.{StringDeserializer, _} 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | class LeftJoinTopologyTests 13 | extends FunSuite 14 | with BeforeAndAfter 15 | with Matchers { 16 | 17 | val props = Settings.createBasicStreamProperties("left-join-application", "localhost:9092") 18 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 19 | val stringDeserializer: StringDeserializer = new StringDeserializer 20 | 21 | before { 22 | } 23 | 24 | after { 25 | } 26 | 27 | 28 | test("Should produce correct output") { 29 | 30 | //arrange 31 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.String] = 32 | new ConsumerRecordFactory[java.lang.Integer, java.lang.String](new IntegerSerializer, new StringSerializer) 33 | val leftJoinTopology = new LeftJoinTopology() 34 | val testDriver = new TopologyTestDriver(leftJoinTopology.createTopolgy(), props) 35 | 36 | //act 37 | testDriver.pipeInput(recordFactory.create("LeftTopic", 1, null, 1900L)) 38 | testDriver.pipeInput(recordFactory.create("RightTopic", 1, null, 1901L)) 39 | 40 | testDriver.pipeInput(recordFactory.create("LeftTopic", 1, "1", 2902L)) 41 | 42 | 43 | OutputVerifier.compareValue(testDriver.readOutput("leftJoinOutput", integerDeserializer, stringDeserializer), 44 | "Left='1', Right='null'") 45 | 46 | 47 | //push these out past 2 seconds (which is what Topology Join Window duration is) 48 | Thread.sleep(2500) 49 | 50 | testDriver.pipeInput(recordFactory.create("LeftTopic", 1, "2", 5916L)) 51 | testDriver.pipeInput(recordFactory.create("RightTopic", 1, "4", 5916L)) 52 | OutputVerifier.compareValue(testDriver.readOutput("leftJoinOutput", integerDeserializer, stringDeserializer), 53 | "Left='2', Right='4'") 54 | OutputVerifier.compareValue(testDriver.readOutput("leftJoinOutput", integerDeserializer, stringDeserializer), 55 | "Left='2', Right='4'") 56 | val result1 = testDriver.readOutput("leftJoinOutput", integerDeserializer, stringDeserializer) 57 | assert(result1 == null) 58 | 59 | cleanup(props, testDriver) 60 | } 61 | 62 | 63 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 64 | 65 | try { 66 | //there is a bug on windows which causes this line to throw exception 67 | testDriver.close 68 | } catch { 69 | case e: Exception => { 70 | delete(new File("C:\\data\\kafka-streams")) 71 | } 72 | } 73 | } 74 | 75 | def delete(file: File) { 76 | if (file.isDirectory) 77 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 78 | file.delete 79 | } 80 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/joining/OuterJoinTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package joining 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization.{StringDeserializer, _} 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | class OuterJoinTopologyTests 13 | extends FunSuite 14 | with BeforeAndAfter 15 | with Matchers { 16 | 17 | val props = Settings.createBasicStreamProperties("outer-join-application", "localhost:9092") 18 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 19 | val stringDeserializer: StringDeserializer = new StringDeserializer 20 | 21 | before { 22 | } 23 | 24 | after { 25 | } 26 | 27 | 28 | test("Should produce correct output") { 29 | 30 | //arrange 31 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.String] = 32 | new ConsumerRecordFactory[java.lang.Integer, java.lang.String](new IntegerSerializer, new StringSerializer) 33 | val outerJoinTopology = new OuterJoinTopology() 34 | val testDriver = new TopologyTestDriver(outerJoinTopology.createTopolgy(), props) 35 | 36 | //act 37 | testDriver.pipeInput(recordFactory.create("LeftTopic", 1, null, 1900L)) 38 | testDriver.pipeInput(recordFactory.create("RightTopic", 1, null, 1901L)) 39 | 40 | testDriver.pipeInput(recordFactory.create("LeftTopic", 1, "1", 2902L)) 41 | 42 | 43 | OutputVerifier.compareValue(testDriver.readOutput("outerJoinOutput", integerDeserializer, stringDeserializer), 44 | "Left='1', Right='null'") 45 | 46 | 47 | //push these out past 2 seconds (which is what Topology Join Window duration is) 48 | Thread.sleep(2500) 49 | 50 | testDriver.pipeInput(recordFactory.create("RightTopic", 1, "4", 5916L)) 51 | OutputVerifier.compareValue(testDriver.readOutput("outerJoinOutput", integerDeserializer, stringDeserializer), 52 | "Left='null', Right='4'") 53 | val result1 = testDriver.readOutput("outerJoinOutput", integerDeserializer, stringDeserializer) 54 | assert(result1 == null) 55 | 56 | cleanup(props, testDriver) 57 | } 58 | 59 | 60 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 61 | 62 | try { 63 | //there is a bug on windows which causes this line to throw exception 64 | testDriver.close 65 | } catch { 66 | case e: Exception => { 67 | delete(new File("C:\\data\\kafka-streams")) 68 | } 69 | } 70 | } 71 | 72 | def delete(file: File) { 73 | if (file.isDirectory) 74 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 75 | file.delete 76 | } 77 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/processorapi/interop/ProcessorApiProcessSupplierTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package processorapi.interop 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import entities.Contributor 7 | import org.apache.kafka.common.serialization.{Serdes, _} 8 | import org.apache.kafka.streams.TopologyTestDriver 9 | import org.apache.kafka.streams.test.ConsumerRecordFactory 10 | import org.scalatest._ 11 | import serialization.{JSONDeserializer, JSONSerde} 12 | import utils.Settings 13 | 14 | import scala.io.Source 15 | 16 | 17 | class ProcessorApiProcessSupplierTopologyTests 18 | extends FunSuite 19 | with BeforeAndAfter 20 | with Matchers { 21 | 22 | val props = Settings.createBasicStreamProperties("processor-api-process-supplier-application", "localhost:9092") 23 | val stringDeserializer: StringDeserializer = new StringDeserializer 24 | val contributorDeserializer: JSONDeserializer[Contributor] = new JSONDeserializer[Contributor] 25 | 26 | before { 27 | } 28 | 29 | after { 30 | } 31 | 32 | 33 | test("Should produce correct output") { 34 | 35 | //arrange 36 | 37 | val path ="c:\\temp\\kafka-streams-process-supplier.txt" 38 | new File(path).delete() 39 | val pw = new PrintWriter(new File(path)) 40 | 41 | val recordFactory: ConsumerRecordFactory[java.lang.String, Array[Byte]] = 42 | new ConsumerRecordFactory[java.lang.String, Array[Byte]](new StringSerializer, Serdes.ByteArray().serializer()) 43 | val processorApiProcessSupplierTopology = new ProcessorApiProcessSupplierTopology(pw) 44 | val jsonSerde = new JSONSerde[Contributor] 45 | 46 | //NOTE : You may find you need to play with these Config values in order 47 | //to get the stateful operation to work correctly/how you want it to 48 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 49 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 50 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 51 | //By playing around with these values you should be able to find the values that work for you 52 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 53 | //cache flushing may occur 54 | val testDriver = new TopologyTestDriver(processorApiProcessSupplierTopology.createTopolgy(), props) 55 | 56 | var a = 0; 57 | 58 | // for loop execution with a range 59 | for( a <- 0 to 4) { 60 | 61 | val contributor = Contributor("sacha@here.com", 0f, System.currentTimeMillis) 62 | val contributorBytes = jsonSerde.serializer().serialize("", contributor) 63 | 64 | //Use the custom JSONSerde[Contributor] 65 | testDriver.pipeInput(recordFactory.create("ProcessorApiProcessorSupplierInputTopic", contributor.email, contributorBytes, 9995L)) 66 | } 67 | 68 | val lines = Source.fromFile(path).getLines.toList 69 | 70 | assert(lines.length == 5) 71 | assert(lines(0) == "key sacha@here.com has been seen 1 times") 72 | assert(lines(1) == "key sacha@here.com has been seen 2 times") 73 | assert(lines(2) == "key sacha@here.com has been seen 3 times") 74 | assert(lines(3) == "key sacha@here.com has been seen 4 times") 75 | assert(lines(4) == "key sacha@here.com has been seen 5 times") 76 | 77 | processorApiProcessSupplierTopology.stop() 78 | cleanup(props, testDriver) 79 | } 80 | 81 | 82 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 83 | 84 | try { 85 | //there is a bug on windows which causes this line to throw exception 86 | testDriver.close 87 | } catch { 88 | case e: Exception => { 89 | delete(new File("C:\\data\\kafka-streams")) 90 | } 91 | } 92 | } 93 | 94 | def delete(file: File) { 95 | if (file.isDirectory) 96 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 97 | file.delete 98 | } 99 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/processorapi/interop/ProcessorApiTransformValuesTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package processorapi.interop 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import entities.{Contributor, Rating} 7 | import org.apache.kafka.common.serialization.{Serdes, _} 8 | import org.apache.kafka.streams.TopologyTestDriver 9 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 10 | import org.scalatest._ 11 | import serialization.{JSONDeserializer, JSONSerde} 12 | import utils.Settings 13 | 14 | 15 | class ProcessorApiTransformValuesTopologyTests 16 | extends FunSuite 17 | with BeforeAndAfter 18 | with Matchers { 19 | 20 | val props = Settings.createBasicStreamProperties("processor-api-transform-values-application", "localhost:9092") 21 | val stringDeserializer: StringDeserializer = new StringDeserializer 22 | val contributorDeserializer: JSONDeserializer[Contributor] = new JSONDeserializer[Contributor] 23 | 24 | before { 25 | } 26 | 27 | after { 28 | } 29 | 30 | 31 | test("Should produce correct output") { 32 | 33 | //arrange 34 | val recordFactory: ConsumerRecordFactory[java.lang.String, Array[Byte]] = 35 | new ConsumerRecordFactory[java.lang.String, Array[Byte]](new StringSerializer, Serdes.ByteArray().serializer()) 36 | val processorApiTransformValuesTopology = new ProcessorApiTransformValuesTopology() 37 | val jsonSerde = new JSONSerde[Contributor] 38 | 39 | //NOTE : You may find you need to play with these Config values in order 40 | //to get the stateful operation to work correctly/how you want it to 41 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 42 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 43 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 44 | //By playing around with these values you should be able to find the values that work for you 45 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 46 | //cache flushing may occur 47 | val testDriver = new TopologyTestDriver(processorApiTransformValuesTopology.createTopolgy(), props) 48 | 49 | var a = 0; 50 | 51 | // for loop execution with a range 52 | val expectedRankings = List[Float](0,1,2,3,4) 53 | for( a <- 0 to 4) { 54 | 55 | val contributor = Contributor("sacha@here.com", 0f, System.currentTimeMillis) 56 | val contributorBytes = jsonSerde.serializer().serialize("", contributor) 57 | 58 | //Use the custom JSONSerde[Contributor] 59 | testDriver.pipeInput(recordFactory.create("ProcessorApiTransformValuesInputTopic", contributor.email, contributorBytes, 9995L)) 60 | val result = testDriver.readOutput("ProcessorApiTransformValuesOutputTopic", stringDeserializer, contributorDeserializer) 61 | val expectedRank = expectedRankings(a) 62 | assert(result.value.ranking == expectedRank) 63 | } 64 | 65 | val result1 = testDriver.readOutput("ProcessorApiTransformValuesOutputTopic", stringDeserializer, contributorDeserializer) 66 | assert(result1 == null) 67 | 68 | cleanup(props, testDriver) 69 | } 70 | 71 | 72 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 73 | 74 | try { 75 | //there is a bug on windows which causes this line to throw exception 76 | testDriver.close 77 | } catch { 78 | case e: Exception => { 79 | delete(new File("C:\\data\\kafka-streams")) 80 | } 81 | } 82 | } 83 | 84 | def delete(file: File) { 85 | if (file.isDirectory) 86 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 87 | file.delete 88 | } 89 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/serialization/CustomSerdesTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package serialization 2 | 3 | import java.io._ 4 | import java.lang 5 | import java.util.Properties 6 | 7 | import entities.Rating 8 | import org.apache.kafka.common.serialization.{LongDeserializer, _} 9 | import org.apache.kafka.streams.TopologyTestDriver 10 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 11 | import org.scalatest._ 12 | import org.apache.kafka.common.serialization.Serdes 13 | import utils.Settings 14 | 15 | 16 | class CustomSerdesTopologyTests 17 | extends FunSuite 18 | with BeforeAndAfter 19 | with Matchers { 20 | 21 | val props = Settings.createBasicStreamProperties("custom-serdes-application", "localhost:9092") 22 | val stringDeserializer: StringDeserializer = new StringDeserializer 23 | val ratingLIstDeserializer: JSONDeserializer[List[Rating]] = new JSONDeserializer[List[Rating]] 24 | 25 | before { 26 | } 27 | 28 | after { 29 | } 30 | 31 | 32 | test("Should produce correct output") { 33 | 34 | //arrange 35 | val recordFactory: ConsumerRecordFactory[java.lang.String, Array[Byte]] = 36 | new ConsumerRecordFactory[java.lang.String, Array[Byte]](new StringSerializer, Serdes.ByteArray().serializer()) 37 | val customSerdesTopology = new CustomSerdesTopology() 38 | 39 | 40 | val jsonSerde = new JSONSerde[Rating] 41 | 42 | val rating = Rating("jarden@here.com","sacha@here.com", 1.5f) 43 | val ratingBytes = jsonSerde.serializer().serialize("", rating) 44 | 45 | 46 | //NOTE : You may find you need to play with these Config values in order 47 | //to get the stateful operation to work correctly/how you want it to 48 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 49 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 50 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 51 | //By playing around with these values you should be able to find the values that work for you 52 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 53 | //cache flushing may occur 54 | val testDriver = new TopologyTestDriver(customSerdesTopology.createTopolgy(), props) 55 | 56 | //Use the custom JSONSerde[Rating] 57 | testDriver.pipeInput(recordFactory.create("CustomSerdesInputTopic", rating.toEmail, ratingBytes, 9995L)) 58 | 59 | val result = testDriver.readOutput("CustomSerdesOutputTopic", stringDeserializer, ratingLIstDeserializer) 60 | 61 | OutputVerifier.compareKeyValue(result, "sacha@here.com",List(Rating("jarden@here.com","sacha@here.com", 1.5f))) 62 | val result1 = testDriver.readOutput("CustomSerdesOutputTopic", stringDeserializer, ratingLIstDeserializer) 63 | assert(result1 == null) 64 | 65 | cleanup(props, testDriver) 66 | } 67 | 68 | 69 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 70 | 71 | try { 72 | //there is a bug on windows which causes this line to throw exception 73 | testDriver.close 74 | } catch { 75 | case e: Exception => { 76 | delete(new File("C:\\data\\kafka-streams")) 77 | } 78 | } 79 | } 80 | 81 | def delete(file: File) { 82 | if (file.isDirectory) 83 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 84 | file.delete 85 | } 86 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateful/transformations/aggregating/AggregateTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateful.transformations.aggregating 2 | 3 | import java.io._ 4 | import java.lang 5 | import java.util.Properties 6 | 7 | import org.apache.kafka.common.serialization.{LongDeserializer, _} 8 | import org.apache.kafka.streams.TopologyTestDriver 9 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 10 | import org.scalatest._ 11 | import utils.Settings 12 | 13 | class AggregateTopologyTests 14 | extends FunSuite 15 | with BeforeAndAfter 16 | with Matchers { 17 | 18 | val props = Settings.createBasicStreamProperties("stateless-aggregate-application", "localhost:9092") 19 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 20 | val longDeserializer: LongDeserializer = new LongDeserializer 21 | 22 | before { 23 | } 24 | 25 | after { 26 | } 27 | 28 | 29 | test("Should produce correct output") { 30 | 31 | //arrange 32 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.Integer] = 33 | new ConsumerRecordFactory[java.lang.Integer, java.lang.Integer](new IntegerSerializer, new IntegerSerializer) 34 | val aggregateTopology = new AggregateTopology() 35 | 36 | 37 | //NOTE : You may find you need to play with these Config values in order 38 | //to get the stateful operation to work correctly/how you want it to 39 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 40 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 41 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 42 | //By playing around with these values you should be able to find the values that work for you 43 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 44 | //cache flushing may occur 45 | val testDriver = new TopologyTestDriver(aggregateTopology.createTopolgy(), props) 46 | 47 | //Aggregate 48 | testDriver.pipeInput(recordFactory.create("AggregateKeyInputTopic", 1, 1, 9995L)) 49 | testDriver.pipeInput(recordFactory.create("AggregateKeyInputTopic", 1, 2, 9995L)) 50 | testDriver.pipeInput(recordFactory.create("AggregateKeyInputTopic", 2, 3, 9997L)) 51 | OutputVerifier.compareKeyValue(testDriver.readOutput("aggregateOutputTopic", integerDeserializer, longDeserializer), 52 | 1.asInstanceOf[Integer],new lang.Long(1)) 53 | OutputVerifier.compareKeyValue(testDriver.readOutput("aggregateOutputTopic", integerDeserializer, longDeserializer), 54 | 1.asInstanceOf[Integer],new lang.Long(3)) 55 | OutputVerifier.compareKeyValue(testDriver.readOutput("aggregateOutputTopic", integerDeserializer, longDeserializer), 56 | 2.asInstanceOf[Integer],new lang.Long(3)) 57 | val result1 = testDriver.readOutput("aggregateOutputTopic", integerDeserializer, longDeserializer) 58 | assert(result1 == null) 59 | 60 | cleanup(props, testDriver) 61 | } 62 | 63 | 64 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 65 | 66 | try { 67 | //there is a bug on windows which causes this line to throw exception 68 | testDriver.close 69 | } catch { 70 | case e: Exception => { 71 | delete(new File("C:\\data\\kafka-streams")) 72 | } 73 | } 74 | } 75 | 76 | def delete(file: File) { 77 | if (file.isDirectory) 78 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 79 | file.delete 80 | } 81 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateful/transformations/aggregating/CountTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateful.transformations.aggregating 2 | 3 | import java.io._ 4 | import java.lang 5 | import java.util.Properties 6 | 7 | import org.apache.kafka.common.serialization.{LongDeserializer, _} 8 | import org.apache.kafka.streams.TopologyTestDriver 9 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 10 | import org.scalatest._ 11 | import utils.Settings 12 | 13 | class CountTopologyTests 14 | extends FunSuite 15 | with BeforeAndAfter 16 | with Matchers { 17 | 18 | val props = Settings.createBasicStreamProperties("stateless-count-application", "localhost:9092") 19 | val stringDeserializer: StringDeserializer = new StringDeserializer 20 | val longDeserializer: LongDeserializer = new LongDeserializer 21 | 22 | before { 23 | } 24 | 25 | after { 26 | } 27 | 28 | 29 | test("Should produce correct output") { 30 | 31 | //arrange 32 | val recordFactory: ConsumerRecordFactory[java.lang.String, java.lang.String] = 33 | new ConsumerRecordFactory[java.lang.String, java.lang.String](new StringSerializer, new StringSerializer) 34 | val countTopology = new CountTopology() 35 | 36 | 37 | //NOTE : You may find you need to play with these Config values in order 38 | //to get the stateful operation to work correctly/how you want it to 39 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 40 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 41 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 42 | //By playing around with these values you should be able to find the values that work for you 43 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 44 | //cache flushing may occur 45 | val testDriver = new TopologyTestDriver(countTopology.createTopolgy(), props) 46 | 47 | //Aggregate 48 | testDriver.pipeInput(recordFactory.create("CountInputTopic", "one", "one two three one two four", 9995L)) 49 | testDriver.pipeInput(recordFactory.create("CountInputTopic", "two", "one two", 9997L)) 50 | OutputVerifier.compareKeyValue(testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, longDeserializer), 51 | "one",new lang.Long(1)) 52 | OutputVerifier.compareKeyValue(testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, longDeserializer), 53 | "two",new lang.Long(1)) 54 | OutputVerifier.compareKeyValue(testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, longDeserializer), 55 | "three",new lang.Long(1)) 56 | OutputVerifier.compareKeyValue(testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, longDeserializer), 57 | "one",new lang.Long(2)) 58 | OutputVerifier.compareKeyValue(testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, longDeserializer), 59 | "two",new lang.Long(2)) 60 | OutputVerifier.compareKeyValue(testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, longDeserializer), 61 | "four",new lang.Long(1)) 62 | 63 | OutputVerifier.compareKeyValue(testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, longDeserializer), 64 | "one",new lang.Long(3)) 65 | OutputVerifier.compareKeyValue(testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, longDeserializer), 66 | "two",new lang.Long(3)) 67 | 68 | val result1 = testDriver.readOutput("WordsWithCountsOutputTopic", stringDeserializer, stringDeserializer) 69 | assert(result1 == null) 70 | 71 | cleanup(props, testDriver) 72 | } 73 | 74 | 75 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 76 | 77 | try { 78 | //there is a bug on windows which causes this line to throw exception 79 | testDriver.close 80 | } catch { 81 | case e: Exception => { 82 | delete(new File("C:\\data\\kafka-streams")) 83 | } 84 | } 85 | } 86 | 87 | def delete(file: File) { 88 | if (file.isDirectory) 89 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 90 | file.delete 91 | } 92 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateful/transformations/aggregating/ReduceTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateful.transformations.aggregating 2 | 3 | import java.io._ 4 | import java.lang 5 | import java.util.Properties 6 | 7 | import org.apache.kafka.common.serialization.{IntegerDeserializer, LongDeserializer, _} 8 | import org.apache.kafka.streams.TopologyTestDriver 9 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 10 | import org.scalatest._ 11 | import utils.Settings 12 | 13 | class ReduceTopologyTests 14 | extends FunSuite 15 | with BeforeAndAfter 16 | with Matchers { 17 | 18 | val props = Settings.createBasicStreamProperties("stateless-reduce-application", "localhost:9092") 19 | val stringDeserializer: StringDeserializer = new StringDeserializer 20 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 21 | 22 | before { 23 | } 24 | 25 | after { 26 | } 27 | 28 | 29 | test("Should produce correct output") { 30 | 31 | //arrange 32 | val recordFactory: ConsumerRecordFactory[java.lang.String, java.lang.String] = 33 | new ConsumerRecordFactory[java.lang.String, java.lang.String](new StringSerializer, new StringSerializer) 34 | val reduceTopology = new ReduceTopology() 35 | 36 | 37 | //NOTE : You may find you need to play with these Config values in order 38 | //to get the stateful operation to work correctly/how you want it to 39 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 40 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 41 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 42 | //By playing around with these values you should be able to find the values that work for you 43 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 44 | //cache flushing may occur 45 | val testDriver = new TopologyTestDriver(reduceTopology.createTopolgy(), props) 46 | 47 | //Aggregate 48 | testDriver.pipeInput(recordFactory.create("ReduceInputTopic", "alice", "E", 9995L)) 49 | testDriver.pipeInput(recordFactory.create("ReduceInputTopic", "bob", "A", 9996L)) 50 | testDriver.pipeInput(recordFactory.create("ReduceInputTopic", "charlie", "A", 9997L)) 51 | 52 | OutputVerifier.compareKeyValue(testDriver.readOutput("ReduceOutputTopic", stringDeserializer, integerDeserializer), 53 | "E",new lang.Integer(5)) 54 | OutputVerifier.compareKeyValue(testDriver.readOutput("ReduceOutputTopic", stringDeserializer, integerDeserializer), 55 | "A",new lang.Integer(3)) 56 | OutputVerifier.compareKeyValue(testDriver.readOutput("ReduceOutputTopic", stringDeserializer, integerDeserializer), 57 | "A",new lang.Integer(10)) 58 | 59 | val result1 = testDriver.readOutput("ReduceOutputTopic", stringDeserializer, integerDeserializer) 60 | assert(result1 == null) 61 | 62 | cleanup(props, testDriver) 63 | } 64 | 65 | 66 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 67 | 68 | try { 69 | //there is a bug on windows which causes this line to throw exception 70 | testDriver.close 71 | } catch { 72 | case e: Exception => { 73 | delete(new File("C:\\data\\kafka-streams")) 74 | } 75 | } 76 | } 77 | 78 | def delete(file: File) { 79 | if (file.isDirectory) 80 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 81 | file.delete 82 | } 83 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/BranchTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | class BranchTopologyTests 13 | extends FunSuite 14 | with BeforeAndAfter 15 | with Matchers { 16 | 17 | val props = Settings.createBasicStreamProperties("stateless-branch-application", "localhost:9092") 18 | val stringDeserializer: StringDeserializer = new StringDeserializer 19 | 20 | before { 21 | } 22 | 23 | after { 24 | } 25 | 26 | 27 | test("Should produce correct output") { 28 | 29 | //arrange 30 | val recordFactory: ConsumerRecordFactory[String, String] = 31 | new ConsumerRecordFactory[String, String](new StringSerializer, new StringSerializer) 32 | val branchTopology = new BranchTopology() 33 | val testDriver = new TopologyTestDriver(branchTopology.createTopolgy(), props) 34 | 35 | //act 36 | val consumerRecord1 = recordFactory.create("InputTopic", "key", "Odd line1", 9995L) 37 | testDriver.pipeInput(consumerRecord1) 38 | val consumerRecord2 = recordFactory.create("InputTopic", "key", "Odd line2", 9996L) 39 | testDriver.pipeInput(consumerRecord2) 40 | val consumerRecord3 = recordFactory.create("InputTopic", "key", "Even line1", 9997L) 41 | testDriver.pipeInput(consumerRecord3) 42 | 43 | //assert 44 | OutputVerifier.compareKeyValue(testDriver.readOutput("OddTopic", stringDeserializer, stringDeserializer), "key", "Odd line1") 45 | OutputVerifier.compareKeyValue(testDriver.readOutput("OddTopic", stringDeserializer, stringDeserializer), "key", "Odd line2") 46 | val result = testDriver.readOutput("OddTopic", stringDeserializer, stringDeserializer) 47 | assert(result == null) 48 | 49 | OutputVerifier.compareKeyValue(testDriver.readOutput("EvenTopic", stringDeserializer, stringDeserializer), "key", "Even line1") 50 | val result2 = testDriver.readOutput("EvenTopic", stringDeserializer, stringDeserializer) 51 | assert(result2 == null) 52 | 53 | cleanup(props, testDriver) 54 | } 55 | 56 | 57 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 58 | 59 | try { 60 | //there is a bug on windows which causes this line to throw exception 61 | testDriver.close 62 | } catch { 63 | case e: Exception => { 64 | delete(new File("C:\\data\\kafka-streams")) 65 | } 66 | } 67 | } 68 | 69 | def delete(file: File) { 70 | if (file.isDirectory) 71 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 72 | file.delete 73 | } 74 | 75 | 76 | //++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 77 | // https://jaceklaskowski.gitbooks.io/mastering-kafka-streams/kafka-logging.html 78 | 79 | 80 | // 81 | // 82 | // test("UsingMatcher should be cool") { 83 | // //equality examples 84 | // Array(1, 2) should equal (Array(1, 2)) 85 | // val resultInt = 3 86 | // resultInt should equal (3) // can customize equality 87 | // resultInt should === (3) // can customize equality and enforce type constraints 88 | // resultInt should be (3) // cannot customize equality, so fastest to compile 89 | // resultInt shouldEqual 3 // can customize equality, no parentheses required 90 | // resultInt shouldBe 3 // cannot customize equality, so fastest to compile, no parentheses required 91 | // 92 | // //length examples 93 | // List(1,2) should have length 2 94 | // "cat" should have length 3 95 | // 96 | // //string examples 97 | // val helloWorld = "Hello world" 98 | // helloWorld should startWith ("Hello") 99 | // helloWorld should endWith ("world") 100 | // 101 | // val sevenString ="six seven eight" 102 | // sevenString should include ("seven") 103 | // 104 | // //greater than / less than 105 | // val one = 1 106 | // val zero = 0 107 | // val seven = 7 108 | // one should be < seven 109 | // one should be > zero 110 | // one should be <= seven 111 | // one should be >= zero 112 | // 113 | // //emptiness 114 | // List() shouldBe empty 115 | // List(1,2) should not be empty 116 | // } 117 | 118 | 119 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/FilterTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.clients.producer.ProducerRecord 7 | import org.apache.kafka.common.serialization.{LongDeserializer, LongSerializer, StringDeserializer, StringSerializer} 8 | import org.apache.kafka.streams.TopologyTestDriver 9 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 10 | import org.scalatest._ 11 | import utils.Settings 12 | 13 | import scala.collection.JavaConverters._ 14 | 15 | class FilterTopologyTests 16 | extends FunSuite 17 | with BeforeAndAfter 18 | with Matchers { 19 | 20 | val props = Settings.createBasicStreamProperties("stateless-filter-application", "localhost:9092") 21 | val stringDeserializer: StringDeserializer = new StringDeserializer 22 | val longDeserializer: LongDeserializer = new LongDeserializer 23 | 24 | before { 25 | } 26 | 27 | after { 28 | } 29 | 30 | 31 | test("Should produce correct output") { 32 | 33 | //arrange 34 | val recordFactory: ConsumerRecordFactory[String, java.lang.Long] = 35 | new ConsumerRecordFactory[String, java.lang.Long](new StringSerializer, new LongSerializer) 36 | val filterTopology = new FilterTopology() 37 | val testDriver = new TopologyTestDriver(filterTopology.createTopolgy(), props) 38 | 39 | //act 40 | List(0L,6L,7L).foreach(x => { 41 | testDriver.pipeInput(recordFactory.create("InputTopic", "key", java.lang.Long.valueOf(x), 9995L + 1)) 42 | }) 43 | 44 | //assert 45 | OutputVerifier.compareValue(testDriver.readOutput("Above5OutputTopic", stringDeserializer, longDeserializer), 6L.asInstanceOf[java.lang.Long]) 46 | OutputVerifier.compareValue(testDriver.readOutput("Above5OutputTopic", stringDeserializer, longDeserializer), 7L.asInstanceOf[java.lang.Long]) 47 | val result = testDriver.readOutput("OddTopic", stringDeserializer, longDeserializer) 48 | assert(result == null) 49 | 50 | OutputVerifier.compareValue(testDriver.readOutput("BelowOrEqualTo5OutputTopic", stringDeserializer, longDeserializer), 0L.asInstanceOf[java.lang.Long]) 51 | val result2 = testDriver.readOutput("BelowOrEqualTo5OutputTopic", stringDeserializer, longDeserializer) 52 | assert(result2 == null) 53 | 54 | cleanup(props, testDriver) 55 | } 56 | 57 | 58 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 59 | 60 | try { 61 | //there is a bug on windows which causes this line to throw exception 62 | testDriver.close 63 | } catch { 64 | case e: Exception => { 65 | delete(new File("C:\\data\\kafka-streams")) 66 | } 67 | } 68 | } 69 | 70 | def delete(file: File) { 71 | if (file.isDirectory) 72 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 73 | file.delete 74 | } 75 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/FlatMapTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization._ 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | class FlatMapTopologyTests 13 | extends FunSuite 14 | with BeforeAndAfter 15 | with Matchers { 16 | 17 | val props = Settings.createBasicStreamProperties("stateless-flatMap-application", "localhost:9092") 18 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 19 | 20 | before { 21 | } 22 | 23 | after { 24 | } 25 | 26 | 27 | test("Should produce correct output") { 28 | 29 | //arrange 30 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.Integer] = 31 | new ConsumerRecordFactory[java.lang.Integer, java.lang.Integer](new IntegerSerializer, new IntegerSerializer) 32 | val flatMapTopology = new FlatMapTopology() 33 | val testDriver = new TopologyTestDriver(flatMapTopology.createTopolgy(), props) 34 | 35 | //act 36 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, 2, 9995L)) 37 | 38 | //assert 39 | 40 | // flatMap is yielding this 41 | // List( 42 | // (k + 1, v + 2), 43 | // (k + 3, v + 4) 44 | // ) 45 | OutputVerifier.compareValue(testDriver.readOutput("flatMappedOutputTopic", integerDeserializer, integerDeserializer), 46 | 4.asInstanceOf[Integer]) 47 | OutputVerifier.compareValue(testDriver.readOutput("flatMappedOutputTopic", integerDeserializer, integerDeserializer), 48 | 6.asInstanceOf[Integer]) 49 | val result1 = testDriver.readOutput("flatMappedOutputTopic", integerDeserializer, integerDeserializer) 50 | assert(result1 == null) 51 | 52 | // flatMapValues is yielding this 53 | // List( 54 | // v + 10, 55 | // v + 20 56 | // ) 57 | OutputVerifier.compareValue(testDriver.readOutput("flatMappedValuesOutputTopic", integerDeserializer, integerDeserializer), 58 | 12.asInstanceOf[Integer]) 59 | OutputVerifier.compareValue(testDriver.readOutput("flatMappedValuesOutputTopic", integerDeserializer, integerDeserializer), 60 | 22.asInstanceOf[Integer]) 61 | val result2 = testDriver.readOutput("flatMappedValuesOutputTopic", integerDeserializer, integerDeserializer) 62 | assert(result2 == null) 63 | 64 | cleanup(props, testDriver) 65 | } 66 | 67 | 68 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 69 | 70 | try { 71 | //there is a bug on windows which causes this line to throw exception 72 | testDriver.close 73 | } catch { 74 | case e: Exception => { 75 | delete(new File("C:\\data\\kafka-streams")) 76 | } 77 | } 78 | } 79 | 80 | def delete(file: File) { 81 | if (file.isDirectory) 82 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 83 | file.delete 84 | } 85 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/ForEachTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization._ 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | import scala.io.Source 13 | 14 | class ForEachTopologyTests 15 | extends FunSuite 16 | with BeforeAndAfter 17 | with Matchers { 18 | 19 | val props = Settings.createBasicStreamProperties("stateless-foreach-application", "localhost:9092") 20 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 21 | 22 | before { 23 | } 24 | 25 | after { 26 | } 27 | 28 | 29 | test("Should produce correct output") { 30 | 31 | //arrange 32 | 33 | val path = "c:\\temp\\kafka-streams-foreach.txt" 34 | new File(path).delete() 35 | val pw = new PrintWriter(new File(path)) 36 | 37 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.Integer] = 38 | new ConsumerRecordFactory[java.lang.Integer, java.lang.Integer](new IntegerSerializer, new IntegerSerializer) 39 | val forEachTopology = new ForEachTopology(pw) 40 | val testDriver = new TopologyTestDriver(forEachTopology.createTopolgy(), props) 41 | 42 | //act 43 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, 1, 9995L)) 44 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, 2, 9995L + 1)) 45 | 46 | //assert 47 | val lines = Source.fromFile(path).getLines.toList 48 | 49 | assert(lines.length == 2) 50 | assert(lines(0) == "Saw input value line '1'") 51 | assert(lines(1) == "Saw input value line '2'") 52 | 53 | 54 | forEachTopology.stop() 55 | cleanup(props, testDriver) 56 | } 57 | 58 | 59 | def cleanup(props: Properties, testDriver: TopologyTestDriver) = { 60 | 61 | try { 62 | //there is a bug on windows which causes this line to throw exception 63 | testDriver.close 64 | } catch { 65 | case e: Exception => { 66 | delete(new File("C:\\data\\kafka-streams")) 67 | } 68 | } 69 | } 70 | 71 | def delete(file: File) { 72 | if (file.isDirectory) 73 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 74 | file.delete 75 | } 76 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/GroupByTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import com.fasterxml.jackson.databind.deser.std.NumberDeserializers.BigIntegerDeserializer 7 | import org.apache.kafka.common.serialization._ 8 | import org.apache.kafka.streams.TopologyTestDriver 9 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 10 | import org.scalatest._ 11 | import utils.Settings 12 | 13 | class GroupByTopologyTests 14 | extends FunSuite 15 | with BeforeAndAfter 16 | with Matchers { 17 | 18 | val props = Settings.createBasicStreamProperties("stateless-groupBy-application", "localhost:9092") 19 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 20 | val stringDeserializer: StringDeserializer = new StringDeserializer 21 | 22 | before { 23 | } 24 | 25 | after { 26 | } 27 | 28 | 29 | test("Should produce correct output") { 30 | 31 | //arrange 32 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.String] = 33 | new ConsumerRecordFactory[java.lang.Integer, java.lang.String](new IntegerSerializer, new StringSerializer) 34 | val groupByTopology = new GroupByTopology() 35 | 36 | 37 | //NOTE : You may find you need to play with these Config values in order 38 | //to get the stateful operation to work correctly/how you want it to 39 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000.asInstanceOf[Object]) 40 | // props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10485760.asInstanceOf[Object]) 41 | // props.put(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, 50000.asInstanceOf[Object]) 42 | //By playing around with these values you should be able to find the values that work for you 43 | //WARNING : Chaning these settings may have impact on the tests, as less frequent commits/state store 44 | //cache flushing may occur 45 | val testDriver = new TopologyTestDriver(groupByTopology.createTopolgy(), props) 46 | 47 | //GroupByKey 48 | testDriver.pipeInput(recordFactory.create("GroupByKeyInputTopic", 1, "one", 9995L)) 49 | testDriver.pipeInput(recordFactory.create("GroupByKeyInputTopic", 1, "one", 9995L)) 50 | testDriver.pipeInput(recordFactory.create("GroupByKeyInputTopic", 2, "two", 9997L)) 51 | OutputVerifier.compareKeyValue(testDriver.readOutput("groupedByKeyOutputTopic", integerDeserializer, stringDeserializer), 52 | 1.asInstanceOf[Integer],"1") 53 | OutputVerifier.compareKeyValue(testDriver.readOutput("groupedByKeyOutputTopic", integerDeserializer, stringDeserializer), 54 | 1.asInstanceOf[Integer],"2") 55 | OutputVerifier.compareKeyValue(testDriver.readOutput("groupedByKeyOutputTopic", integerDeserializer, stringDeserializer), 56 | 2.asInstanceOf[Integer],"1") 57 | val result1 = testDriver.readOutput("groupedByKeyOutputTopic", integerDeserializer, stringDeserializer) 58 | assert(result1 == null) 59 | 60 | //GroupBy 61 | testDriver.pipeInput(recordFactory.create("GroupByInputTopic", 1, "one two three one two", 9998L)) 62 | OutputVerifier.compareKeyValue(testDriver.readOutput("groupedByOutputTopic", stringDeserializer, stringDeserializer), 63 | "one","1") 64 | OutputVerifier.compareKeyValue(testDriver.readOutput("groupedByOutputTopic", stringDeserializer, stringDeserializer), 65 | "two","1") 66 | OutputVerifier.compareKeyValue(testDriver.readOutput("groupedByOutputTopic", stringDeserializer, stringDeserializer), 67 | "three","1") 68 | OutputVerifier.compareKeyValue(testDriver.readOutput("groupedByOutputTopic", stringDeserializer, stringDeserializer), 69 | "one","2") 70 | OutputVerifier.compareKeyValue(testDriver.readOutput("groupedByOutputTopic", stringDeserializer, stringDeserializer), 71 | "two","2") 72 | 73 | 74 | val result2 = testDriver.readOutput("groupedByOutputTopic", stringDeserializer, stringDeserializer) 75 | assert(result2 == null) 76 | 77 | 78 | cleanup(props, testDriver) 79 | } 80 | 81 | 82 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 83 | 84 | try { 85 | //there is a bug on windows which causes this line to throw exception 86 | testDriver.close 87 | } catch { 88 | case e: Exception => { 89 | delete(new File("C:\\data\\kafka-streams")) 90 | } 91 | } 92 | } 93 | 94 | def delete(file: File) { 95 | if (file.isDirectory) 96 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 97 | file.delete 98 | } 99 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/MapTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization._ 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | class MapTopologyTests 13 | extends FunSuite 14 | with BeforeAndAfter 15 | with Matchers { 16 | 17 | val props = Settings.createBasicStreamProperties("stateless-map-application", "localhost:9092") 18 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 19 | 20 | before { 21 | } 22 | 23 | after { 24 | } 25 | 26 | 27 | test("Should produce correct output") { 28 | 29 | //arrange 30 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.Integer] = 31 | new ConsumerRecordFactory[java.lang.Integer, java.lang.Integer](new IntegerSerializer, new IntegerSerializer) 32 | val mapTopology = new MapTopology() 33 | val testDriver = new TopologyTestDriver(mapTopology.createTopolgy(), props) 34 | 35 | //act 36 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, 2, 9995L)) 37 | 38 | //assert 39 | 40 | // map is yielding this 41 | // (k + 1, v + 2) 42 | // ) 43 | OutputVerifier.compareValue(testDriver.readOutput("mappedOutputTopic", integerDeserializer, integerDeserializer), 44 | 4.asInstanceOf[Integer]) 45 | val result1 = testDriver.readOutput("mappedOutputTopic", integerDeserializer, integerDeserializer) 46 | assert(result1 == null) 47 | 48 | // mapValues is yielding this 49 | // v + 10 50 | // ) 51 | OutputVerifier.compareValue(testDriver.readOutput("mappedValuesOutputTopic", integerDeserializer, integerDeserializer), 52 | 12.asInstanceOf[Integer]) 53 | val result2 = testDriver.readOutput("mappedValuesOutputTopic", integerDeserializer, integerDeserializer) 54 | assert(result2 == null) 55 | 56 | cleanup(props, testDriver) 57 | } 58 | 59 | 60 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 61 | 62 | try { 63 | //there is a bug on windows which causes this line to throw exception 64 | testDriver.close 65 | } catch { 66 | case e: Exception => { 67 | delete(new File("C:\\data\\kafka-streams")) 68 | } 69 | } 70 | } 71 | 72 | def delete(file: File) { 73 | if (file.isDirectory) 74 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 75 | file.delete 76 | } 77 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/PeekTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization._ 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | import scala.io.Source 13 | 14 | class PeekTopologyTests 15 | extends FunSuite 16 | with BeforeAndAfter 17 | with Matchers { 18 | 19 | val props = Settings.createBasicStreamProperties("stateless-peek-application", "localhost:9092") 20 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 21 | 22 | before { 23 | } 24 | 25 | after { 26 | } 27 | 28 | 29 | test("Should produce correct output") { 30 | 31 | //arrange 32 | 33 | val path ="c:\\temp\\kafka-streams-peek.txt" 34 | new File(path).delete() 35 | val pw = new PrintWriter(new File(path)) 36 | 37 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.Integer] = 38 | new ConsumerRecordFactory[java.lang.Integer, java.lang.Integer](new IntegerSerializer, new IntegerSerializer) 39 | val peekTopology = new PeekTopology(pw) 40 | val testDriver = new TopologyTestDriver(peekTopology.createTopolgy(), props) 41 | 42 | //act 43 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, 1, 9995L)) 44 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, 2, 9995L + 1)) 45 | 46 | //assert 47 | val lines = Source.fromFile(path).getLines.toList 48 | 49 | assert(lines.length == 2) 50 | assert(lines(0) == "Saw input value line '1'") 51 | assert(lines(1) == "Saw input value line '2'") 52 | 53 | OutputVerifier.compareValue(testDriver.readOutput("peekedOutputTopic", integerDeserializer, integerDeserializer), 54 | 1.asInstanceOf[Integer]) 55 | OutputVerifier.compareValue(testDriver.readOutput("peekedOutputTopic", integerDeserializer, integerDeserializer), 56 | 2.asInstanceOf[Integer]) 57 | 58 | val result1 = testDriver.readOutput("peekedOutputTopic", integerDeserializer, integerDeserializer) 59 | assert(result1 == null) 60 | 61 | peekTopology.stop() 62 | cleanup(props, testDriver) 63 | } 64 | 65 | 66 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 67 | 68 | try { 69 | //there is a bug on windows which causes this line to throw exception 70 | testDriver.close 71 | } catch { 72 | case e: Exception => { 73 | delete(new File("C:\\data\\kafka-streams")) 74 | } 75 | } 76 | } 77 | 78 | def delete(file: File) { 79 | if (file.isDirectory) 80 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 81 | file.delete 82 | } 83 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/SelectKeyTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization.{StringDeserializer, _} 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | class SelectKeyTopologyTests 13 | extends FunSuite 14 | with BeforeAndAfter 15 | with Matchers { 16 | 17 | val props = Settings.createBasicStreamProperties("stateless-selectKey-application", "localhost:9092") 18 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 19 | val stringDeserializer: StringDeserializer = new StringDeserializer 20 | 21 | before { 22 | } 23 | 24 | after { 25 | } 26 | 27 | 28 | test("Should produce correct output") { 29 | 30 | //arrange 31 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.Integer] = 32 | new ConsumerRecordFactory[java.lang.Integer, java.lang.Integer](new IntegerSerializer, new IntegerSerializer) 33 | val selectKeyTopology = new SelectKeyTopology() 34 | val testDriver = new TopologyTestDriver(selectKeyTopology.createTopolgy(), props) 35 | 36 | //act 37 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, 2, 9995L)) 38 | 39 | //assert 40 | 41 | // selectKey is yielding this 42 | // ("1", 2) 43 | // ) 44 | OutputVerifier.compareKeyValue(testDriver.readOutput("selectKeyOutputTopic", stringDeserializer, integerDeserializer), 45 | "1",2.asInstanceOf[Integer]) 46 | val result1 = testDriver.readOutput("selectKeyOutputTopic", stringDeserializer, integerDeserializer) 47 | assert(result1 == null) 48 | 49 | cleanup(props, testDriver) 50 | } 51 | 52 | 53 | def cleanup(props:Properties, testDriver: TopologyTestDriver) = { 54 | 55 | try { 56 | //there is a bug on windows which causes this line to throw exception 57 | testDriver.close 58 | } catch { 59 | case e: Exception => { 60 | delete(new File("C:\\data\\kafka-streams")) 61 | } 62 | } 63 | } 64 | 65 | def delete(file: File) { 66 | if (file.isDirectory) 67 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 68 | file.delete 69 | } 70 | } -------------------------------------------------------------------------------- /ScalaKafkaStreamsDemo/src/test/scala/stateless/transformations/ThroughCustomPartitionerTopologyTests.scala: -------------------------------------------------------------------------------- 1 | package stateless.transformations 2 | 3 | import java.io._ 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.common.serialization._ 7 | import org.apache.kafka.streams.TopologyTestDriver 8 | import org.apache.kafka.streams.test.{ConsumerRecordFactory, OutputVerifier} 9 | import org.scalatest._ 10 | import utils.Settings 11 | 12 | import scala.io.Source 13 | 14 | class ThroughCustomPartitionerTopologyTests 15 | extends FunSuite 16 | with BeforeAndAfter 17 | with Matchers { 18 | 19 | val props = Settings.createBasicStreamProperties("stateless-custompartitioner-application", "localhost:9092") 20 | val integerDeserializer: IntegerDeserializer = new IntegerDeserializer 21 | val stringDeserializer: StringDeserializer = new StringDeserializer 22 | 23 | before { 24 | } 25 | 26 | after { 27 | } 28 | 29 | 30 | test("Should produce correct output") { 31 | 32 | //arrange 33 | val recordFactory: ConsumerRecordFactory[java.lang.Integer, java.lang.String] = 34 | new ConsumerRecordFactory[java.lang.Integer, java.lang.String](new IntegerSerializer, new StringSerializer) 35 | val throughCustomPartitionerTopology = new ThroughCustomPartitionerTopology() 36 | val testDriver = new TopologyTestDriver(throughCustomPartitionerTopology.createTopolgy(), props) 37 | 38 | //act 39 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, "1", 9995L)) 40 | testDriver.pipeInput(recordFactory.create("InputTopic", 1, "2", 9995L + 1)) 41 | 42 | //assert 43 | OutputVerifier.compareValue(testDriver.readOutput("OutputTopic", integerDeserializer, stringDeserializer), 44 | "1 Through") 45 | OutputVerifier.compareValue(testDriver.readOutput("OutputTopic", integerDeserializer, stringDeserializer), 46 | "2 Through") 47 | val result2 = testDriver.readOutput("OutputTopic", integerDeserializer, stringDeserializer) 48 | assert(result2 == null) 49 | 50 | cleanup(props, testDriver) 51 | } 52 | 53 | 54 | def cleanup(props: Properties, testDriver: TopologyTestDriver) = { 55 | 56 | try { 57 | //there is a bug on windows which causes this line to throw exception 58 | testDriver.close 59 | } catch { 60 | case e: Exception => { 61 | delete(new File("C:\\data\\kafka-streams")) 62 | } 63 | } 64 | } 65 | 66 | def delete(file: File) { 67 | if (file.isDirectory) 68 | Option(file.listFiles).map(_.toList).getOrElse(Nil).foreach(delete(_)) 69 | file.delete 70 | } 71 | } --------------------------------------------------------------------------------