├── .gitignore ├── .travis.yml ├── README.md ├── build.sbt ├── project ├── build.properties └── plugins.sbt ├── scalastyle-config.xml └── src └── main ├── resources └── log4j.properties └── scala └── example ├── ClickstreamJoinExample.scala ├── DeduplicationExample.scala ├── Kafka.scala ├── KryoSerde.scala └── package.scala /.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | target 3 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | sudo: false 2 | 3 | cache: 4 | directories: 5 | - $HOME/.ivy2/cache 6 | - $HOME/.sbt/boot/ 7 | 8 | language: scala 9 | 10 | scala: 11 | - 2.12.4 12 | 13 | jdk: 14 | - oraclejdk8 15 | 16 | script: 17 | - sbt ++$TRAVIS_SCALA_VERSION clean compile 18 | 19 | # Trick to avoid unnecessary cache updates 20 | - find $HOME/.sbt -name "*.lock" | xargs rm 21 | - find $HOME/.ivy2 -name "ivydata-*.properties" | xargs rm 22 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Kafka Streams DSL vs Processor API 2 | 3 | [![Build Status](https://travis-ci.org/mkuthan/example-kafkastreams.svg?branch=master)](https://travis-ci.org/mkuthan/example-kafkastreams) 4 | 5 | Features 6 | ======== 7 | 8 | * Warm-up Processor API exercise, see 9 | [DeduplicationExample](https://github.com/mkuthan/example-kafkastreams/blob/master/src/main/scala/example/DeduplicationExample.scala). 10 | * Clickstream join topology implemented using DSL and Processor API, see 11 | [ClickstreamJoinExample](https://github.com/mkuthan/example-kafkastreams/blob/master/src/main/scala/example/ClickstreamJoinExample.scala). 12 | * Processor API version is up to 10 times more efficient than DSL version. 13 | * Examples are configured with 14 | [Embedded Kafka](https://github.com/manub/scalatest-embedded-kafka) 15 | and does not require any additional setup. 16 | * Implemented with Kafka 1.0 17 | 18 | Missing Features 19 | ======== 20 | * Tests 21 | * Closing WindowStoreIterator (it is not feasible with javaIt.asScala converter) 22 | 23 | References 24 | ========== 25 | 26 | * [http://mkuthan.github.io/blog/2017/11/02/kafka-streams-dsl-vs-processor-api/](http://mkuthan.github.io/blog/2017/11/02/kafka-streams-dsl-vs-processor-api/) 27 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | 2 | // Copyright (C) 2011-2012 the original author or authors. 3 | // See the LICENCE.txt file distributed with this work for additional 4 | // information regarding copyright ownership. 5 | // 6 | // Licensed under the Apache License, Version 2.0 (the "License"); 7 | // you may not use this file except in compliance with the License. 8 | // You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, software 13 | // distributed under the License is distributed on an "AS IS" BASIS, 14 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | // See the License for the specific language governing permissions and 16 | // limitations under the License. 17 | 18 | lazy val kafkaVersion = "1.0.0" 19 | 20 | lazy val commonSettings = Seq( 21 | name := "example-kafkastreams", 22 | version := "1.0", 23 | organization := "http://mkuthan.github.io/", 24 | scalaVersion := "2.12.4" 25 | ) 26 | 27 | lazy val customScalacOptions = Seq( 28 | "-deprecation", 29 | "-encoding", "UTF-8", 30 | "-feature", 31 | "-unchecked", 32 | "-Xfatal-warnings", 33 | "-Xfuture", 34 | "-Xlint", 35 | "-Yno-adapted-args", 36 | "-Ywarn-dead-code", 37 | "-Ywarn-numeric-widen", 38 | "-Ywarn-unused-import" 39 | ) 40 | 41 | lazy val customResolvers = Seq( 42 | "Apache Staging" at "https://repository.apache.org/content/groups/staging/" 43 | ) 44 | 45 | lazy val customLibraryDependencies = Seq( 46 | "org.apache.kafka" %% "kafka" % kafkaVersion, 47 | "org.apache.kafka" % "kafka-streams" % kafkaVersion, 48 | 49 | "net.manub" %% "scalatest-embedded-kafka" % "0.16.0", 50 | 51 | "com.twitter" %% "chill" % "0.9.2", 52 | 53 | "com.typesafe.scala-logging" %% "scala-logging" % "3.5.0", 54 | "org.slf4j" % "slf4j-api" % "1.7.22", 55 | "log4j" % "log4j" % "1.2.16" 56 | ) 57 | 58 | lazy val customJavaOptions = Seq( 59 | "-Xms1024m", 60 | "-Xmx1024m", 61 | "-XX:-MaxFDLimit" 62 | ) 63 | 64 | lazy val compileScalastyle = taskKey[Unit]("compileScalastyle") 65 | lazy val testScalastyle = taskKey[Unit]("testScalastyle") 66 | 67 | lazy val root = (project in file(".")) 68 | .settings(commonSettings) 69 | .settings(scalacOptions ++= customScalacOptions) 70 | .settings(resolvers ++= customResolvers) 71 | .settings(libraryDependencies ++= customLibraryDependencies) 72 | .settings(fork in run := true) 73 | .settings(connectInput in run := true) 74 | .settings(javaOptions in run ++= customJavaOptions) 75 | .settings( 76 | scalastyleFailOnError := true, 77 | compileScalastyle := scalastyle.in(Compile).toTask("").value, 78 | (compile in Compile) := ((compile in Compile) dependsOn compileScalastyle).value, 79 | testScalastyle := scalastyle.in(Test).toTask("").value, 80 | (test in Test) := ((test in Test) dependsOn testScalastyle).value) 81 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.0.3 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2011-2012 the original author or authors. 2 | // See the LICENCE.txt file distributed with this work for additional 3 | // information regarding copyright ownership. 4 | // 5 | // Licensed under the Apache License, Version 2.0 (the "License"); 6 | // you may not use this file except in compliance with the License. 7 | // You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, software 12 | // distributed under the License is distributed on an "AS IS" BASIS, 13 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | // See the License for the specific language governing permissions and 15 | // limitations under the License. 16 | 17 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "1.0.0") 18 | -------------------------------------------------------------------------------- /scalastyle-config.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 800 5 | 6 | 7 | 8 | 9 | 10 | 120 11 | 2 12 | true 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 35 | 36 | 37 | 38 | 39 | 40 | 2 41 | 2 42 | 2 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 10 70 | true 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | LPAREN 79 | 80 | 81 | 82 | 83 | 84 | COLON, COMMA, RPAREN 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, 95 | LARROW, RARROW 96 | 97 | 98 | 99 | 100 | 101 | 102 | ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | true 121 | false 122 | 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | 131 | 132 | 133 | 134 | 135 | 136 | java,scala,others 137 | javax?\..+ 138 | scala\..+ 139 | .+ 140 | true 141 | 142 | 143 | 144 | 145 | 146 | 147 | 148 | 149 | 150 | 151 | 152 | 153 | 154 | ^[a-z][A-Za-z0-9]*$ 155 | ^$ 156 | 157 | 158 | 159 | 160 | 161 | 162 | 163 | 164 | 165 | 166 | 167 | 168 | 169 | 170 | 171 | 172 | 173 | 5 174 | ^\"\"$ 175 | 176 | 177 | 178 | 179 | 180 | false 181 | ^set.+$ 182 | 183 | 184 | 185 | 186 | 187 | 188 | 189 | 190 | 191 | 192 | 193 | 194 | 195 | 196 | 197 | 198 | 199 | 200 | 201 | 202 | 203 | 15 204 | 205 | 206 | 207 | 208 | 209 | 10 210 | 211 | 212 | 213 | 214 | 215 | 216 | 217 | 218 | 219 | 220 | 221 | 222 | 223 | 224 | 225 | 226 | 227 | 228 | 229 | 230 | 231 | 232 | 233 | 234 | 235 | 8 236 | 237 | 238 | 239 | 240 | 241 | 242 | 243 | 244 | 245 | 246 | 247 | 248 | 249 | 250 | 251 | 252 | 253 | (.*Spec$)|(.*SpecIT$) 254 | PatDefOrDcl,TypeDefOrDcl,FunDefOrDcl,TmplDef 255 | false 256 | anydoc 257 | 258 | 259 | 260 | 261 | 262 | 263 | 264 | 265 | 266 | 267 | 268 | 269 | 270 | 271 | 272 | TODO|FIXME 273 | 274 | 275 | 276 | 277 | 278 | 279 | collection\.JavaConverters\._|scala\.concurrent\.duration\._ 280 | 281 | 282 | 283 | 284 | 285 | 286 | 287 | 288 | 289 | 290 | 291 | 292 | 293 | 294 | 297 | 298 | 299 | (?m)^\s*$(\r|)\n^\s*$(\r|)\n 300 | false 301 | 302 | No double blank lines 303 | 304 | 305 | 308 | 309 | 310 | 311 | 312 | No println, use logger 313 | 314 | 315 | 316 | 317 | JavaConversions 318 | 319 | No implicits for conversions, use .asScala / .asJava methods 320 | 321 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=ERROR, stdout 2 | 3 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 4 | log4j.appender.stdout.Target=System.out 5 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 6 | log4j.appender.stdout.layout.ConversionPattern=%d{HH:mm:ss} %-5p %c{1} - %m%n 7 | 8 | log4j.logger.example=INFO 9 | log4j.logger.org.apache.zookeeper=OFF 10 | -------------------------------------------------------------------------------- /src/main/scala/example/ClickstreamJoinExample.scala: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2011-2012 the original author or authors. 2 | // See the LICENCE.txt file distributed with this work for additional 3 | // information regarding copyright ownership. 4 | // 5 | // Licensed under the Apache License, Version 2.0 (the "License"); 6 | // you may not use this file except in compliance with the License. 7 | // You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, software 12 | // distributed under the License is distributed on an "AS IS" BASIS, 13 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | // See the License for the specific language governing permissions and 15 | // limitations under the License. 16 | 17 | package example 18 | 19 | import scala.concurrent.duration._ 20 | 21 | import com.typesafe.scalalogging.LazyLogging 22 | import org.apache.kafka.streams.StreamsBuilder 23 | import org.apache.kafka.streams.Topology 24 | import org.apache.kafka.streams.kstream.JoinWindows 25 | import org.apache.kafka.streams.kstream.KStream 26 | import org.apache.kafka.streams.kstream.KeyValueMapper 27 | import org.apache.kafka.streams.kstream.Materialized 28 | import org.apache.kafka.streams.kstream.Reducer 29 | import org.apache.kafka.streams.kstream.TimeWindows 30 | import org.apache.kafka.streams.kstream.ValueJoiner 31 | import org.apache.kafka.streams.kstream.Windowed 32 | import org.apache.kafka.streams.processor.AbstractProcessor 33 | import org.apache.kafka.streams.processor.ProcessorSupplier 34 | import org.apache.kafka.streams.state.StoreBuilder 35 | import org.apache.kafka.streams.state.Stores 36 | import org.apache.kafka.streams.state.WindowStore 37 | 38 | object ClickstreamJoinExample extends LazyLogging with Kafka { 39 | 40 | import Kafka._ 41 | 42 | type ClientId = String 43 | 44 | type PvId = String 45 | 46 | type EvId = String 47 | 48 | case class ClientKey(clientId: ClientId) { 49 | override def toString: String = clientId 50 | } 51 | 52 | case class PvKey(clientId: ClientId, pvId: PvId) 53 | 54 | case class Pv(pvId: PvId, value: String) 55 | 56 | case class Ev(pvId: PvId, evId: EvId, value: String) 57 | 58 | case class EvPvKey(clientId: ClientId, pvId: PvId, evId: EvId) 59 | 60 | case class EvPv(evId: EvId, evValue: String, pvId: Option[PvId], pvValue: Option[String]) 61 | 62 | private val PvTopic = "clickstream.page_views" 63 | 64 | private val EvTopic = "clickstream.events" 65 | 66 | private val EvPvTopic = "clickstream.events_enriched" 67 | 68 | private val ClientKeySerde = KryoSerde[ClientKey]() 69 | 70 | private val PvSerde = KryoSerde[Pv]() 71 | 72 | private val EvPvKeySerde = KryoSerde[EvPvKey]() 73 | 74 | private val EvPvSerde = KryoSerde[EvPv]() 75 | 76 | private val PvWindow = 15.seconds 77 | 78 | private val EvPvWindow = 5.seconds 79 | 80 | def main(args: Array[String]): Unit = { 81 | kafkaStart() 82 | 83 | daemonThread { 84 | sleep(1.second) 85 | kafkaProduce { producer => 86 | clickstream(ClientKey("bob"), producer) 87 | } 88 | } 89 | 90 | daemonThread { 91 | sleep(2.seconds) 92 | kafkaProduce { producer => 93 | // clickstream(ClientKey("jim"), producer) 94 | } 95 | } 96 | 97 | daemonThread { 98 | kafkaConsume(EvPvTopic) { record => 99 | logger.info(s"${record.key}: ${record.value}") 100 | } 101 | } 102 | 103 | daemonThread { 104 | sleep(3.seconds) 105 | 106 | startStreams(clickstreamJoinProcessorApi()) 107 | // startStreams(clickstreamJoinDsl()) 108 | } 109 | 110 | readLine() 111 | 112 | kafkaStop() 113 | } 114 | 115 | def clickstream(clientKey: ClientKey, producer: GenericProducer): Unit = { 116 | 117 | def sendPv(pvId: PvId, pvValue: String): Unit = 118 | producer.send(PvTopic, clientKey, Pv(pvId, pvValue)) 119 | 120 | def sendEv(pvId: String, evId: EvId, evValue: String): Unit = 121 | producer.send(EvTopic, clientKey, Ev(pvId, evId, evValue)) 122 | 123 | 10 to 99 foreach { i => 124 | val pv1Id = s"${i}_1" 125 | 126 | // main page view 127 | sendPv(pv1Id, "/") 128 | 129 | // a few impression events collected almost immediately 130 | sleep(100.millis) 131 | sendEv(pv1Id, "ev0", "show header") 132 | sendEv(pv1Id, "ev1", "show ads") 133 | sendEv(pv1Id, "ev2", "show recommendation") 134 | 135 | // single duplicated event, welcome to distributed world 136 | sendEv(pv1Id, "ev1", "show ads") 137 | 138 | // client clicks on one of the offers 139 | sleep(10.seconds) 140 | sendEv(pv1Id, "ev3", "click recommendation") 141 | 142 | val pv2Id = s"${i}_2" 143 | 144 | // out of order event collected before page view from offer page 145 | sendEv(pv2Id, "ev0", "show header") 146 | sleep(100.millis) 147 | 148 | // offer page view 149 | sendPv(pv2Id, "/offer?id=1234") 150 | 151 | // an impression event collected almost immediately 152 | sleep(100.millis) 153 | sendEv(pv2Id, "ev1", "show ads") 154 | 155 | // purchase after short coffee break (but longer than PvWindow) 156 | sleep(20.seconds) 157 | sendEv(pv1Id, "ev2", "add to cart") 158 | 159 | sleep(1.minute) 160 | } 161 | } 162 | 163 | def clickstreamJoinProcessorApi(): Topology = { 164 | val pvStoreName = "pv-store" 165 | val evPvStoreName = "evpv-store" 166 | val pvWindowProcessorName = "pv-window-processor" 167 | val evJoinProcessorName = "ev-join-processor" 168 | val evPvMapProcessorName = "ev-pv-processor" 169 | 170 | val pvStore = pvStoreBuilder(pvStoreName, PvWindow) 171 | val evPvStore = evPvStoreBuilder(evPvStoreName, EvPvWindow) 172 | 173 | val pvWindowProcessor: ProcessorSupplier[ClientKey, Pv] = 174 | () => new PvWindowProcessor(pvStoreName) 175 | 176 | val evJoinProcessor: ProcessorSupplier[ClientKey, Ev] = 177 | () => new EvJoinProcessor(pvStoreName, evPvStoreName, PvWindow, EvPvWindow) 178 | 179 | val evPvMapProcessor: ProcessorSupplier[EvPvKey, EvPv] = 180 | () => new EvPvMapProcessor() 181 | 182 | new Topology() 183 | // sources 184 | .addSource(PvTopic, PvTopic) 185 | .addSource(EvTopic, EvTopic) 186 | // window for page views 187 | .addProcessor(pvWindowProcessorName, pvWindowProcessor, PvTopic) 188 | // join on (clientId + pvId + evId) and deduplicate 189 | .addProcessor(evJoinProcessorName, evJoinProcessor, EvTopic) 190 | // map key again into clientId 191 | .addProcessor(evPvMapProcessorName, evPvMapProcessor, evJoinProcessorName) 192 | // sink 193 | .addSink(EvPvTopic, EvPvTopic, evPvMapProcessorName) 194 | // state stores 195 | .addStateStore(pvStore, pvWindowProcessorName, evJoinProcessorName) 196 | .addStateStore(evPvStore, evJoinProcessorName) 197 | } 198 | 199 | def clickstreamJoinDsl(): Topology = { 200 | val builder = new StreamsBuilder() 201 | // sources 202 | val evStream: KStream[ClientKey, Ev] = 203 | builder.stream[ClientKey, Ev](EvTopic) 204 | val pvStream: KStream[ClientKey, Pv] = 205 | builder.stream[ClientKey, Pv](PvTopic) 206 | 207 | // repartition events by clientKey + pvKey 208 | val evToPvKeyMapper: KeyValueMapper[ClientKey, Ev, PvKey] = 209 | (clientKey, ev) => PvKey(clientKey.clientId, ev.pvId) 210 | 211 | val evByPvKeyStream: KStream[PvKey, Ev] = 212 | evStream.selectKey(evToPvKeyMapper) 213 | 214 | // repartition page views by clientKey + pvKey 215 | val pvToPvKeyMapper: KeyValueMapper[ClientKey, Pv, PvKey] = 216 | (clientKey, pv) => PvKey(clientKey.clientId, pv.pvId) 217 | 218 | val pvByPvKeyStream: KStream[PvKey, Pv] = 219 | pvStream.selectKey(pvToPvKeyMapper) 220 | 221 | // join 222 | val evPvJoiner: ValueJoiner[Ev, Pv, EvPv] = { (ev, pv) => 223 | if (pv == null) { 224 | EvPv(ev.evId, ev.value, None, None) 225 | } else { 226 | EvPv(ev.evId, ev.value, Some(pv.pvId), Some(pv.value)) 227 | } 228 | } 229 | 230 | val joinRetention = PvWindow.toMillis * 2 + 1 231 | val joinWindow = JoinWindows.of(PvWindow.toMillis).until(joinRetention) 232 | 233 | val evPvStream: KStream[PvKey, EvPv] = 234 | evByPvKeyStream.leftJoin(pvByPvKeyStream, evPvJoiner, joinWindow) 235 | 236 | // repartition by clientKey + pvKey + evKey 237 | val evPvToEvPvKeyMapper: KeyValueMapper[PvKey, EvPv, EvPvKey] = 238 | (pvKey, evPv) => EvPvKey(pvKey.clientId, pvKey.pvId, evPv.evId) 239 | 240 | val evPvByEvPvKeyStream: KStream[EvPvKey, EvPv] = 241 | evPvStream.selectKey(evPvToEvPvKeyMapper) 242 | 243 | // deduplicate 244 | val evPvReducer: Reducer[EvPv] = 245 | (evPv1, _) => evPv1 246 | 247 | val deduplicationRetention = EvPvWindow.toMillis * 2 + 1 248 | val deduplicationWindow = TimeWindows 249 | .of(EvPvWindow.toMillis) 250 | .until(deduplicationRetention) 251 | 252 | val deduplicatedStream: KStream[Windowed[EvPvKey], EvPv] = 253 | evPvByEvPvKeyStream 254 | .groupByKey() 255 | .windowedBy(deduplicationWindow) 256 | .reduce(evPvReducer, Materialized.as("deduplication_store")) 257 | .toStream() 258 | 259 | // map key again into client id 260 | val evPvToClientKeyMapper: KeyValueMapper[Windowed[EvPvKey], EvPv, ClientId] = 261 | (windowedEvPvKey, _) => windowedEvPvKey.key.clientId 262 | 263 | val finalStream: KStream[ClientId, EvPv] = 264 | deduplicatedStream.selectKey(evPvToClientKeyMapper) 265 | 266 | // sink 267 | finalStream.to(EvPvTopic) 268 | 269 | builder.build() 270 | } 271 | 272 | def pvStoreBuilder(storeName: String, storeWindow: FiniteDuration): StoreBuilder[WindowStore[ClientKey, Pv]] = { 273 | import scala.collection.JavaConverters._ 274 | 275 | val retention = storeWindow.toMillis 276 | val window = storeWindow.toMillis 277 | val segments = 3 278 | val retainDuplicates = true 279 | 280 | val loggingConfig = Map[String, String]() 281 | 282 | Stores 283 | .windowStoreBuilder( 284 | Stores.persistentWindowStore(storeName, retention, segments, window, retainDuplicates), 285 | ClientKeySerde, 286 | PvSerde 287 | ).withLoggingEnabled(loggingConfig.asJava) 288 | } 289 | 290 | def evPvStoreBuilder(storeName: String, storeWindow: FiniteDuration): StoreBuilder[WindowStore[EvPvKey, EvPv]] = { 291 | val retention = storeWindow.toMillis 292 | val window = storeWindow.toMillis 293 | val segments = 3 294 | val retainDuplicates = false 295 | 296 | Stores.windowStoreBuilder( 297 | Stores.persistentWindowStore(storeName, retention, segments, window, retainDuplicates), 298 | EvPvKeySerde, 299 | EvPvSerde 300 | ) 301 | } 302 | 303 | class PvWindowProcessor(pvStoreName: String) extends AbstractProcessor[ClientKey, Pv] { 304 | 305 | private lazy val pvStore: WindowStore[ClientKey, Pv] = 306 | context() 307 | .getStateStore(pvStoreName) 308 | .asInstanceOf[WindowStore[ClientKey, Pv]] 309 | 310 | override def process(key: ClientKey, value: Pv): Unit = 311 | pvStore.put(key, value) 312 | } 313 | 314 | class EvJoinProcessor( 315 | pvStoreName: String, 316 | evPvStoreName: String, 317 | joinWindow: FiniteDuration, 318 | deduplicationWindow: FiniteDuration 319 | ) extends AbstractProcessor[ClientKey, Ev] { 320 | 321 | import scala.collection.JavaConverters._ 322 | 323 | private lazy val pvStore: WindowStore[ClientKey, Pv] = 324 | context() 325 | .getStateStore(pvStoreName) 326 | .asInstanceOf[WindowStore[ClientKey, Pv]] 327 | 328 | private lazy val evPvStore: WindowStore[EvPvKey, EvPv] = 329 | context() 330 | .getStateStore(evPvStoreName) 331 | .asInstanceOf[WindowStore[EvPvKey, EvPv]] 332 | 333 | override def process(key: ClientKey, ev: Ev): Unit = { 334 | val timestamp = context().timestamp() 335 | val evPvKey = EvPvKey(key.clientId, ev.pvId, ev.evId) 336 | 337 | if (isNotDuplicate(evPvKey, timestamp, deduplicationWindow)) { 338 | val evPv = storedPvs(key, timestamp, joinWindow) 339 | .find { pv => 340 | pv.pvId == ev.pvId 341 | } 342 | .map { pv => 343 | EvPv(ev.evId, ev.value, Some(pv.pvId), Some(pv.value)) 344 | } 345 | .getOrElse { 346 | EvPv(ev.evId, ev.value, None, None) 347 | } 348 | 349 | context().forward(evPvKey, evPv) 350 | evPvStore.put(evPvKey, evPv) 351 | } 352 | } 353 | 354 | private def isNotDuplicate(evPvKey: EvPvKey, timestamp: Long, deduplicationWindow: FiniteDuration) = 355 | evPvStore 356 | .fetch(evPvKey, timestamp - deduplicationWindow.toMillis, timestamp) 357 | .asScala 358 | .isEmpty 359 | 360 | private def storedPvs(key: ClientKey, timestamp: Long, joinWindow: FiniteDuration) = 361 | pvStore 362 | .fetch(key, timestamp - joinWindow.toMillis, timestamp) 363 | .asScala 364 | .map(_.value) 365 | } 366 | 367 | class EvPvMapProcessor extends AbstractProcessor[EvPvKey, EvPv] { 368 | override def process(key: EvPvKey, value: EvPv): Unit = 369 | context().forward(ClientKey(key.clientId), value) 370 | } 371 | 372 | } 373 | -------------------------------------------------------------------------------- /src/main/scala/example/DeduplicationExample.scala: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2011-2012 the original author or authors. 2 | // See the LICENCE.txt file distributed with this work for additional 3 | // information regarding copyright ownership. 4 | // 5 | // Licensed under the Apache License, Version 2.0 (the "License"); 6 | // you may not use this file except in compliance with the License. 7 | // You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, software 12 | // distributed under the License is distributed on an "AS IS" BASIS, 13 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | // See the License for the specific language governing permissions and 15 | // limitations under the License. 16 | 17 | package example 18 | 19 | import scala.concurrent.duration._ 20 | 21 | import com.typesafe.scalalogging.LazyLogging 22 | import org.apache.kafka.common.serialization.Serdes 23 | import org.apache.kafka.streams.Topology 24 | import org.apache.kafka.streams.processor.AbstractProcessor 25 | import org.apache.kafka.streams.processor.ProcessorSupplier 26 | import org.apache.kafka.streams.state.StoreBuilder 27 | import org.apache.kafka.streams.state.Stores 28 | import org.apache.kafka.streams.state.WindowStore 29 | 30 | object DeduplicationExample extends LazyLogging with Kafka { 31 | 32 | import Kafka._ 33 | 34 | type K = String 35 | 36 | type V = String 37 | 38 | private val InputTopic = "deduplication_in" 39 | 40 | private val OutputTopic = "deduplication_out" 41 | 42 | private val DeduplicationWindow = 15.seconds 43 | 44 | def main(args: Array[String]): Unit = { 45 | kafkaStart() 46 | 47 | daemonThread { 48 | kafkaProduce { producer => 49 | duplicates(producer) 50 | } 51 | } 52 | 53 | daemonThread { 54 | kafkaConsume(OutputTopic) { record => 55 | logger.info(s"${record.key}: ${record.value}") 56 | } 57 | } 58 | 59 | daemonThread { 60 | sleep(3.seconds) // scalastyle:off 61 | 62 | startStreams(deduplicate()) 63 | } 64 | 65 | readLine() 66 | 67 | kafkaStop() 68 | } 69 | 70 | def duplicates(producer: GenericProducer): Unit = { 71 | def send(k: K, v: V) = 72 | producer.send(InputTopic, k, s"v_$v") 73 | 74 | 1 to 999 foreach { i => 75 | val key = "%03d".format(i) 76 | 77 | send(key, "00") 78 | 79 | sleep(1.second) 80 | send(key, "01") 81 | 82 | sleep(2.seconds) 83 | send(key, "02") 84 | 85 | sleep(4.seconds) 86 | send(key, "03") 87 | 88 | // duplicates 89 | sleep(10.seconds) 90 | send(key, "99") 91 | 92 | sleep(5.seconds) 93 | } 94 | } 95 | 96 | def deduplicate(): Topology = { 97 | val processorName = "deduplication-processor" 98 | val storeName = "deduplication-store" 99 | 100 | val deduplicationStore = 101 | deduplicationStoreBuilder(storeName, DeduplicationWindow) 102 | 103 | val deduplicationProcessor: ProcessorSupplier[K, V] = 104 | () => new DeduplicationProcessor(storeName, DeduplicationWindow) 105 | 106 | new Topology() 107 | .addSource(InputTopic, InputTopic) 108 | .addProcessor(processorName, deduplicationProcessor, InputTopic) 109 | .addSink(OutputTopic, OutputTopic, processorName) 110 | .addStateStore(deduplicationStore, processorName) 111 | } 112 | 113 | def deduplicationStoreBuilder(storeName: String, storeWindow: FiniteDuration): StoreBuilder[WindowStore[K, V]] = { 114 | val retention = storeWindow.toMillis 115 | val window = storeWindow.toMillis 116 | val segments = 3 117 | val retainDuplicates = false 118 | 119 | Stores.windowStoreBuilder( 120 | Stores.persistentWindowStore(storeName, retention, segments, window, retainDuplicates), 121 | Serdes.String(), 122 | Serdes.String() 123 | ) 124 | } 125 | 126 | class DeduplicationProcessor(val storeName: String, val window: FiniteDuration) extends AbstractProcessor[K, V] { 127 | 128 | import scala.collection.JavaConverters._ 129 | 130 | private lazy val store: WindowStore[K, V] = 131 | context().getStateStore(storeName).asInstanceOf[WindowStore[K, V]] 132 | 133 | override def process(key: K, value: V): Unit = { 134 | val timestamp = context().timestamp() 135 | val existingValues = store.fetch(key, timestamp - window.toMillis, timestamp).asScala 136 | 137 | if (existingValues.isEmpty) { 138 | context().forward(key, value) 139 | store.put(key, value) 140 | } 141 | } 142 | } 143 | 144 | } 145 | -------------------------------------------------------------------------------- /src/main/scala/example/Kafka.scala: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2011-2012 the original author or authors. 2 | // See the LICENCE.txt file distributed with this work for additional 3 | // information regarding copyright ownership. 4 | // 5 | // Licensed under the Apache License, Version 2.0 (the "License"); 6 | // you may not use this file except in compliance with the License. 7 | // You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, software 12 | // distributed under the License is distributed on an "AS IS" BASIS, 13 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | // See the License for the specific language governing permissions and 15 | // limitations under the License. 16 | 17 | package example 18 | 19 | import java.util.Properties 20 | 21 | import scala.concurrent.duration._ 22 | 23 | import com.typesafe.scalalogging.LazyLogging 24 | import net.manub.embeddedkafka.EmbeddedKafka 25 | import net.manub.embeddedkafka.EmbeddedKafkaConfig 26 | import org.apache.kafka.clients.consumer.ConsumerConfig 27 | import org.apache.kafka.clients.consumer.ConsumerRecord 28 | import org.apache.kafka.clients.consumer.KafkaConsumer 29 | import org.apache.kafka.clients.producer.KafkaProducer 30 | import org.apache.kafka.clients.producer.ProducerConfig 31 | import org.apache.kafka.clients.producer.ProducerRecord 32 | import org.apache.kafka.streams.KafkaStreams 33 | import org.apache.kafka.streams.StreamsConfig 34 | import org.apache.kafka.streams.Topology 35 | import org.apache.kafka.streams.processor.FailOnInvalidTimestamp 36 | 37 | object Kafka { 38 | 39 | type GenericProducer = KafkaProducer[AnyRef, AnyRef] 40 | 41 | type GenericConsumerRecord = ConsumerRecord[AnyRef, AnyRef] 42 | 43 | private val SerdeName = classOf[KryoSerde[AnyRef]].getName 44 | 45 | private val TimestampExtractorName = classOf[FailOnInvalidTimestamp].getName 46 | 47 | private val DefaultNameLeght = 10 48 | 49 | private val DefaultPollTime = 60.seconds 50 | 51 | private val DefaultCommitInterval = 5.seconds 52 | 53 | private val DefaultKafkaPort = 9092 54 | 55 | private val DefaultZkPort = 2181 56 | 57 | private val DefaultBootstrapServers = s"localhost:$DefaultKafkaPort" 58 | 59 | private lazy val ProducerProps = { 60 | val props = new Properties() 61 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, DefaultBootstrapServers) 62 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, SerdeName) 63 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, SerdeName) 64 | props 65 | } 66 | 67 | private lazy val ConsumerProps = { 68 | val props = new Properties() 69 | props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, DefaultBootstrapServers) 70 | props.put(ConsumerConfig.GROUP_ID_CONFIG, randomName(DefaultNameLeght)) 71 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, SerdeName) 72 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, SerdeName) 73 | props 74 | } 75 | 76 | private lazy val StreamApplicationId = randomName(DefaultNameLeght) 77 | 78 | private lazy val StreamProps = { 79 | val props = new Properties() 80 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, DefaultBootstrapServers) 81 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, StreamApplicationId) 82 | props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, SerdeName) 83 | props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SerdeName) 84 | props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractorName) 85 | props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, DefaultCommitInterval.toMillis.asInstanceOf[AnyRef]) 86 | props 87 | } 88 | 89 | implicit class KafkaProducerOps[K, V](val producer: KafkaProducer[K, V]) { 90 | def send(topic: String, key: K, value: V): Unit = 91 | producer.send(new ProducerRecord(topic, key, value)) 92 | } 93 | 94 | } 95 | 96 | trait Kafka extends LazyLogging { 97 | 98 | import Kafka._ 99 | 100 | def kafkaStart(): Unit = { 101 | logger.info("Starting embedded Kafka") 102 | 103 | implicit val config = EmbeddedKafkaConfig(DefaultKafkaPort, DefaultZkPort) 104 | EmbeddedKafka.start() 105 | 106 | logger.info(s"Embedded Kafka started on $DefaultKafkaPort") 107 | } 108 | 109 | def kafkaStop(): Unit = { 110 | logger.info(s"Stopping embedded Kafka on $DefaultKafkaPort") 111 | 112 | EmbeddedKafka.stop() 113 | 114 | logger.info("Embedded Kafka stopped") 115 | 116 | } 117 | 118 | def startStreams(topology: Topology): Unit = { 119 | logger.info("Starting stream") 120 | 121 | val streams = new KafkaStreams(topology, StreamProps) 122 | streams.cleanUp() 123 | streams.start() 124 | 125 | logger.info(s"Stream started '$StreamApplicationId'") 126 | } 127 | 128 | def kafkaProduce(block: GenericProducer => Unit): Unit = { 129 | val producer = new KafkaProducer[AnyRef, AnyRef](ProducerProps) 130 | block(producer) 131 | } 132 | 133 | def kafkaConsume(topicName: String)(block: GenericConsumerRecord => Unit): Unit = { 134 | import scala.collection.JavaConverters._ 135 | 136 | val consumer = new KafkaConsumer[AnyRef, AnyRef](ConsumerProps) 137 | consumer.subscribe(Seq(topicName).asJava) 138 | 139 | while (true) { 140 | val recs = consumer.poll(DefaultPollTime.toMillis) 141 | recs.iterator().asScala.foreach { record => 142 | block(record) 143 | } 144 | } 145 | } 146 | 147 | } 148 | -------------------------------------------------------------------------------- /src/main/scala/example/KryoSerde.scala: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2011-2012 the original author or authors. 2 | // See the LICENCE.txt file distributed with this work for additional 3 | // information regarding copyright ownership. 4 | // 5 | // Licensed under the Apache License, Version 2.0 (the "License"); 6 | // you may not use this file except in compliance with the License. 7 | // You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, software 12 | // distributed under the License is distributed on an "AS IS" BASIS, 13 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | // See the License for the specific language governing permissions and 15 | // limitations under the License. 16 | 17 | package example 18 | 19 | import com.twitter.chill.KryoPool 20 | import com.twitter.chill.ScalaKryoInstantiator 21 | import org.apache.kafka.common.serialization.Deserializer 22 | import org.apache.kafka.common.serialization.Serde 23 | import org.apache.kafka.common.serialization.Serializer 24 | 25 | class KryoSerde[T <: AnyRef] extends Deserializer[T] with Serializer[T] with Serde[T] { 26 | 27 | import KryoSerde._ 28 | 29 | lazy val kryo = KryoPool.withBuffer( 30 | DefaultPoolSize, 31 | new ScalaKryoInstantiator(), 32 | OutputBufferInitial, 33 | OutputBufferMax 34 | ) 35 | 36 | override def deserialize(topic: String, data: Array[Byte]): T = 37 | kryo.fromBytes(data).asInstanceOf[T] 38 | 39 | override def serialize(topic: String, data: T): Array[Byte] = 40 | kryo.toBytesWithClass(data) 41 | 42 | override def deserializer(): Deserializer[T] = this 43 | 44 | override def serializer(): Serializer[T] = this 45 | 46 | override def configure(configs: java.util.Map[String, _], isKey: Boolean): Unit = {} 47 | 48 | override def close(): Unit = {} 49 | 50 | } 51 | 52 | object KryoSerde { 53 | 54 | private val DefaultPoolSize = 10 55 | private val OutputBufferInitial = 1024 56 | private val OutputBufferMax = 10 * 1024 * 1024 57 | 58 | def apply[T <: AnyRef](): KryoSerde[T] = new KryoSerde[T]() 59 | 60 | } 61 | -------------------------------------------------------------------------------- /src/main/scala/example/package.scala: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2011-2012 the original author or authors. 2 | // See the LICENCE.txt file distributed with this work for additional 3 | // information regarding copyright ownership. 4 | // 5 | // Licensed under the Apache License, Version 2.0 (the "License"); 6 | // you may not use this file except in compliance with the License. 7 | // You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, software 12 | // distributed under the License is distributed on an "AS IS" BASIS, 13 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | // See the License for the specific language governing permissions and 15 | // limitations under the License. 16 | 17 | import scala.concurrent.duration.FiniteDuration 18 | import scala.util.Random 19 | 20 | package object example { 21 | 22 | def daemonThread(block: => Unit): Unit = { 23 | val t = new Thread(() => block) 24 | t.setDaemon(true) 25 | t.start() 26 | } 27 | 28 | def readLine(): Unit = { 29 | import scala.io.StdIn 30 | StdIn.readLine() 31 | } 32 | 33 | def randomName(length: Int): String = 34 | Random.alphanumeric.take(length).mkString 35 | 36 | def sleep(sleep: FiniteDuration): Unit = 37 | Thread.sleep(sleep.toMillis) 38 | 39 | } 40 | --------------------------------------------------------------------------------