|
|
@ -21,7 +21,17 @@ package org.apache.kafka.streams.scala.kstream |
|
|
|
import java.time.Duration.ofSeconds |
|
|
|
import java.time.Duration.ofSeconds |
|
|
|
import java.time.Instant |
|
|
|
import java.time.Instant |
|
|
|
|
|
|
|
|
|
|
|
import org.apache.kafka.streams.kstream.JoinWindows |
|
|
|
import org.apache.kafka.streams.KeyValue |
|
|
|
|
|
|
|
import org.apache.kafka.streams.kstream.{ |
|
|
|
|
|
|
|
JoinWindows, |
|
|
|
|
|
|
|
Transformer, |
|
|
|
|
|
|
|
TransformerSupplier, |
|
|
|
|
|
|
|
ValueTransformer, |
|
|
|
|
|
|
|
ValueTransformerSupplier, |
|
|
|
|
|
|
|
ValueTransformerWithKey, |
|
|
|
|
|
|
|
ValueTransformerWithKeySupplier |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
import org.apache.kafka.streams.processor.ProcessorContext |
|
|
|
import org.apache.kafka.streams.scala.ImplicitConversions._ |
|
|
|
import org.apache.kafka.streams.scala.ImplicitConversions._ |
|
|
|
import org.apache.kafka.streams.scala.Serdes._ |
|
|
|
import org.apache.kafka.streams.scala.Serdes._ |
|
|
|
import org.apache.kafka.streams.scala.StreamsBuilder |
|
|
|
import org.apache.kafka.streams.scala.StreamsBuilder |
|
|
@ -173,4 +183,140 @@ class KStreamTest extends FlatSpec with Matchers with TestDriver { |
|
|
|
|
|
|
|
|
|
|
|
testDriver.close() |
|
|
|
testDriver.close() |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
"transform a KStream" should "transform correctly records" in { |
|
|
|
|
|
|
|
class TestTransformer extends Transformer[String, String, KeyValue[String, String]] { |
|
|
|
|
|
|
|
override def init(context: ProcessorContext): Unit = {} |
|
|
|
|
|
|
|
override def transform(key: String, value: String): KeyValue[String, String] = |
|
|
|
|
|
|
|
new KeyValue(s"$key-transformed", s"$value-transformed") |
|
|
|
|
|
|
|
override def close(): Unit = {} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
val builder = new StreamsBuilder() |
|
|
|
|
|
|
|
val sourceTopic = "source" |
|
|
|
|
|
|
|
val sinkTopic = "sink" |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val stream = builder.stream[String, String](sourceTopic) |
|
|
|
|
|
|
|
stream |
|
|
|
|
|
|
|
.transform(new TransformerSupplier[String, String, KeyValue[String, String]] { |
|
|
|
|
|
|
|
def get(): Transformer[String, String, KeyValue[String, String]] = |
|
|
|
|
|
|
|
new TestTransformer |
|
|
|
|
|
|
|
}) |
|
|
|
|
|
|
|
.to(sinkTopic) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val now = Instant.now() |
|
|
|
|
|
|
|
val testDriver = createTestDriver(builder, now) |
|
|
|
|
|
|
|
val testInput = testDriver.createInput[String, String](sourceTopic) |
|
|
|
|
|
|
|
val testOutput = testDriver.createOutput[String, String](sinkTopic) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testInput.pipeInput("1", "value", now) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val result = testOutput.readKeyValue() |
|
|
|
|
|
|
|
result.value shouldBe "value-transformed" |
|
|
|
|
|
|
|
result.key shouldBe "1-transformed" |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testOutput.isEmpty shouldBe true |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testDriver.close() |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
"flatTransform a KStream" should "flatTransform correctly records" in { |
|
|
|
|
|
|
|
class TestTransformer extends Transformer[String, String, Iterable[KeyValue[String, String]]] { |
|
|
|
|
|
|
|
override def init(context: ProcessorContext): Unit = {} |
|
|
|
|
|
|
|
override def transform(key: String, value: String): Iterable[KeyValue[String, String]] = |
|
|
|
|
|
|
|
Array(new KeyValue(s"$key-transformed", s"$value-transformed")) |
|
|
|
|
|
|
|
override def close(): Unit = {} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
val builder = new StreamsBuilder() |
|
|
|
|
|
|
|
val sourceTopic = "source" |
|
|
|
|
|
|
|
val sinkTopic = "sink" |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val stream = builder.stream[String, String](sourceTopic) |
|
|
|
|
|
|
|
stream |
|
|
|
|
|
|
|
.flatTransform(new TransformerSupplier[String, String, Iterable[KeyValue[String, String]]] { |
|
|
|
|
|
|
|
def get(): Transformer[String, String, Iterable[KeyValue[String, String]]] = |
|
|
|
|
|
|
|
new TestTransformer |
|
|
|
|
|
|
|
}) |
|
|
|
|
|
|
|
.to(sinkTopic) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val now = Instant.now() |
|
|
|
|
|
|
|
val testDriver = createTestDriver(builder, now) |
|
|
|
|
|
|
|
val testInput = testDriver.createInput[String, String](sourceTopic) |
|
|
|
|
|
|
|
val testOutput = testDriver.createOutput[String, String](sinkTopic) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testInput.pipeInput("1", "value", now) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val result = testOutput.readKeyValue() |
|
|
|
|
|
|
|
result.value shouldBe "value-transformed" |
|
|
|
|
|
|
|
result.key shouldBe "1-transformed" |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testOutput.isEmpty shouldBe true |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testDriver.close() |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
"flatTransformValues a KStream" should "correctly flatTransform values in records" in { |
|
|
|
|
|
|
|
class TestTransformer extends ValueTransformer[String, Iterable[String]] { |
|
|
|
|
|
|
|
override def init(context: ProcessorContext): Unit = {} |
|
|
|
|
|
|
|
override def transform(value: String): Iterable[String] = |
|
|
|
|
|
|
|
Array(s"$value-transformed") |
|
|
|
|
|
|
|
override def close(): Unit = {} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
val builder = new StreamsBuilder() |
|
|
|
|
|
|
|
val sourceTopic = "source" |
|
|
|
|
|
|
|
val sinkTopic = "sink" |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val stream = builder.stream[String, String](sourceTopic) |
|
|
|
|
|
|
|
stream |
|
|
|
|
|
|
|
.flatTransformValues(new ValueTransformerSupplier[String, Iterable[String]] { |
|
|
|
|
|
|
|
def get(): ValueTransformer[String, Iterable[String]] = |
|
|
|
|
|
|
|
new TestTransformer |
|
|
|
|
|
|
|
}) |
|
|
|
|
|
|
|
.to(sinkTopic) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val now = Instant.now() |
|
|
|
|
|
|
|
val testDriver = createTestDriver(builder, now) |
|
|
|
|
|
|
|
val testInput = testDriver.createInput[String, String](sourceTopic) |
|
|
|
|
|
|
|
val testOutput = testDriver.createOutput[String, String](sinkTopic) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testInput.pipeInput("1", "value", now) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testOutput.readValue shouldBe "value-transformed" |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testOutput.isEmpty shouldBe true |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testDriver.close() |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
"flatTransformValues with key in a KStream" should "correctly flatTransformValues in records" in { |
|
|
|
|
|
|
|
class TestTransformer extends ValueTransformerWithKey[String, String, Iterable[String]] { |
|
|
|
|
|
|
|
override def init(context: ProcessorContext): Unit = {} |
|
|
|
|
|
|
|
override def transform(key: String, value: String): Iterable[String] = |
|
|
|
|
|
|
|
Array(s"$value-transformed-$key") |
|
|
|
|
|
|
|
override def close(): Unit = {} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
val builder = new StreamsBuilder() |
|
|
|
|
|
|
|
val sourceTopic = "source" |
|
|
|
|
|
|
|
val sinkTopic = "sink" |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val stream = builder.stream[String, String](sourceTopic) |
|
|
|
|
|
|
|
stream |
|
|
|
|
|
|
|
.flatTransformValues(new ValueTransformerWithKeySupplier[String, String, Iterable[String]] { |
|
|
|
|
|
|
|
def get(): ValueTransformerWithKey[String, String, Iterable[String]] = |
|
|
|
|
|
|
|
new TestTransformer |
|
|
|
|
|
|
|
}) |
|
|
|
|
|
|
|
.to(sinkTopic) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val now = Instant.now() |
|
|
|
|
|
|
|
val testDriver = createTestDriver(builder, now) |
|
|
|
|
|
|
|
val testInput = testDriver.createInput[String, String](sourceTopic) |
|
|
|
|
|
|
|
val testOutput = testDriver.createOutput[String, String](sinkTopic) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testInput.pipeInput("1", "value", now) |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testOutput.readValue shouldBe "value-transformed-1" |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testOutput.isEmpty shouldBe true |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
testDriver.close() |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|