Skip to content
This repository was archived by the owner on May 25, 2023. It is now read-only.

Commit 8f132c7

Browse files
committed
Merge branch 'develop'
2 parents 64d0688 + 8e3b56b commit 8f132c7

15 files changed

+148
-6
lines changed

Diff for: NOTICE

+3
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,3 @@
1+
Kafka Streams Scala
2+
Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
Copyright 2017-2018 Alexis Seigneurin.

Diff for: README.md

+2-2
Original file line numberDiff line numberDiff line change
@@ -16,15 +16,15 @@ The design of the library was inspired by the work started by Alexis Seigneurin
1616
`kafka-streams-scala` is published and cross-built for Scala `2.11`, and `2.12`, so you can just add the following to your build:
1717

1818
```scala
19-
val kafka_streams_scala_version = "0.2.0"
19+
val kafka_streams_scala_version = "0.2.1"
2020

2121
libraryDependencies ++= Seq("com.lightbend" %%
2222
"kafka-streams-scala" % kafka_streams_scala_version)
2323
```
2424

2525
> Note: `kafka-streams-scala` supports onwards Kafka Streams `1.0.0`.
2626
27-
The API docs for `kafka-streams-scala` is available [here](https://developer.lightbend.com/docs/api/kafka-streams-scala/0.2.0/com/lightbend/kafka/scala/streams) for Scala 2.12 and [here](https://developer.lightbend.com/docs/api/kafka-streams-scala_2.11/0.2.0/#package) for Scala 2.11.
27+
The API docs for `kafka-streams-scala` is available [here](https://developer.lightbend.com/docs/api/kafka-streams-scala/0.2.1/com/lightbend/kafka/scala/streams) for Scala 2.12 and [here](https://developer.lightbend.com/docs/api/kafka-streams-scala_2.11/0.2.1/#package) for Scala 2.11.
2828

2929
## Running the Tests
3030

Diff for: build.sbt

+1-1
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@ name := "kafka-streams-scala"
44

55
organization := "com.lightbend"
66

7-
version := "0.2.0"
7+
version := "0.2.1"
88

99
scalaVersion := Versions.Scala_2_12_Version
1010

Diff for: project/Versions.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -7,7 +7,7 @@ object Versions {
77
val CuratorVersion = "4.0.0"
88
val MinitestVersion = "2.0.0"
99
val JDKVersion = "1.8"
10-
val Scala_2_12_Version = "2.12.4"
10+
val Scala_2_12_Version = "2.12.5"
1111
val Scala_2_11_Version = "2.11.11"
1212
val Avro4sVersion = "1.8.3"
1313
val CrossScalaVersions = Seq(Scala_2_12_Version, Scala_2_11_Version )

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/DefaultSerdes.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/FunctionConversions.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/ImplicitConversions.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/KGroupedStreamS.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/KGroupedTableS.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/KStreamS.scala

+3-2
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams
@@ -137,7 +138,7 @@ class KStreamS[K, V](val inner: KStream[K, V]) {
137138

138139
def join[VT, VR](table: KTableS[K, VT],
139140
joiner: (V, VT) => VR)(implicit joined: Joined[K, V, VT]): KStreamS[K, VR] =
140-
inner.leftJoin[VT, VR](table.inner, joiner.asValueJoiner, joined)
141+
inner.join[VT, VR](table.inner, joiner.asValueJoiner, joined)
141142

142143
def join[GK, GV, RV](globalKTable: GlobalKTable[GK, GV],
143144
keyValueMapper: (K, V) => GK,
@@ -165,7 +166,7 @@ class KStreamS[K, V](val inner: KStream[K, V]) {
165166
windows: JoinWindows)(implicit joined: Joined[K, V, VO]): KStreamS[K, VR] =
166167
inner.outerJoin[VO, VR](otherStream.inner, joiner.asValueJoiner, windows, joined)
167168

168-
def merge(stream: KStreamS[K, V]): KStreamS[K, V] = inner.merge(stream)
169+
def merge(stream: KStreamS[K, V]): KStreamS[K, V] = inner.merge(stream.inner)
169170

170171
def peek(action: (K, V) => Unit): KStreamS[K, V] = {
171172
inner.peek(action(_,_))

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/KTableS.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/SessionWindowedKStreamS.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/StreamsBuilderS.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams

Diff for: src/main/scala/com/lightbend/kafka/scala/streams/TimeWindowedKStreamS.scala

+1
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
/**
22
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
* Copyright 2017-2018 Alexis Seigneurin.
34
*/
45

56
package com.lightbend.kafka.scala.streams
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,129 @@
1+
/**
2+
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
3+
*/
4+
5+
package com.lightbend.kafka.scala.streams
6+
7+
import java.util.Properties
8+
import java.util.regex.Pattern
9+
10+
import com.lightbend.kafka.scala.server.{KafkaLocalServer, MessageListener, MessageSender, RecordProcessorTrait}
11+
import minitest.TestSuite
12+
import org.apache.kafka.clients.consumer.ConsumerRecord
13+
import org.apache.kafka.common.serialization._
14+
import org.apache.kafka.streams.{KafkaStreams, KeyValue, StreamsConfig}
15+
import ImplicitConversions._
16+
import com.typesafe.scalalogging.LazyLogging
17+
18+
object KafkaStreamsMergeTest extends TestSuite[KafkaLocalServer] with WordCountMergeTestData with LazyLogging {
19+
20+
override def setup(): KafkaLocalServer = {
21+
val s = KafkaLocalServer(cleanOnStart = true, Some(localStateDir))
22+
s.start()
23+
s
24+
}
25+
26+
override def tearDown(server: KafkaLocalServer): Unit = {
27+
server.stop()
28+
}
29+
30+
test("should count words") { server =>
31+
32+
server.createTopic(inputTopic1)
33+
server.createTopic(inputTopic2)
34+
server.createTopic(outputTopic)
35+
36+
//
37+
// Step 1: Configure and start the processor topology.
38+
//
39+
import DefaultSerdes._
40+
41+
val streamsConfiguration = new Properties()
42+
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, s"wordcount-${scala.util.Random.nextInt(100)}")
43+
streamsConfiguration.put(StreamsConfig.CLIENT_ID_CONFIG, "wordcountgroup")
44+
45+
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, brokers)
46+
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, localStateDir)
47+
48+
val builder = new StreamsBuilderS()
49+
50+
val textLines1 = builder.stream[String, String](inputTopic1)
51+
val textLines2 = builder.stream[String, String](inputTopic2)
52+
53+
val textLines = textLines1.merge(textLines2)
54+
55+
val pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS)
56+
57+
val wordCounts: KTableS[String, Long] =
58+
textLines.flatMapValues(v => pattern.split(v.toLowerCase))
59+
.groupBy((k, v) => v)
60+
.count()
61+
62+
wordCounts.toStream.to(outputTopic)
63+
64+
val streams = new KafkaStreams(builder.build(), streamsConfiguration)
65+
streams.start()
66+
67+
//
68+
// Step 2: Produce some input data to the input topics.
69+
//
70+
val sender = MessageSender[String, String](brokers, classOf[StringSerializer].getName, classOf[StringSerializer].getName)
71+
val mvals1 = sender.batchWriteValue(inputTopic1, inputValues)
72+
val mvals2 = sender.batchWriteValue(inputTopic2, inputValues)
73+
74+
//
75+
// Step 3: Verify the application's output data.
76+
//
77+
val listener = MessageListener(brokers, outputTopic, "wordcountgroup",
78+
classOf[StringDeserializer].getName,
79+
classOf[LongDeserializer].getName,
80+
new RecordProcessor
81+
)
82+
83+
val l = listener.waitUntilMinKeyValueRecordsReceived(expectedWordCounts.size, 30000)
84+
85+
assertEquals(l.sortBy(_.key), expectedWordCounts.sortBy(_.key))
86+
87+
streams.close()
88+
}
89+
90+
class RecordProcessor extends RecordProcessorTrait[String, Long] {
91+
override def processRecord(record: ConsumerRecord[String, Long]): Unit = {
92+
// logger.info(s"Get Message $record")
93+
}
94+
}
95+
96+
}
97+
98+
trait WordCountMergeTestData {
99+
val inputTopic1 = s"inputTopic1.${scala.util.Random.nextInt(100)}"
100+
val inputTopic2 = s"inputTopic2.${scala.util.Random.nextInt(100)}"
101+
val outputTopic = s"outputTpic.${scala.util.Random.nextInt(100)}"
102+
val brokers = "localhost:9092"
103+
val localStateDir = "local_state_data"
104+
105+
val inputValues = List(
106+
"Hello Kafka Streams",
107+
"All streams lead to Kafka",
108+
"Join Kafka Summit",
109+
"И теперь пошли русские слова"
110+
)
111+
112+
val expectedWordCounts: List[KeyValue[String, Long]] = List(
113+
new KeyValue("hello", 2L),
114+
new KeyValue("all", 2L),
115+
new KeyValue("streams", 4L),
116+
new KeyValue("lead", 2L),
117+
new KeyValue("to", 2L),
118+
new KeyValue("join", 2L),
119+
new KeyValue("kafka", 6L),
120+
new KeyValue("summit", 2L),
121+
new KeyValue("и", 2L),
122+
new KeyValue("теперь", 2L),
123+
new KeyValue("пошли", 2L),
124+
new KeyValue("русские", 2L),
125+
new KeyValue("слова", 2L)
126+
)
127+
}
128+
129+

0 commit comments

Comments
 (0)