├── .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 | }
--------------------------------------------------------------------------------