Add km module kafka

This commit is contained in:
leewei
2023-02-14 16:27:47 +08:00
parent 229140f067
commit 0b8160a714
4039 changed files with 718112 additions and 46204 deletions

View File

@@ -0,0 +1,86 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import org.apache.kafka.streams.KeyValue
import org.apache.kafka.streams.kstream._
import scala.collection.JavaConverters._
import java.lang.{Iterable => JIterable}
@deprecated("This object is for internal use only", since = "2.1.0")
object FunctionConversions {
implicit private[scala] class ForeachActionFromFunction[K, V](val p: (K, V) => Unit) extends AnyVal {
def asForeachAction: ForeachAction[K, V] = (key, value) => p(key, value)
}
implicit class PredicateFromFunction[K, V](val p: (K, V) => Boolean) extends AnyVal {
def asPredicate: Predicate[K, V] = (key: K, value: V) => p(key, value)
}
implicit class MapperFromFunction[T, U, VR](val f: (T, U) => VR) extends AnyVal {
def asKeyValueMapper: KeyValueMapper[T, U, VR] = (key: T, value: U) => f(key, value)
def asValueJoiner: ValueJoiner[T, U, VR] = (value1: T, value2: U) => f(value1, value2)
}
implicit class KeyValueMapperFromFunction[K, V, KR, VR](val f: (K, V) => (KR, VR)) extends AnyVal {
def asKeyValueMapper: KeyValueMapper[K, V, KeyValue[KR, VR]] = (key: K, value: V) => {
val (kr, vr) = f(key, value)
KeyValue.pair(kr, vr)
}
}
implicit class ValueMapperFromFunction[V, VR](val f: V => VR) extends AnyVal {
def asValueMapper: ValueMapper[V, VR] = (value: V) => f(value)
}
implicit class FlatValueMapperFromFunction[V, VR](val f: V => Iterable[VR]) extends AnyVal {
def asValueMapper: ValueMapper[V, JIterable[VR]] = (value: V) => f(value).asJava
}
implicit class ValueMapperWithKeyFromFunction[K, V, VR](val f: (K, V) => VR) extends AnyVal {
def asValueMapperWithKey: ValueMapperWithKey[K, V, VR] = (readOnlyKey: K, value: V) => f(readOnlyKey, value)
}
implicit class FlatValueMapperWithKeyFromFunction[K, V, VR](val f: (K, V) => Iterable[VR]) extends AnyVal {
def asValueMapperWithKey: ValueMapperWithKey[K, V, JIterable[VR]] =
(readOnlyKey: K, value: V) => f(readOnlyKey, value).asJava
}
implicit class AggregatorFromFunction[K, V, VA](val f: (K, V, VA) => VA) extends AnyVal {
def asAggregator: Aggregator[K, V, VA] = (key: K, value: V, aggregate: VA) => f(key, value, aggregate)
}
implicit class MergerFromFunction[K, VR](val f: (K, VR, VR) => VR) extends AnyVal {
def asMerger: Merger[K, VR] = (aggKey: K, aggOne: VR, aggTwo: VR) => f(aggKey, aggOne, aggTwo)
}
implicit class ReducerFromFunction[V](val f: (V, V) => V) extends AnyVal {
def asReducer: Reducer[V] = (value1: V, value2: V) => f(value1, value2)
}
implicit class InitializerFromFunction[VA](val f: () => VA) extends AnyVal {
def asInitializer: Initializer[VA] = () => f()
}
implicit class TransformerSupplierFromFunction[K, V, VO](val f: () => Transformer[K, V, VO]) extends AnyVal {
def asTransformerSupplier: TransformerSupplier[K, V, VO] = () => f()
}
}

View File

@@ -0,0 +1,128 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import org.apache.kafka.streams.KeyValue
import org.apache.kafka.streams.kstream._
import scala.collection.JavaConverters._
import java.lang.{Iterable => JIterable}
import org.apache.kafka.streams.processor.ProcessorContext
/**
* Implicit classes that offer conversions of Scala function literals to
* SAM (Single Abstract Method) objects in Java. These make the Scala APIs much
* more expressive, with less boilerplate and more succinct.
*/
private[scala] object FunctionsCompatConversions {
implicit class ForeachActionFromFunction[K, V](val p: (K, V) => Unit) extends AnyVal {
def asForeachAction: ForeachAction[K, V] = (key: K, value: V) => p(key, value)
}
implicit class PredicateFromFunction[K, V](val p: (K, V) => Boolean) extends AnyVal {
def asPredicate: Predicate[K, V] = (key: K, value: V) => p(key, value)
}
implicit class MapperFromFunction[T, U, VR](val f: (T, U) => VR) extends AnyVal {
def asKeyValueMapper: KeyValueMapper[T, U, VR] = (key: T, value: U) => f(key, value)
def asValueJoiner: ValueJoiner[T, U, VR] = (value1: T, value2: U) => f(value1, value2)
}
implicit class KeyValueMapperFromFunction[K, V, KR, VR](val f: (K, V) => (KR, VR)) extends AnyVal {
def asKeyValueMapper: KeyValueMapper[K, V, KeyValue[KR, VR]] = (key: K, value: V) => {
val (kr, vr) = f(key, value)
KeyValue.pair(kr, vr)
}
}
implicit class FunctionFromFunction[V, VR](val f: V => VR) extends AnyVal {
def asJavaFunction: java.util.function.Function[V, VR] = (value: V) => f(value)
}
implicit class ValueMapperFromFunction[V, VR](val f: V => VR) extends AnyVal {
def asValueMapper: ValueMapper[V, VR] = (value: V) => f(value)
}
implicit class FlatValueMapperFromFunction[V, VR](val f: V => Iterable[VR]) extends AnyVal {
def asValueMapper: ValueMapper[V, JIterable[VR]] = (value: V) => f(value).asJava
}
implicit class ValueMapperWithKeyFromFunction[K, V, VR](val f: (K, V) => VR) extends AnyVal {
def asValueMapperWithKey: ValueMapperWithKey[K, V, VR] = (readOnlyKey: K, value: V) => f(readOnlyKey, value)
}
implicit class FlatValueMapperWithKeyFromFunction[K, V, VR](val f: (K, V) => Iterable[VR]) extends AnyVal {
def asValueMapperWithKey: ValueMapperWithKey[K, V, JIterable[VR]] =
(readOnlyKey: K, value: V) => f(readOnlyKey, value).asJava
}
implicit class AggregatorFromFunction[K, V, VA](val f: (K, V, VA) => VA) extends AnyVal {
def asAggregator: Aggregator[K, V, VA] = (key: K, value: V, aggregate: VA) => f(key, value, aggregate)
}
implicit class MergerFromFunction[K, VR](val f: (K, VR, VR) => VR) extends AnyVal {
def asMerger: Merger[K, VR] = (aggKey: K, aggOne: VR, aggTwo: VR) => f(aggKey, aggOne, aggTwo)
}
implicit class ReducerFromFunction[V](val f: (V, V) => V) extends AnyVal {
def asReducer: Reducer[V] = (value1: V, value2: V) => f(value1, value2)
}
implicit class InitializerFromFunction[VA](val f: () => VA) extends AnyVal {
def asInitializer: Initializer[VA] = () => f()
}
implicit class TransformerSupplierFromFunction[K, V, VO](val f: () => Transformer[K, V, VO]) extends AnyVal {
def asTransformerSupplier: TransformerSupplier[K, V, VO] = () => f()
}
implicit class TransformerSupplierAsJava[K, V, VO](val supplier: TransformerSupplier[K, V, Iterable[VO]])
extends AnyVal {
def asJava: TransformerSupplier[K, V, JIterable[VO]] = () => {
val innerTransformer = supplier.get()
new Transformer[K, V, JIterable[VO]] {
override def transform(key: K, value: V): JIterable[VO] = innerTransformer.transform(key, value).asJava
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
override def close(): Unit = innerTransformer.close()
}
}
}
implicit class ValueTransformerSupplierAsJava[V, VO](val supplier: ValueTransformerSupplier[V, Iterable[VO]])
extends AnyVal {
def asJava: ValueTransformerSupplier[V, JIterable[VO]] = () => {
val innerTransformer = supplier.get()
new ValueTransformer[V, JIterable[VO]] {
override def transform(value: V): JIterable[VO] = innerTransformer.transform(value).asJava
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
override def close(): Unit = innerTransformer.close()
}
}
}
implicit class ValueTransformerSupplierWithKeyAsJava[K, V, VO](
val supplier: ValueTransformerWithKeySupplier[K, V, Iterable[VO]]
) extends AnyVal {
def asJava: ValueTransformerWithKeySupplier[K, V, JIterable[VO]] = () => {
val innerTransformer = supplier.get()
new ValueTransformerWithKey[K, V, JIterable[VO]] {
override def transform(key: K, value: V): JIterable[VO] = innerTransformer.transform(key, value).asJava
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
override def close(): Unit = innerTransformer.close()
}
}
}
}

View File

@@ -0,0 +1,104 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import org.apache.kafka.common.serialization.Serde
import org.apache.kafka.streams.KeyValue
import org.apache.kafka.streams.kstream.{
KStream => KStreamJ,
KGroupedStream => KGroupedStreamJ,
TimeWindowedKStream => TimeWindowedKStreamJ,
SessionWindowedKStream => SessionWindowedKStreamJ,
CogroupedKStream => CogroupedKStreamJ,
TimeWindowedCogroupedKStream => TimeWindowedCogroupedKStreamJ,
SessionWindowedCogroupedKStream => SessionWindowedCogroupedKStreamJ,
KTable => KTableJ,
KGroupedTable => KGroupedTableJ
}
import org.apache.kafka.streams.processor.StateStore
import org.apache.kafka.streams.scala.kstream._
import scala.language.implicitConversions
/**
* Implicit conversions between the Scala wrapper objects and the underlying Java
* objects.
*/
object ImplicitConversions {
implicit def wrapKStream[K, V](inner: KStreamJ[K, V]): KStream[K, V] =
new KStream[K, V](inner)
implicit def wrapKGroupedStream[K, V](inner: KGroupedStreamJ[K, V]): KGroupedStream[K, V] =
new KGroupedStream[K, V](inner)
implicit def wrapTimeWindowedKStream[K, V](inner: TimeWindowedKStreamJ[K, V]): TimeWindowedKStream[K, V] =
new TimeWindowedKStream[K, V](inner)
implicit def wrapSessionWindowedKStream[K, V](inner: SessionWindowedKStreamJ[K, V]): SessionWindowedKStream[K, V] =
new SessionWindowedKStream[K, V](inner)
implicit def wrapCogroupedKStream[K, V](inner: CogroupedKStreamJ[K, V]): CogroupedKStream[K, V] =
new CogroupedKStream[K, V](inner)
implicit def wrapTimeWindowedCogroupedKStream[K, V](
inner: TimeWindowedCogroupedKStreamJ[K, V]
): TimeWindowedCogroupedKStream[K, V] =
new TimeWindowedCogroupedKStream[K, V](inner)
implicit def wrapSessionWindowedCogroupedKStream[K, V](
inner: SessionWindowedCogroupedKStreamJ[K, V]
): SessionWindowedCogroupedKStream[K, V] =
new SessionWindowedCogroupedKStream[K, V](inner)
implicit def wrapKTable[K, V](inner: KTableJ[K, V]): KTable[K, V] =
new KTable[K, V](inner)
implicit def wrapKGroupedTable[K, V](inner: KGroupedTableJ[K, V]): KGroupedTable[K, V] =
new KGroupedTable[K, V](inner)
implicit def tuple2ToKeyValue[K, V](tuple: (K, V)): KeyValue[K, V] = new KeyValue(tuple._1, tuple._2)
// we would also like to allow users implicit serdes
// and these implicits will convert them to `Grouped`, `Produced` or `Consumed`
implicit def consumedFromSerde[K, V](implicit keySerde: Serde[K], valueSerde: Serde[V]): Consumed[K, V] =
Consumed.`with`[K, V]
implicit def groupedFromSerde[K, V](implicit keySerde: Serde[K], valueSerde: Serde[V]): Grouped[K, V] =
Grouped.`with`[K, V]
implicit def joinedFromKeyValueOtherSerde[K, V, VO](implicit keySerde: Serde[K],
valueSerde: Serde[V],
otherValueSerde: Serde[VO]): Joined[K, V, VO] =
Joined.`with`[K, V, VO]
implicit def materializedFromSerde[K, V, S <: StateStore](implicit keySerde: Serde[K],
valueSerde: Serde[V]): Materialized[K, V, S] =
Materialized.`with`[K, V, S]
implicit def producedFromSerde[K, V](implicit keySerde: Serde[K], valueSerde: Serde[V]): Produced[K, V] =
Produced.`with`[K, V]
implicit def streamJoinFromKeyValueOtherSerde[K, V, VO](implicit keySerde: Serde[K],
valueSerde: Serde[V],
otherValueSerde: Serde[VO]): StreamJoined[K, V, VO] =
StreamJoined.`with`[K, V, VO]
}

View File

@@ -0,0 +1,74 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import java.util
import org.apache.kafka.common.serialization.{Deserializer, Serde, Serdes => JSerdes, Serializer}
import org.apache.kafka.streams.kstream.WindowedSerdes
object Serdes {
implicit def String: Serde[String] = JSerdes.String()
implicit def Long: Serde[Long] = JSerdes.Long().asInstanceOf[Serde[Long]]
implicit def JavaLong: Serde[java.lang.Long] = JSerdes.Long()
implicit def ByteArray: Serde[Array[Byte]] = JSerdes.ByteArray()
implicit def Bytes: Serde[org.apache.kafka.common.utils.Bytes] = JSerdes.Bytes()
implicit def Float: Serde[Float] = JSerdes.Float().asInstanceOf[Serde[Float]]
implicit def JavaFloat: Serde[java.lang.Float] = JSerdes.Float()
implicit def Double: Serde[Double] = JSerdes.Double().asInstanceOf[Serde[Double]]
implicit def JavaDouble: Serde[java.lang.Double] = JSerdes.Double()
implicit def Integer: Serde[Int] = JSerdes.Integer().asInstanceOf[Serde[Int]]
implicit def JavaInteger: Serde[java.lang.Integer] = JSerdes.Integer()
implicit def timeWindowedSerde[T](implicit tSerde: Serde[T]): WindowedSerdes.TimeWindowedSerde[T] =
new WindowedSerdes.TimeWindowedSerde[T](tSerde)
implicit def sessionWindowedSerde[T](implicit tSerde: Serde[T]): WindowedSerdes.SessionWindowedSerde[T] =
new WindowedSerdes.SessionWindowedSerde[T](tSerde)
def fromFn[T >: Null](serializer: T => Array[Byte], deserializer: Array[Byte] => Option[T]): Serde[T] =
JSerdes.serdeFrom(
new Serializer[T] {
override def serialize(topic: String, data: T): Array[Byte] = serializer(data)
override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = ()
override def close(): Unit = ()
},
new Deserializer[T] {
override def deserialize(topic: String, data: Array[Byte]): T = deserializer(data).orNull
override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = ()
override def close(): Unit = ()
}
)
def fromFn[T >: Null](serializer: (String, T) => Array[Byte],
deserializer: (String, Array[Byte]) => Option[T]): Serde[T] =
JSerdes.serdeFrom(
new Serializer[T] {
override def serialize(topic: String, data: T): Array[Byte] = serializer(topic, data)
override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = ()
override def close(): Unit = ()
},
new Deserializer[T] {
override def deserialize(topic: String, data: Array[Byte]): T = deserializer(topic, data).orNull
override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = ()
override def close(): Unit = ()
}
)
}

View File

@@ -0,0 +1,196 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import java.util.Properties
import java.util.regex.Pattern
import org.apache.kafka.streams.kstream.GlobalKTable
import org.apache.kafka.streams.processor.{ProcessorSupplier, StateStore}
import org.apache.kafka.streams.state.StoreBuilder
import org.apache.kafka.streams.{StreamsBuilder => StreamsBuilderJ, Topology}
import org.apache.kafka.streams.scala.kstream.{Consumed, KStream, KTable, Materialized}
import scala.collection.JavaConverters._
/**
* Wraps the Java class StreamsBuilder and delegates method calls to the underlying Java object.
*/
class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) {
/**
* Create a [[kstream.KStream]] from the specified topic.
* <p>
* The `implicit Consumed` instance provides the values of `auto.offset.reset` strategy, `TimestampExtractor`,
* key and value deserializers etc. If the implicit is not found in scope, compiler error will result.
* <p>
* A convenient alternative is to have the necessary implicit serdes in scope, which will be implicitly
* converted to generate an instance of `Consumed`. @see [[ImplicitConversions]].
* {{{
* // Brings all implicit conversions in scope
* import ImplicitConversions._
*
* // Bring implicit default serdes in scope
* import Serdes._
*
* val builder = new StreamsBuilder()
*
* // stream function gets the implicit Consumed which is constructed automatically
* // from the serdes through the implicits in ImplicitConversions#consumedFromSerde
* val userClicksStream: KStream[String, Long] = builder.stream(userClicksTopic)
* }}}
*
* @param topic the topic name
* @return a [[kstream.KStream]] for the specified topic
*/
def stream[K, V](topic: String)(implicit consumed: Consumed[K, V]): KStream[K, V] =
new KStream(inner.stream[K, V](topic, consumed))
/**
* Create a [[kstream.KStream]] from the specified topics.
*
* @param topics the topic names
* @return a [[kstream.KStream]] for the specified topics
* @see #stream(String)
* @see `org.apache.kafka.streams.StreamsBuilder#stream`
*/
def stream[K, V](topics: Set[String])(implicit consumed: Consumed[K, V]): KStream[K, V] =
new KStream(inner.stream[K, V](topics.asJava, consumed))
/**
* Create a [[kstream.KStream]] from the specified topic pattern.
*
* @param topicPattern the topic name pattern
* @return a [[kstream.KStream]] for the specified topics
* @see #stream(String)
* @see `org.apache.kafka.streams.StreamsBuilder#stream`
*/
def stream[K, V](topicPattern: Pattern)(implicit consumed: Consumed[K, V]): KStream[K, V] =
new KStream(inner.stream[K, V](topicPattern, consumed))
/**
* Create a [[kstream.KTable]] from the specified topic.
* <p>
* The `implicit Consumed` instance provides the values of `auto.offset.reset` strategy, `TimestampExtractor`,
* key and value deserializers etc. If the implicit is not found in scope, compiler error will result.
* <p>
* A convenient alternative is to have the necessary implicit serdes in scope, which will be implicitly
* converted to generate an instance of `Consumed`. @see [[ImplicitConversions]].
* {{{
* // Brings all implicit conversions in scope
* import ImplicitConversions._
*
* // Bring implicit default serdes in scope
* import Serdes._
*
* val builder = new StreamsBuilder()
*
* // stream function gets the implicit Consumed which is constructed automatically
* // from the serdes through the implicits in ImplicitConversions#consumedFromSerde
* val userClicksStream: KTable[String, Long] = builder.table(userClicksTopic)
* }}}
*
* @param topic the topic name
* @return a [[kstream.KTable]] for the specified topic
* @see `org.apache.kafka.streams.StreamsBuilder#table`
*/
def table[K, V](topic: String)(implicit consumed: Consumed[K, V]): KTable[K, V] =
new KTable(inner.table[K, V](topic, consumed))
/**
* Create a [[kstream.KTable]] from the specified topic.
*
* @param topic the topic name
* @param materialized the instance of `Materialized` used to materialize a state store
* @return a [[kstream.KTable]] for the specified topic
* @see #table(String)
* @see `org.apache.kafka.streams.StreamsBuilder#table`
*/
def table[K, V](topic: String, materialized: Materialized[K, V, ByteArrayKeyValueStore])(
implicit consumed: Consumed[K, V]
): KTable[K, V] =
new KTable(inner.table[K, V](topic, consumed, materialized))
/**
* Create a `GlobalKTable` from the specified topic. The serializers from the implicit `Consumed`
* instance will be used. Input records with `null` key will be dropped.
*
* @param topic the topic name
* @return a `GlobalKTable` for the specified topic
* @see `org.apache.kafka.streams.StreamsBuilder#globalTable`
*/
def globalTable[K, V](topic: String)(implicit consumed: Consumed[K, V]): GlobalKTable[K, V] =
inner.globalTable(topic, consumed)
/**
* Create a `GlobalKTable` from the specified topic. The resulting `GlobalKTable` will be materialized
* in a local `KeyValueStore` configured with the provided instance of `Materialized`. The serializers
* from the implicit `Consumed` instance will be used.
*
* @param topic the topic name
* @param materialized the instance of `Materialized` used to materialize a state store
* @return a `GlobalKTable` for the specified topic
* @see `org.apache.kafka.streams.StreamsBuilder#globalTable`
*/
def globalTable[K, V](topic: String, materialized: Materialized[K, V, ByteArrayKeyValueStore])(
implicit consumed: Consumed[K, V]
): GlobalKTable[K, V] =
inner.globalTable(topic, consumed, materialized)
/**
* Adds a state store to the underlying `Topology`. The store must still be "connected" to a `Processor`,
* `Transformer`, or `ValueTransformer` before it can be used.
* <p>
* It is required to connect state stores to `Processor`, `Transformer`, or `ValueTransformer` before they can be used.
*
* @param builder the builder used to obtain this state store `StateStore` instance
* @return the underlying Java abstraction `StreamsBuilder` after adding the `StateStore`
* @throws org.apache.kafka.streams.errors.TopologyException if state store supplier is already added
* @see `org.apache.kafka.streams.StreamsBuilder#addStateStore`
*/
def addStateStore(builder: StoreBuilder[_ <: StateStore]): StreamsBuilderJ = inner.addStateStore(builder)
/**
* Adds a global `StateStore` to the topology. Global stores should not be added to `Processor`, `Transformer`,
* or `ValueTransformer` (in contrast to regular stores).
* <p>
* It is not required to connect a global store to `Processor`, `Transformer`, or `ValueTransformer`;
* those have read-only access to all global stores by default.
*
* @see `org.apache.kafka.streams.StreamsBuilder#addGlobalStore`
*/
def addGlobalStore(storeBuilder: StoreBuilder[_ <: StateStore],
topic: String,
consumed: Consumed[_, _],
stateUpdateSupplier: ProcessorSupplier[_, _]): StreamsBuilderJ =
inner.addGlobalStore(storeBuilder, topic, consumed, stateUpdateSupplier)
def build(): Topology = inner.build()
/**
* Returns the `Topology` that represents the specified processing logic and accepts
* a `Properties` instance used to indicate whether to optimize topology or not.
*
* @param props the `Properties` used for building possibly optimized topology
* @return the `Topology` that represents the specified processing logic
* @see `org.apache.kafka.streams.StreamsBuilder#build`
*/
def build(props: Properties): Topology = inner.build(props)
}

View File

@@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.streams.kstream.{CogroupedKStream => CogroupedKStreamJ, SessionWindows, Window, Windows}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{AggregatorFromFunction, InitializerFromFunction}
/**
* Wraps the Java class CogroupedKStream and delegates method calls to the underlying Java object.
*
* @tparam KIn Type of keys
* @tparam VOut Type of values
* @param inner The underlying Java abstraction for CogroupedKStream
*
* @see `org.apache.kafka.streams.kstream.CogroupedKStream`
*/
class CogroupedKStream[KIn, VOut](val inner: CogroupedKStreamJ[KIn, VOut]) {
def cogroup[VIn](groupedStream: KGroupedStream[KIn, VIn],
aggregator: (KIn, VIn, VOut) => VOut): CogroupedKStream[KIn, VOut] =
new CogroupedKStream(inner.cogroup(groupedStream.inner, aggregator.asAggregator))
/**
* Aggregate the values of records in these streams by the grouped key and defined window.
*
* @param initializer an `Initializer` that computes an initial intermediate aggregation result.
* Cannot be { @code null}.
* @param materialized an instance of `Materialized` used to materialize a state store.
* Cannot be { @code null}.
* @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the latest
* (rolling) aggregate for each key
* @see `org.apache.kafka.streams.kstream.CogroupedKStream#aggregate`
*/
def aggregate(initializer: => VOut)(
implicit materialized: Materialized[KIn, VOut, ByteArrayKeyValueStore]
): KTable[KIn, VOut] = new KTable(inner.aggregate((() => initializer).asInitializer, materialized))
/**
* Create a new [[TimeWindowedCogroupedKStream]] instance that can be used to perform windowed aggregations.
*
* @param windows the specification of the aggregation `Windows`
* @return an instance of [[TimeWindowedCogroupedKStream]]
* @see `org.apache.kafka.streams.kstream.CogroupedKStream#windowedBy`
*/
def windowedBy[W <: Window](windows: Windows[W]): TimeWindowedCogroupedKStream[KIn, VOut] =
new TimeWindowedCogroupedKStream(inner.windowedBy(windows))
/**
* Create a new [[SessionWindowedKStream]] instance that can be used to perform session windowed aggregations.
*
* @param windows the specification of the aggregation `SessionWindows`
* @return an instance of [[SessionWindowedKStream]]
* @see `org.apache.kafka.streams.kstream.KGroupedStream#windowedBy`
*/
def windowedBy(windows: SessionWindows): SessionWindowedCogroupedKStream[KIn, VOut] =
new SessionWindowedCogroupedKStream(inner.windowedBy(windows))
}

View File

@@ -0,0 +1,79 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.common.serialization.Serde
import org.apache.kafka.streams.kstream.{Consumed => ConsumedJ}
import org.apache.kafka.streams.Topology
import org.apache.kafka.streams.processor.TimestampExtractor
object Consumed {
/**
* Create an instance of [[Consumed]] with the supplied arguments. `null` values are acceptable.
*
* @tparam K key type
* @tparam V value type
* @param timestampExtractor the timestamp extractor to used. If `null` the default timestamp extractor from
* config will be used
* @param resetPolicy the offset reset policy to be used. If `null` the default reset policy from config
* will be used
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @return a new instance of [[Consumed]]
*/
def `with`[K, V](
timestampExtractor: TimestampExtractor,
resetPolicy: Topology.AutoOffsetReset
)(implicit keySerde: Serde[K], valueSerde: Serde[V]): ConsumedJ[K, V] =
ConsumedJ.`with`(keySerde, valueSerde, timestampExtractor, resetPolicy)
/**
* Create an instance of [[Consumed]] with key and value [[Serde]]s.
*
* @tparam K key type
* @tparam V value type
* @return a new instance of [[Consumed]]
*/
def `with`[K, V](implicit keySerde: Serde[K], valueSerde: Serde[V]): ConsumedJ[K, V] =
ConsumedJ.`with`(keySerde, valueSerde)
/**
* Create an instance of [[Consumed]] with a [[TimestampExtractor]].
*
* @param timestampExtractor the timestamp extractor to used. If `null` the default timestamp extractor from
* config will be used
* @tparam K key type
* @tparam V value type
* @return a new instance of [[Consumed]]
*/
def `with`[K, V](timestampExtractor: TimestampExtractor)(implicit keySerde: Serde[K],
valueSerde: Serde[V]): ConsumedJ[K, V] =
ConsumedJ.`with`(timestampExtractor).withKeySerde(keySerde).withValueSerde(valueSerde)
/**
* Create an instance of [[Consumed]] with a [[Topology.AutoOffsetReset]].
*
* @tparam K key type
* @tparam V value type
* @param resetPolicy the offset reset policy to be used. If `null` the default reset policy from config will be used
* @return a new instance of [[Consumed]]
*/
def `with`[K, V](resetPolicy: Topology.AutoOffsetReset)(implicit keySerde: Serde[K],
valueSerde: Serde[V]): ConsumedJ[K, V] =
ConsumedJ.`with`(resetPolicy).withKeySerde(keySerde).withValueSerde(valueSerde)
}

View File

@@ -0,0 +1,51 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.common.serialization.Serde
import org.apache.kafka.streams.kstream.{Grouped => GroupedJ}
object Grouped {
/**
* Construct a `Grouped` instance with the provided key and value [[Serde]]s.
* If the [[Serde]] params are `null` the default serdes defined in the configs will be used.
*
* @tparam K the key type
* @tparam V the value type
* @param keySerde keySerde that will be used to materialize a stream
* @param valueSerde valueSerde that will be used to materialize a stream
* @return a new instance of [[Grouped]] configured with the provided serdes
*/
def `with`[K, V](implicit keySerde: Serde[K], valueSerde: Serde[V]): GroupedJ[K, V] =
GroupedJ.`with`(keySerde, valueSerde)
/**
* Construct a `Grouped` instance with the provided key and value [[Serde]]s.
* If the [[Serde]] params are `null` the default serdes defined in the configs will be used.
*
* @tparam K the key type
* @tparam V the value type
* @param name the name used as part of a potential repartition topic
* @param keySerde keySerde that will be used to materialize a stream
* @param valueSerde valueSerde that will be used to materialize a stream
* @return a new instance of [[Grouped]] configured with the provided serdes
*/
def `with`[K, V](name: String)(implicit keySerde: Serde[K], valueSerde: Serde[V]): GroupedJ[K, V] =
GroupedJ.`with`(name, keySerde, valueSerde)
}

View File

@@ -0,0 +1,64 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.common.serialization.Serde
import org.apache.kafka.streams.kstream.{Joined => JoinedJ}
object Joined {
/**
* Create an instance of [[org.apache.kafka.streams.kstream.Joined]] with key, value, and otherValue [[Serde]]
* instances.
* `null` values are accepted and will be replaced by the default serdes as defined in config.
*
* @tparam K key type
* @tparam V value type
* @tparam VO other value type
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @param otherValueSerde the otherValue serde to use. If `null` the default value serde from config will be used
* @return new [[org.apache.kafka.streams.kstream.Joined]] instance with the provided serdes
*/
def `with`[K, V, VO](implicit keySerde: Serde[K],
valueSerde: Serde[V],
otherValueSerde: Serde[VO]): JoinedJ[K, V, VO] =
JoinedJ.`with`(keySerde, valueSerde, otherValueSerde)
/**
* Create an instance of [[org.apache.kafka.streams.kstream.Joined]] with key, value, and otherValue [[Serde]]
* instances.
* `null` values are accepted and will be replaced by the default serdes as defined in config.
*
* @tparam K key type
* @tparam V value type
* @tparam VO other value type
* @param name name of possible repartition topic
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @param otherValueSerde the otherValue serde to use. If `null` the default value serde from config will be used
* @return new [[org.apache.kafka.streams.kstream.Joined]] instance with the provided serdes
*/
// disable spotless scala, which wants to make a mess of the argument lists
// format: off
def `with`[K, V, VO](name: String)
(implicit keySerde: Serde[K],
valueSerde: Serde[V],
otherValueSerde: Serde[VO]): JoinedJ[K, V, VO] =
JoinedJ.`with`(keySerde, valueSerde, otherValueSerde, name)
// format:on
}

View File

@@ -0,0 +1,128 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.streams.kstream.internals.KTableImpl
import org.apache.kafka.streams.kstream.{
SessionWindows,
Window,
Windows,
KGroupedStream => KGroupedStreamJ,
KTable => KTableJ
}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{
AggregatorFromFunction,
InitializerFromFunction,
ReducerFromFunction,
ValueMapperFromFunction
}
/**
* Wraps the Java class KGroupedStream and delegates method calls to the underlying Java object.
*
* @tparam K Type of keys
* @tparam V Type of values
* @param inner The underlying Java abstraction for KGroupedStream
*
* @see `org.apache.kafka.streams.kstream.KGroupedStream`
*/
class KGroupedStream[K, V](val inner: KGroupedStreamJ[K, V]) {
/**
* Count the number of records in this stream by the grouped key.
* The result is written into a local `KeyValueStore` (which is basically an ever-updating materialized view)
* provided by the given `materialized`.
*
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys and `Long` values that
* represent the latest (rolling) count (i.e., number of records) for each key
* @see `org.apache.kafka.streams.kstream.KGroupedStream#count`
*/
def count()(implicit materialized: Materialized[K, Long, ByteArrayKeyValueStore]): KTable[K, Long] = {
val javaCountTable: KTableJ[K, java.lang.Long] =
inner.count(materialized.asInstanceOf[Materialized[K, java.lang.Long, ByteArrayKeyValueStore]])
val tableImpl = javaCountTable.asInstanceOf[KTableImpl[K, ByteArrayKeyValueStore, java.lang.Long]]
new KTable(
javaCountTable.mapValues[Long](
((l: java.lang.Long) => Long2long(l)).asValueMapper,
Materialized.`with`[K, Long, ByteArrayKeyValueStore](tableImpl.keySerde(), Serdes.Long)
)
)
}
/**
* Combine the values of records in this stream by the grouped key.
*
* @param reducer a function `(V, V) => V` that computes a new aggregate result.
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the
* latest (rolling) aggregate for each key
* @see `org.apache.kafka.streams.kstream.KGroupedStream#reduce`
*/
def reduce(reducer: (V, V) => V)(implicit materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
new KTable(inner.reduce(reducer.asReducer, materialized))
/**
* Aggregate the values of records in this stream by the grouped key.
*
* @param initializer an `Initializer` that computes an initial intermediate aggregation result
* @param aggregator an `Aggregator` that computes a new aggregate result
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the
* latest (rolling) aggregate for each key
* @see `org.apache.kafka.streams.kstream.KGroupedStream#aggregate`
*/
def aggregate[VR](initializer: => VR)(aggregator: (K, V, VR) => VR)(
implicit materialized: Materialized[K, VR, ByteArrayKeyValueStore]
): KTable[K, VR] =
new KTable(inner.aggregate((() => initializer).asInitializer, aggregator.asAggregator, materialized))
/**
* Create a new [[TimeWindowedKStream]] instance that can be used to perform windowed aggregations.
*
* @param windows the specification of the aggregation `Windows`
* @return an instance of [[TimeWindowedKStream]]
* @see `org.apache.kafka.streams.kstream.KGroupedStream#windowedBy`
*/
def windowedBy[W <: Window](windows: Windows[W]): TimeWindowedKStream[K, V] =
new TimeWindowedKStream(inner.windowedBy(windows))
/**
* Create a new [[SessionWindowedKStream]] instance that can be used to perform session windowed aggregations.
*
* @param windows the specification of the aggregation `SessionWindows`
* @return an instance of [[SessionWindowedKStream]]
* @see `org.apache.kafka.streams.kstream.KGroupedStream#windowedBy`
*/
def windowedBy(windows: SessionWindows): SessionWindowedKStream[K, V] =
new SessionWindowedKStream(inner.windowedBy(windows))
/**
* Create a new [[CogroupedKStream]] from this grouped KStream to allow cogrouping other [[KGroupedStream]] to it.
*
* @param aggregator an `Aggregator` that computes a new aggregate result
* @return an instance of [[CogroupedKStream]]
* @see `org.apache.kafka.streams.kstream.KGroupedStream#cogroup`
*/
def cogroup[VR](aggregator: (K, V, VR) => VR): CogroupedKStream[K, VR] =
new CogroupedKStream(inner.cogroup(aggregator.asAggregator))
}

View File

@@ -0,0 +1,89 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.streams.kstream.{KGroupedTable => KGroupedTableJ}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{
AggregatorFromFunction,
InitializerFromFunction,
ReducerFromFunction
}
/**
* Wraps the Java class KGroupedTable and delegates method calls to the underlying Java object.
*
* @tparam K Type of keys
* @tparam V Type of values
* @param inner The underlying Java abstraction for KGroupedTable
*
* @see `org.apache.kafka.streams.kstream.KGroupedTable`
*/
class KGroupedTable[K, V](inner: KGroupedTableJ[K, V]) {
/**
* Count number of records of the original [[KTable]] that got [[KTable#groupBy]] to
* the same key into a new instance of [[KTable]].
*
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys and `Long` values that
* represent the latest (rolling) count (i.e., number of records) for each key
* @see `org.apache.kafka.streams.kstream.KGroupedTable#count`
*/
def count()(implicit materialized: Materialized[K, Long, ByteArrayKeyValueStore]): KTable[K, Long] = {
val c: KTable[K, java.lang.Long] =
new KTable(inner.count(materialized.asInstanceOf[Materialized[K, java.lang.Long, ByteArrayKeyValueStore]]))
c.mapValues[Long](Long2long _)
}
/**
* Combine the value of records of the original [[KTable]] that got [[KTable#groupBy]]
* to the same key into a new instance of [[KTable]].
*
* @param adder a function that adds a new value to the aggregate result
* @param subtractor a function that removed an old value from the aggregate result
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the
* latest (rolling) aggregate for each key
* @see `org.apache.kafka.streams.kstream.KGroupedTable#reduce`
*/
def reduce(adder: (V, V) => V,
subtractor: (V, V) => V)(implicit materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
new KTable(inner.reduce(adder.asReducer, subtractor.asReducer, materialized))
/**
* Aggregate the value of records of the original [[KTable]] that got [[KTable#groupBy]]
* to the same key into a new instance of [[KTable]] using default serializers and deserializers.
*
* @param initializer a function that provides an initial aggregate result value
* @param adder a function that adds a new record to the aggregate result
* @param subtractor an aggregator function that removed an old record from the aggregate result
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the
* latest (rolling) aggregate for each key
* @see `org.apache.kafka.streams.kstream.KGroupedTable#aggregate`
*/
def aggregate[VR](initializer: => VR)(adder: (K, V, VR) => VR, subtractor: (K, V, VR) => VR)(
implicit materialized: Materialized[K, VR, ByteArrayKeyValueStore]
): KTable[K, VR] =
new KTable(
inner.aggregate((() => initializer).asInitializer, adder.asAggregator, subtractor.asAggregator, materialized)
)
}

View File

@@ -0,0 +1,686 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.streams.KeyValue
import org.apache.kafka.streams.kstream.{
GlobalKTable,
JoinWindows,
Printed,
TransformerSupplier,
ValueTransformerSupplier,
ValueTransformerWithKeySupplier,
KStream => KStreamJ
}
import org.apache.kafka.streams.processor.{Processor, ProcessorSupplier, TopicNameExtractor}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{
FlatValueMapperFromFunction,
FlatValueMapperWithKeyFromFunction,
ForeachActionFromFunction,
KeyValueMapperFromFunction,
MapperFromFunction,
PredicateFromFunction,
TransformerSupplierAsJava,
ValueMapperFromFunction,
ValueMapperWithKeyFromFunction,
ValueTransformerSupplierAsJava,
ValueTransformerSupplierWithKeyAsJava
}
import scala.collection.JavaConverters._
/**
* Wraps the Java class [[org.apache.kafka.streams.kstream.KStream KStream]] and delegates method calls to the
* underlying Java object.
*
* @tparam K Type of keys
* @tparam V Type of values
* @param inner The underlying Java abstraction for KStream
*
* @see `org.apache.kafka.streams.kstream.KStream`
*/
class KStream[K, V](val inner: KStreamJ[K, V]) {
/**
* Create a new [[KStream]] that consists all records of this stream which satisfies the given predicate.
*
* @param predicate a filter that is applied to each record
* @return a [[KStream]] that contains only those records that satisfy the given predicate
* @see `org.apache.kafka.streams.kstream.KStream#filter`
*/
def filter(predicate: (K, V) => Boolean): KStream[K, V] =
new KStream(inner.filter(predicate.asPredicate))
/**
* Create a new [[KStream]] that consists all records of this stream which do <em>not</em> satisfy the given
* predicate.
*
* @param predicate a filter that is applied to each record
* @return a [[KStream]] that contains only those records that do <em>not</em> satisfy the given predicate
* @see `org.apache.kafka.streams.kstream.KStream#filterNot`
*/
def filterNot(predicate: (K, V) => Boolean): KStream[K, V] =
new KStream(inner.filterNot(predicate.asPredicate))
/**
* Set a new key (with possibly new type) for each input record.
* <p>
* The function `mapper` passed is applied to every record and results in the generation of a new
* key `KR`. The function outputs a new [[KStream]] where each record has this new key.
*
* @param mapper a function `(K, V) => KR` that computes a new key for each record
* @return a [[KStream]] that contains records with new key (possibly of different type) and unmodified value
* @see `org.apache.kafka.streams.kstream.KStream#selectKey`
*/
def selectKey[KR](mapper: (K, V) => KR): KStream[KR, V] =
new KStream(inner.selectKey[KR](mapper.asKeyValueMapper))
/**
* Transform each record of the input stream into a new record in the output stream (both key and value type can be
* altered arbitrarily).
* <p>
* The provided `mapper`, a function `(K, V) => (KR, VR)` is applied to each input record and computes a new output record.
*
* @param mapper a function `(K, V) => (KR, VR)` that computes a new output record
* @return a [[KStream]] that contains records with new key and value (possibly both of different type)
* @see `org.apache.kafka.streams.kstream.KStream#map`
*/
def map[KR, VR](mapper: (K, V) => (KR, VR)): KStream[KR, VR] =
new KStream(inner.map[KR, VR](mapper.asKeyValueMapper))
/**
* Transform the value of each input record into a new value (with possible new type) of the output record.
* <p>
* The provided `mapper`, a function `V => VR` is applied to each input record value and computes a new value for it
*
* @param mapper, a function `V => VR` that computes a new output value
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#mapValues`
*/
def mapValues[VR](mapper: V => VR): KStream[K, VR] =
new KStream(inner.mapValues[VR](mapper.asValueMapper))
/**
* Transform the value of each input record into a new value (with possible new type) of the output record.
* <p>
* The provided `mapper`, a function `(K, V) => VR` is applied to each input record value and computes a new value for it
*
* @param mapper, a function `(K, V) => VR` that computes a new output value
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#mapValues`
*/
def mapValues[VR](mapper: (K, V) => VR): KStream[K, VR] =
new KStream(inner.mapValues[VR](mapper.asValueMapperWithKey))
/**
* Transform each record of the input stream into zero or more records in the output stream (both key and value type
* can be altered arbitrarily).
* <p>
* The provided `mapper`, function `(K, V) => Iterable[(KR, VR)]` is applied to each input record and computes zero or more output records.
*
* @param mapper function `(K, V) => Iterable[(KR, VR)]` that computes the new output records
* @return a [[KStream]] that contains more or less records with new key and value (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#flatMap`
*/
def flatMap[KR, VR](mapper: (K, V) => Iterable[(KR, VR)]): KStream[KR, VR] = {
val kvMapper = mapper.tupled.andThen(_.map(ImplicitConversions.tuple2ToKeyValue).asJava)
new KStream(inner.flatMap[KR, VR](((k: K, v: V) => kvMapper(k, v)).asKeyValueMapper))
}
/**
* Create a new [[KStream]] by transforming the value of each record in this stream into zero or more values
* with the same key in the new stream.
* <p>
* Transform the value of each input record into zero or more records with the same (unmodified) key in the output
* stream (value type can be altered arbitrarily).
* The provided `mapper`, a function `V => Iterable[VR]` is applied to each input record and computes zero or more output values.
*
* @param mapper a function `V => Iterable[VR]` that computes the new output values
* @return a [[KStream]] that contains more or less records with unmodified keys and new values of different type
* @see `org.apache.kafka.streams.kstream.KStream#flatMapValues`
*/
def flatMapValues[VR](mapper: V => Iterable[VR]): KStream[K, VR] =
new KStream(inner.flatMapValues[VR](mapper.asValueMapper))
/**
* Create a new [[KStream]] by transforming the value of each record in this stream into zero or more values
* with the same key in the new stream.
* <p>
* Transform the value of each input record into zero or more records with the same (unmodified) key in the output
* stream (value type can be altered arbitrarily).
* The provided `mapper`, a function `(K, V) => Iterable[VR]` is applied to each input record and computes zero or more output values.
*
* @param mapper a function `(K, V) => Iterable[VR]` that computes the new output values
* @return a [[KStream]] that contains more or less records with unmodified keys and new values of different type
* @see `org.apache.kafka.streams.kstream.KStream#flatMapValues`
*/
def flatMapValues[VR](mapper: (K, V) => Iterable[VR]): KStream[K, VR] =
new KStream(inner.flatMapValues[VR](mapper.asValueMapperWithKey))
/**
* Print the records of this KStream using the options provided by `Printed`
*
* @param printed options for printing
* @see `org.apache.kafka.streams.kstream.KStream#print`
*/
def print(printed: Printed[K, V]): Unit = inner.print(printed)
/**
* Perform an action on each record of `KStream`
*
* @param action an action to perform on each record
* @see `org.apache.kafka.streams.kstream.KStream#foreach`
*/
def foreach(action: (K, V) => Unit): Unit =
inner.foreach(action.asForeachAction)
/**
* Creates an array of `KStream` from this stream by branching the records in the original stream based on
* the supplied predicates.
*
* @param predicates the ordered list of functions that return a Boolean
* @return multiple distinct substreams of this [[KStream]]
* @see `org.apache.kafka.streams.kstream.KStream#branch`
*/
//noinspection ScalaUnnecessaryParentheses
def branch(predicates: ((K, V) => Boolean)*): Array[KStream[K, V]] =
inner.branch(predicates.map(_.asPredicate): _*).map(kstream => new KStream(kstream))
/**
* Materialize this stream to a topic and creates a new [[KStream]] from the topic using the `Produced` instance for
* configuration of the `Serde key serde`, `Serde value serde`, and `StreamPartitioner`
* <p>
* The user can either supply the `Produced` instance as an implicit in scope or she can also provide implicit
* key and value serdes that will be converted to a `Produced` instance implicitly.
* <p>
* {{{
* Example:
*
* // brings implicit serdes in scope
* import Serdes._
*
* //..
* val clicksPerRegion: KTable[String, Long] = //..
*
* // Implicit serdes in scope will generate an implicit Produced instance, which
* // will be passed automatically to the call of through below
* clicksPerRegion.through(topic)
*
* // Similarly you can create an implicit Produced and it will be passed implicitly
* // to the through call
* }}}
*
* @param topic the topic name
* @param produced the instance of Produced that gives the serdes and `StreamPartitioner`
* @return a [[KStream]] that contains the exact same (and potentially repartitioned) records as this [[KStream]]
* @see `org.apache.kafka.streams.kstream.KStream#through`
*/
def through(topic: String)(implicit produced: Produced[K, V]): KStream[K, V] =
new KStream(inner.through(topic, produced))
/**
* Materialize this stream to a topic using the `Produced` instance for
* configuration of the `Serde key serde`, `Serde value serde`, and `StreamPartitioner`
* <p>
* The user can either supply the `Produced` instance as an implicit in scope or she can also provide implicit
* key and value serdes that will be converted to a `Produced` instance implicitly.
* <p>
* {{{
* Example:
*
* // brings implicit serdes in scope
* import Serdes._
*
* //..
* val clicksPerRegion: KTable[String, Long] = //..
*
* // Implicit serdes in scope will generate an implicit Produced instance, which
* // will be passed automatically to the call of through below
* clicksPerRegion.to(topic)
*
* // Similarly you can create an implicit Produced and it will be passed implicitly
* // to the through call
* }}}
*
* @param topic the topic name
* @param produced the instance of Produced that gives the serdes and `StreamPartitioner`
* @see `org.apache.kafka.streams.kstream.KStream#to`
*/
def to(topic: String)(implicit produced: Produced[K, V]): Unit =
inner.to(topic, produced)
/**
* Dynamically materialize this stream to topics using the `Produced` instance for
* configuration of the `Serde key serde`, `Serde value serde`, and `StreamPartitioner`.
* The topic names for each record to send to is dynamically determined based on the given mapper.
* <p>
* The user can either supply the `Produced` instance as an implicit in scope or she can also provide implicit
* key and value serdes that will be converted to a `Produced` instance implicitly.
* <p>
* {{{
* Example:
*
* // brings implicit serdes in scope
* import Serdes._
*
* //..
* val clicksPerRegion: KTable[String, Long] = //..
*
* // Implicit serdes in scope will generate an implicit Produced instance, which
* // will be passed automatically to the call of through below
* clicksPerRegion.to(topicChooser)
*
* // Similarly you can create an implicit Produced and it will be passed implicitly
* // to the through call
* }}}
*
* @param extractor the extractor to determine the name of the Kafka topic to write to for reach record
* @param produced the instance of Produced that gives the serdes and `StreamPartitioner`
* @see `org.apache.kafka.streams.kstream.KStream#to`
*/
def to(extractor: TopicNameExtractor[K, V])(implicit produced: Produced[K, V]): Unit =
inner.to(extractor, produced)
/**
* Convert this stream to a [[KTable]].
*
* @return a [[KTable]] that contains the same records as this [[KStream]]
* @see `org.apache.kafka.streams.kstream.KStream#toTable`
*/
def toTable: KTable[K, V] =
new KTable(inner.toTable)
/**
* Convert this stream to a [[KTable]].
*
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains the same records as this [[KStream]]
* @see `org.apache.kafka.streams.kstream.KStream#toTable`
*/
def toTable(materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
new KTable(inner.toTable(materialized))
/**
* Transform each record of the input stream into zero or more records in the output stream (both key and value type
* can be altered arbitrarily).
* A `Transformer` (provided by the given `TransformerSupplier`) is applied to each input record
* and computes zero or more output records.
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
* to the `Transformer`.
* It's not required to connect global state stores that are added via `addGlobalStore`;
* read-only access to global state stores is available by default.
*
* @param transformerSupplier the `TransformerSuplier` that generates `Transformer`
* @param stateStoreNames the names of the state stores used by the processor
* @return a [[KStream]] that contains more or less records with new key and value (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#transform`
*/
def transform[K1, V1](transformerSupplier: TransformerSupplier[K, V, KeyValue[K1, V1]],
stateStoreNames: String*): KStream[K1, V1] =
new KStream(inner.transform(transformerSupplier, stateStoreNames: _*))
/**
* Transform each record of the input stream into zero or more records in the output stream (both key and value type
* can be altered arbitrarily).
* A `Transformer` (provided by the given `TransformerSupplier`) is applied to each input record
* and computes zero or more output records.
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
* to the `Transformer`.
* It's not required to connect global state stores that are added via `addGlobalStore`;
* read-only access to global state stores is available by default.
*
* @param transformerSupplier the `TransformerSuplier` that generates `Transformer`
* @param stateStoreNames the names of the state stores used by the processor
* @return a [[KStream]] that contains more or less records with new key and value (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#transform`
*/
def flatTransform[K1, V1](transformerSupplier: TransformerSupplier[K, V, Iterable[KeyValue[K1, V1]]],
stateStoreNames: String*): KStream[K1, V1] =
new KStream(inner.flatTransform(transformerSupplier.asJava, stateStoreNames: _*))
/**
* Transform the value of each input record into zero or more records (with possible new type) in the
* output stream.
* A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input
* record value and computes a new value for it.
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
* to the `ValueTransformer`.
* It's not required to connect global state stores that are added via `addGlobalStore`;
* read-only access to global state stores is available by default.
*
* @param valueTransformerSupplier a instance of `ValueTransformerSupplier` that generates a `ValueTransformer`
* @param stateStoreNames the names of the state stores used by the processor
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
*/
def flatTransformValues[VR](valueTransformerSupplier: ValueTransformerSupplier[V, Iterable[VR]],
stateStoreNames: String*): KStream[K, VR] =
new KStream(inner.flatTransformValues[VR](valueTransformerSupplier.asJava, stateStoreNames: _*))
/**
* Transform the value of each input record into zero or more records (with possible new type) in the
* output stream.
* A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input
* record value and computes a new value for it.
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
* to the `ValueTransformer`.
* It's not required to connect global state stores that are added via `addGlobalStore`;
* read-only access to global state stores is available by default.
*
* @param valueTransformerSupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey`
* @param stateStoreNames the names of the state stores used by the processor
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
*/
def flatTransformValues[VR](valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, Iterable[VR]],
stateStoreNames: String*): KStream[K, VR] =
new KStream(inner.flatTransformValues[VR](valueTransformerSupplier.asJava, stateStoreNames: _*))
/**
* Transform the value of each input record into a new value (with possible new type) of the output record.
* A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input
* record value and computes a new value for it.
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
* to the `ValueTransformer`.
* It's not required to connect global state stores that are added via `addGlobalStore`;
* read-only access to global state stores is available by default.
*
* @param valueTransformerSupplier a instance of `ValueTransformerSupplier` that generates a `ValueTransformer`
* @param stateStoreNames the names of the state stores used by the processor
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
*/
def transformValues[VR](valueTransformerSupplier: ValueTransformerSupplier[V, VR],
stateStoreNames: String*): KStream[K, VR] =
new KStream(inner.transformValues[VR](valueTransformerSupplier, stateStoreNames: _*))
/**
* Transform the value of each input record into a new value (with possible new type) of the output record.
* A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input
* record value and computes a new value for it.
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
* to the `ValueTransformer`.
* It's not required to connect global state stores that are added via `addGlobalStore`;
* read-only access to global state stores is available by default.
*
* @param valueTransformerSupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey`
* @param stateStoreNames the names of the state stores used by the processor
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
*/
def transformValues[VR](valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, VR],
stateStoreNames: String*): KStream[K, VR] =
new KStream(inner.transformValues[VR](valueTransformerSupplier, stateStoreNames: _*))
/**
* Process all records in this stream, one record at a time, by applying a `Processor` (provided by the given
* `processorSupplier`).
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
* to the `Processor`.
* It's not required to connect global state stores that are added via `addGlobalStore`;
* read-only access to global state stores is available by default.
*
* @param processorSupplier a function that generates a [[org.apache.kafka.streams.processor.Processor]]
* @param stateStoreNames the names of the state store used by the processor
* @see `org.apache.kafka.streams.kstream.KStream#process`
*/
def process(processorSupplier: () => Processor[K, V], stateStoreNames: String*): Unit = {
val processorSupplierJ: ProcessorSupplier[K, V] = () => processorSupplier()
inner.process(processorSupplierJ, stateStoreNames: _*)
}
/**
* Group the records by their current key into a [[KGroupedStream]]
* <p>
* The user can either supply the `Grouped` instance as an implicit in scope or she can also provide an implicit
* serdes that will be converted to a `Grouped` instance implicitly.
* <p>
* {{{
* Example:
*
* // brings implicit serdes in scope
* import Serdes._
*
* val clicksPerRegion: KTable[String, Long] =
* userClicksStream
* .leftJoin(userRegionsTable, (clicks: Long, region: String) => (if (region == null) "UNKNOWN" else region, clicks))
* .map((_, regionWithClicks) => regionWithClicks)
*
* // the groupByKey gets the Grouped instance through an implicit conversion of the
* // serdes brought into scope through the import Serdes._ above
* .groupByKey
* .reduce(_ + _)
*
* // Similarly you can create an implicit Grouped and it will be passed implicitly
* // to the groupByKey call
* }}}
*
* @param grouped the instance of Grouped that gives the serdes
* @return a [[KGroupedStream]] that contains the grouped records of the original [[KStream]]
* @see `org.apache.kafka.streams.kstream.KStream#groupByKey`
*/
def groupByKey(implicit grouped: Grouped[K, V]): KGroupedStream[K, V] =
new KGroupedStream(inner.groupByKey(grouped))
/**
* Group the records of this [[KStream]] on a new key that is selected using the provided key transformation function
* and the `Grouped` instance.
* <p>
* The user can either supply the `Grouped` instance as an implicit in scope or she can also provide an implicit
* serdes that will be converted to a `Grouped` instance implicitly.
* <p>
* {{{
* Example:
*
* // brings implicit serdes in scope
* import Serdes._
*
* val textLines = streamBuilder.stream[String, String](inputTopic)
*
* val pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS)
*
* val wordCounts: KTable[String, Long] =
* textLines.flatMapValues(v => pattern.split(v.toLowerCase))
*
* // the groupBy gets the Grouped instance through an implicit conversion of the
* // serdes brought into scope through the import Serdes._ above
* .groupBy((k, v) => v)
*
* .count()
* }}}
*
* @param selector a function that computes a new key for grouping
* @return a [[KGroupedStream]] that contains the grouped records of the original [[KStream]]
* @see `org.apache.kafka.streams.kstream.KStream#groupBy`
*/
def groupBy[KR](selector: (K, V) => KR)(implicit grouped: Grouped[KR, V]): KGroupedStream[KR, V] =
new KGroupedStream(inner.groupBy(selector.asKeyValueMapper, grouped))
/**
* Join records of this stream with another [[KStream]]'s records using windowed inner equi join with
* serializers and deserializers supplied by the implicit `StreamJoined` instance.
*
* @param otherStream the [[KStream]] to be joined with this stream
* @param joiner a function that computes the join result for a pair of matching records
* @param windows the specification of the `JoinWindows`
* @param streamJoin an implicit `StreamJoin` instance that defines the serdes to be used to serialize/deserialize
* inputs and outputs of the joined streams. Instead of `StreamJoin`, the user can also supply
* key serde, value serde and other value serde in implicit scope and they will be
* converted to the instance of `Stream` through implicit conversion. The `StreamJoin` instance can
* also name the repartition topic (if required), the state stores for the join, and the join
* processor node.
* @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`,
* one for each matched record-pair with the same key and within the joining window intervals
* @see `org.apache.kafka.streams.kstream.KStream#join`
*/
def join[VO, VR](otherStream: KStream[K, VO])(
joiner: (V, VO) => VR,
windows: JoinWindows
)(implicit streamJoin: StreamJoined[K, V, VO]): KStream[K, VR] =
new KStream(inner.join[VO, VR](otherStream.inner, joiner.asValueJoiner, windows, streamJoin))
/**
* Join records of this stream with another [[KStream]]'s records using windowed left equi join with
* serializers and deserializers supplied by the implicit `StreamJoined` instance.
*
* @param otherStream the [[KStream]] to be joined with this stream
* @param joiner a function that computes the join result for a pair of matching records
* @param windows the specification of the `JoinWindows`
* @param streamJoin an implicit `StreamJoin` instance that defines the serdes to be used to serialize/deserialize
* inputs and outputs of the joined streams. Instead of `StreamJoin`, the user can also supply
* key serde, value serde and other value serde in implicit scope and they will be
* converted to the instance of `Stream` through implicit conversion. The `StreamJoin` instance can
* also name the repartition topic (if required), the state stores for the join, and the join
* processor node.
* @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`,
* one for each matched record-pair with the same key and within the joining window intervals
* @see `org.apache.kafka.streams.kstream.KStream#leftJoin`
*/
def leftJoin[VO, VR](otherStream: KStream[K, VO])(
joiner: (V, VO) => VR,
windows: JoinWindows
)(implicit streamJoin: StreamJoined[K, V, VO]): KStream[K, VR] =
new KStream(inner.leftJoin[VO, VR](otherStream.inner, joiner.asValueJoiner, windows, streamJoin))
/**
* Join records of this stream with another [[KStream]]'s records using windowed outer equi join with
* serializers and deserializers supplied by the implicit `Joined` instance.
*
* @param otherStream the [[KStream]] to be joined with this stream
* @param joiner a function that computes the join result for a pair of matching records
* @param windows the specification of the `JoinWindows`
* @param streamJoin an implicit `StreamJoin` instance that defines the serdes to be used to serialize/deserialize
* inputs and outputs of the joined streams. Instead of `StreamJoin`, the user can also supply
* key serde, value serde and other value serde in implicit scope and they will be
* converted to the instance of `Stream` through implicit conversion. The `StreamJoin` instance can
* also name the repartition topic (if required), the state stores for the join, and the join
* processor node.
* @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`,
* one for each matched record-pair with the same key and within the joining window intervals
* @see `org.apache.kafka.streams.kstream.KStream#outerJoin`
*/
def outerJoin[VO, VR](otherStream: KStream[K, VO])(
joiner: (V, VO) => VR,
windows: JoinWindows
)(implicit streamJoin: StreamJoined[K, V, VO]): KStream[K, VR] =
new KStream(inner.outerJoin[VO, VR](otherStream.inner, joiner.asValueJoiner, windows, streamJoin))
/**
* Join records of this stream with another [[KTable]]'s records using inner equi join with
* serializers and deserializers supplied by the implicit `Joined` instance.
*
* @param table the [[KTable]] to be joined with this stream
* @param joiner a function that computes the join result for a pair of matching records
* @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize
* inputs and outputs of the joined streams. Instead of `Joined`, the user can also supply
* key serde, value serde and other value serde in implicit scope and they will be
* converted to the instance of `Joined` through implicit conversion
* @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`,
* one for each matched record-pair with the same key
* @see `org.apache.kafka.streams.kstream.KStream#join`
*/
def join[VT, VR](table: KTable[K, VT])(joiner: (V, VT) => VR)(implicit joined: Joined[K, V, VT]): KStream[K, VR] =
new KStream(inner.join[VT, VR](table.inner, joiner.asValueJoiner, joined))
/**
* Join records of this stream with another [[KTable]]'s records using left equi join with
* serializers and deserializers supplied by the implicit `Joined` instance.
*
* @param table the [[KTable]] to be joined with this stream
* @param joiner a function that computes the join result for a pair of matching records
* @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize
* inputs and outputs of the joined streams. Instead of `Joined`, the user can also supply
* key serde, value serde and other value serde in implicit scope and they will be
* converted to the instance of `Joined` through implicit conversion
* @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`,
* one for each matched record-pair with the same key
* @see `org.apache.kafka.streams.kstream.KStream#leftJoin`
*/
def leftJoin[VT, VR](table: KTable[K, VT])(joiner: (V, VT) => VR)(implicit joined: Joined[K, V, VT]): KStream[K, VR] =
new KStream(inner.leftJoin[VT, VR](table.inner, joiner.asValueJoiner, joined))
/**
* Join records of this stream with `GlobalKTable`'s records using non-windowed inner equi join.
*
* @param globalKTable the `GlobalKTable` to be joined with this stream
* @param keyValueMapper a function used to map from the (key, value) of this stream
* to the key of the `GlobalKTable`
* @param joiner a function that computes the join result for a pair of matching records
* @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`,
* one output for each input [[KStream]] record
* @see `org.apache.kafka.streams.kstream.KStream#join`
*/
def join[GK, GV, RV](globalKTable: GlobalKTable[GK, GV])(
keyValueMapper: (K, V) => GK,
joiner: (V, GV) => RV
): KStream[K, RV] =
new KStream(
inner.join[GK, GV, RV](
globalKTable,
((k: K, v: V) => keyValueMapper(k, v)).asKeyValueMapper,
((v: V, gv: GV) => joiner(v, gv)).asValueJoiner
)
)
/**
* Join records of this stream with `GlobalKTable`'s records using non-windowed left equi join.
*
* @param globalKTable the `GlobalKTable` to be joined with this stream
* @param keyValueMapper a function used to map from the (key, value) of this stream
* to the key of the `GlobalKTable`
* @param joiner a function that computes the join result for a pair of matching records
* @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`,
* one output for each input [[KStream]] record
* @see `org.apache.kafka.streams.kstream.KStream#leftJoin`
*/
def leftJoin[GK, GV, RV](globalKTable: GlobalKTable[GK, GV])(
keyValueMapper: (K, V) => GK,
joiner: (V, GV) => RV
): KStream[K, RV] =
new KStream(inner.leftJoin[GK, GV, RV](globalKTable, keyValueMapper.asKeyValueMapper, joiner.asValueJoiner))
/**
* Merge this stream and the given stream into one larger stream.
* <p>
* There is no ordering guarantee between records from this `KStream` and records from the provided `KStream`
* in the merged stream. Relative order is preserved within each input stream though (ie, records within
* one input stream are processed in order).
*
* @param stream a stream which is to be merged into this stream
* @return a merged stream containing all records from this and the provided [[KStream]]
* @see `org.apache.kafka.streams.kstream.KStream#merge`
*/
def merge(stream: KStream[K, V]): KStream[K, V] =
new KStream(inner.merge(stream.inner))
/**
* Perform an action on each record of `KStream`.
* <p>
* Peek is a non-terminal operation that triggers a side effect (such as logging or statistics collection)
* and returns an unchanged stream.
*
* @param action an action to perform on each record
* @see `org.apache.kafka.streams.kstream.KStream#peek`
*/
def peek(action: (K, V) => Unit): KStream[K, V] =
new KStream(inner.peek(action.asForeachAction))
}

View File

@@ -0,0 +1,371 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.common.utils.Bytes
import org.apache.kafka.streams.kstream.{ValueJoiner, ValueTransformerWithKeySupplier, KTable => KTableJ}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{
FunctionFromFunction,
KeyValueMapperFromFunction,
MapperFromFunction,
PredicateFromFunction,
ValueMapperFromFunction,
ValueMapperWithKeyFromFunction
}
import org.apache.kafka.streams.state.KeyValueStore
/**
* Wraps the Java class [[org.apache.kafka.streams.kstream.KTable]] and delegates method calls to the underlying Java object.
*
* @tparam K Type of keys
* @tparam V Type of values
* @param inner The underlying Java abstraction for KTable
*
* @see `org.apache.kafka.streams.kstream.KTable`
*/
class KTable[K, V](val inner: KTableJ[K, V]) {
/**
* Create a new [[KTable]] that consists all records of this [[KTable]] which satisfies the given
* predicate
*
* @param predicate a filter that is applied to each record
* @return a [[KTable]] that contains only those records that satisfy the given predicate
* @see `org.apache.kafka.streams.kstream.KTable#filter`
*/
def filter(predicate: (K, V) => Boolean): KTable[K, V] =
new KTable(inner.filter(predicate.asPredicate))
/**
* Create a new [[KTable]] that consists all records of this [[KTable]] which satisfies the given
* predicate
*
* @param predicate a filter that is applied to each record
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains only those records that satisfy the given predicate
* @see `org.apache.kafka.streams.kstream.KTable#filter`
*/
def filter(predicate: (K, V) => Boolean, materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
new KTable(inner.filter(predicate.asPredicate, materialized))
/**
* Create a new [[KTable]] that consists all records of this [[KTable]] which do <em>not</em> satisfy the given
* predicate
*
* @param predicate a filter that is applied to each record
* @return a [[KTable]] that contains only those records that do <em>not</em> satisfy the given predicate
* @see `org.apache.kafka.streams.kstream.KTable#filterNot`
*/
def filterNot(predicate: (K, V) => Boolean): KTable[K, V] =
new KTable(inner.filterNot(predicate.asPredicate))
/**
* Create a new [[KTable]] that consists all records of this [[KTable]] which do <em>not</em> satisfy the given
* predicate
*
* @param predicate a filter that is applied to each record
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains only those records that do <em>not</em> satisfy the given predicate
* @see `org.apache.kafka.streams.kstream.KTable#filterNot`
*/
def filterNot(predicate: (K, V) => Boolean, materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
new KTable(inner.filterNot(predicate.asPredicate, materialized))
/**
* Create a new [[KTable]] by transforming the value of each record in this [[KTable]] into a new value
* (with possible new type) in the new [[KTable]].
* <p>
* The provided `mapper`, a function `V => VR` is applied to each input record value and computes a new value for it
*
* @param mapper, a function `V => VR` that computes a new output value
* @return a [[KTable]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KTable#mapValues`
*/
def mapValues[VR](mapper: V => VR): KTable[K, VR] =
new KTable(inner.mapValues[VR](mapper.asValueMapper))
/**
* Create a new [[KTable]] by transforming the value of each record in this [[KTable]] into a new value
* (with possible new type) in the new [[KTable]].
* <p>
* The provided `mapper`, a function `V => VR` is applied to each input record value and computes a new value for it
*
* @param mapper, a function `V => VR` that computes a new output value
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KTable#mapValues`
*/
def mapValues[VR](mapper: V => VR, materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] =
new KTable(inner.mapValues[VR](mapper.asValueMapper, materialized))
/**
* Create a new [[KTable]] by transforming the value of each record in this [[KTable]] into a new value
* (with possible new type) in the new [[KTable]].
* <p>
* The provided `mapper`, a function `(K, V) => VR` is applied to each input record value and computes a new value for it
*
* @param mapper, a function `(K, V) => VR` that computes a new output value
* @return a [[KTable]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KTable#mapValues`
*/
def mapValues[VR](mapper: (K, V) => VR): KTable[K, VR] =
new KTable(inner.mapValues[VR](mapper.asValueMapperWithKey))
/**
* Create a new [[KTable]] by transforming the value of each record in this [[KTable]] into a new value
* (with possible new type) in the new [[KTable]].
* <p>
* The provided `mapper`, a function `(K, V) => VR` is applied to each input record value and computes a new value for it
*
* @param mapper, a function `(K, V) => VR` that computes a new output value
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KTable#mapValues`
*/
def mapValues[VR](mapper: (K, V) => VR, materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] =
new KTable(inner.mapValues[VR](mapper.asValueMapperWithKey))
/**
* Convert this changelog stream to a [[KStream]].
*
* @return a [[KStream]] that contains the same records as this [[KTable]]
* @see `org.apache.kafka.streams.kstream.KTable#toStream`
*/
def toStream: KStream[K, V] =
new KStream(inner.toStream)
/**
* Convert this changelog stream to a [[KStream]] using the given key/value mapper to select the new key
*
* @param mapper a function that computes a new key for each record
* @return a [[KStream]] that contains the same records as this [[KTable]]
* @see `org.apache.kafka.streams.kstream.KTable#toStream`
*/
def toStream[KR](mapper: (K, V) => KR): KStream[KR, V] =
new KStream(inner.toStream[KR](mapper.asKeyValueMapper))
/**
* Suppress some updates from this changelog stream, determined by the supplied [[Suppressed]] configuration.
*
* This controls what updates downstream table and stream operations will receive.
*
* @param suppressed Configuration object determining what, if any, updates to suppress.
* @return A new KTable with the desired suppression characteristics.
* @see `org.apache.kafka.streams.kstream.KTable#suppress`
*/
def suppress(suppressed: org.apache.kafka.streams.kstream.Suppressed[_ >: K]): KTable[K, V] =
new KTable(inner.suppress(suppressed))
/**
* Create a new `KTable` by transforming the value of each record in this `KTable` into a new value, (with possibly new type).
* Transform the value of each input record into a new value (with possible new type) of the output record.
* A `ValueTransformerWithKey` (provided by the given `ValueTransformerWithKeySupplier`) is applied to each input
* record value and computes a new value for it.
* This is similar to `#mapValues(ValueMapperWithKey)`, but more flexible, allowing access to additional state-stores,
* and to the `ProcessorContext`.
* If the downstream topology uses aggregation functions, (e.g. `KGroupedTable#reduce`, `KGroupedTable#aggregate`, etc),
* care must be taken when dealing with state, (either held in state-stores or transformer instances), to ensure correct
* aggregate results.
* In contrast, if the resulting KTable is materialized, (cf. `#transformValues(ValueTransformerWithKeySupplier, Materialized, String...)`),
* such concerns are handled for you.
* In order to assign a state, the state must be created and registered
* beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer`
*
* @param valueTransformerWithKeySupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey`.
* At least one transformer instance will be created per streaming task.
* Transformer implementations doe not need to be thread-safe.
* @param stateStoreNames the names of the state stores used by the processor
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
*/
def transformValues[VR](valueTransformerWithKeySupplier: ValueTransformerWithKeySupplier[K, V, VR],
stateStoreNames: String*): KTable[K, VR] =
new KTable(inner.transformValues[VR](valueTransformerWithKeySupplier, stateStoreNames: _*))
/**
* Create a new `KTable` by transforming the value of each record in this `KTable` into a new value, (with possibly new type).
* A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input
* record value and computes a new value for it.
* This is similar to `#mapValues(ValueMapperWithKey)`, but more flexible, allowing stateful, rather than stateless,
* record-by-record operation, access to additional state-stores, and access to the `ProcessorContext`.
* In order to assign a state, the state must be created and registered
* beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer`
* The resulting `KTable` is materialized into another state store (additional to the provided state store names)
* as specified by the user via `Materialized` parameter, and is queryable through its given name.
*
* @param valueTransformerWithKeySupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey`
* At least one transformer instance will be created per streaming task.
* Transformer implementations doe not need to be thread-safe.
* @param materialized an instance of `Materialized` used to describe how the state store of the
* resulting table should be materialized.
* @param stateStoreNames the names of the state stores used by the processor
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
*/
def transformValues[VR](valueTransformerWithKeySupplier: ValueTransformerWithKeySupplier[K, V, VR],
materialized: Materialized[K, VR, KeyValueStore[Bytes, Array[Byte]]],
stateStoreNames: String*): KTable[K, VR] =
new KTable(inner.transformValues[VR](valueTransformerWithKeySupplier, materialized, stateStoreNames: _*))
/**
* Re-groups the records of this [[KTable]] using the provided key/value mapper
* and `Serde`s as specified by `Grouped`.
*
* @param selector a function that computes a new grouping key and value to be aggregated
* @param grouped the `Grouped` instance used to specify `Serdes`
* @return a [[KGroupedTable]] that contains the re-grouped records of the original [[KTable]]
* @see `org.apache.kafka.streams.kstream.KTable#groupBy`
*/
def groupBy[KR, VR](selector: (K, V) => (KR, VR))(implicit grouped: Grouped[KR, VR]): KGroupedTable[KR, VR] =
new KGroupedTable(inner.groupBy(selector.asKeyValueMapper, grouped))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed inner equi join.
*
* @param other the other [[KTable]] to be joined with this [[KTable]]
* @param joiner a function that computes the join result for a pair of matching records
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
* @see `org.apache.kafka.streams.kstream.KTable#join`
*/
def join[VO, VR](other: KTable[K, VO])(joiner: (V, VO) => VR): KTable[K, VR] =
new KTable(inner.join[VO, VR](other.inner, joiner.asValueJoiner))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed inner equi join.
*
* @param other the other [[KTable]] to be joined with this [[KTable]]
* @param joiner a function that computes the join result for a pair of matching records
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
* @see `org.apache.kafka.streams.kstream.KTable#join`
*/
def join[VO, VR](other: KTable[K, VO], materialized: Materialized[K, VR, ByteArrayKeyValueStore])(
joiner: (V, VO) => VR
): KTable[K, VR] =
new KTable(inner.join[VO, VR](other.inner, joiner.asValueJoiner, materialized))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed left equi join.
*
* @param other the other [[KTable]] to be joined with this [[KTable]]
* @param joiner a function that computes the join result for a pair of matching records
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
* @see `org.apache.kafka.streams.kstream.KTable#leftJoin`
*/
def leftJoin[VO, VR](other: KTable[K, VO])(joiner: (V, VO) => VR): KTable[K, VR] =
new KTable(inner.leftJoin[VO, VR](other.inner, joiner.asValueJoiner))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed left equi join.
*
* @param other the other [[KTable]] to be joined with this [[KTable]]
* @param joiner a function that computes the join result for a pair of matching records
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
* @see `org.apache.kafka.streams.kstream.KTable#leftJoin`
*/
def leftJoin[VO, VR](other: KTable[K, VO], materialized: Materialized[K, VR, ByteArrayKeyValueStore])(
joiner: (V, VO) => VR
): KTable[K, VR] =
new KTable(inner.leftJoin[VO, VR](other.inner, joiner.asValueJoiner, materialized))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed outer equi join.
*
* @param other the other [[KTable]] to be joined with this [[KTable]]
* @param joiner a function that computes the join result for a pair of matching records
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
* @see `org.apache.kafka.streams.kstream.KTable#leftJoin`
*/
def outerJoin[VO, VR](other: KTable[K, VO])(joiner: (V, VO) => VR): KTable[K, VR] =
new KTable(inner.outerJoin[VO, VR](other.inner, joiner.asValueJoiner))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed outer equi join.
*
* @param other the other [[KTable]] to be joined with this [[KTable]]
* @param joiner a function that computes the join result for a pair of matching records
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
* @see `org.apache.kafka.streams.kstream.KTable#leftJoin`
*/
def outerJoin[VO, VR](other: KTable[K, VO], materialized: Materialized[K, VR, ByteArrayKeyValueStore])(
joiner: (V, VO) => VR
): KTable[K, VR] =
new KTable(inner.outerJoin[VO, VR](other.inner, joiner.asValueJoiner, materialized))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed inner join. Records from this
* table are joined according to the result of keyExtractor on the other KTable.
*
* @param other the other [[KTable]] to be joined with this [[KTable]], keyed on the value obtained from keyExtractor
* @param keyExtractor a function that extracts the foreign key from this table's value
* @param joiner a function that computes the join result for a pair of matching records
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
*/
def join[VR, KO, VO](other: KTable[KO, VO],
keyExtractor: Function[V, KO],
joiner: ValueJoiner[V, VO, VR],
materialized: Materialized[K, VR, KeyValueStore[Bytes, Array[Byte]]]): KTable[K, VR] =
new KTable(inner.join(other.inner, keyExtractor.asJavaFunction, joiner, materialized))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed left join. Records from this
* table are joined according to the result of keyExtractor on the other KTable.
*
* @param other the other [[KTable]] to be joined with this [[KTable]], keyed on the value obtained from keyExtractor
* @param keyExtractor a function that extracts the foreign key from this table's value
* @param joiner a function that computes the join result for a pair of matching records
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
*/
def leftJoin[VR, KO, VO](other: KTable[KO, VO],
keyExtractor: Function[V, KO],
joiner: ValueJoiner[V, VO, VR],
materialized: Materialized[K, VR, KeyValueStore[Bytes, Array[Byte]]]): KTable[K, VR] =
new KTable(inner.leftJoin(other.inner, keyExtractor.asJavaFunction, joiner, materialized))
/**
* Get the name of the local state store used that can be used to query this [[KTable]].
*
* @return the underlying state store name, or `null` if this [[KTable]] cannot be queried.
*/
def queryableStoreName: String =
inner.queryableStoreName
}

View File

@@ -0,0 +1,107 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.common.serialization.Serde
import org.apache.kafka.streams.kstream.{Materialized => MaterializedJ}
import org.apache.kafka.streams.processor.StateStore
import org.apache.kafka.streams.scala.{ByteArrayKeyValueStore, ByteArraySessionStore, ByteArrayWindowStore}
import org.apache.kafka.streams.state.{KeyValueBytesStoreSupplier, SessionBytesStoreSupplier, WindowBytesStoreSupplier}
object Materialized {
/**
* Materialize a [[StateStore]] with the provided key and value [[Serde]]s.
* An internal name will be used for the store.
*
* @tparam K key type
* @tparam V value type
* @tparam S store type
* @param keySerde the key [[Serde]] to use.
* @param valueSerde the value [[Serde]] to use.
* @return a new [[Materialized]] instance with the given key and value serdes
*/
def `with`[K, V, S <: StateStore](implicit keySerde: Serde[K], valueSerde: Serde[V]): MaterializedJ[K, V, S] =
MaterializedJ.`with`(keySerde, valueSerde)
/**
* Materialize a [[StateStore]] with the given name.
*
* @tparam K key type of the store
* @tparam V value type of the store
* @tparam S type of the [[StateStore]]
* @param storeName the name of the underlying [[org.apache.kafka.streams.scala.kstream.KTable]] state store;
* valid characters are ASCII alphanumerics, '.', '_' and '-'.
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @return a new [[Materialized]] instance with the given storeName
*/
def as[K, V, S <: StateStore](storeName: String)(implicit keySerde: Serde[K],
valueSerde: Serde[V]): MaterializedJ[K, V, S] =
MaterializedJ.as(storeName).withKeySerde(keySerde).withValueSerde(valueSerde)
/**
* Materialize a [[org.apache.kafka.streams.state.WindowStore]] using the provided [[WindowBytesStoreSupplier]].
*
* Important: Custom subclasses are allowed here, but they should respect the retention contract:
* Window stores are required to retain windows at least as long as (window size + window grace period).
* Stores constructed via [[org.apache.kafka.streams.state.Stores]] already satisfy this contract.
*
* @tparam K key type of the store
* @tparam V value type of the store
* @param supplier the [[WindowBytesStoreSupplier]] used to materialize the store
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @return a new [[Materialized]] instance with the given supplier
*/
def as[K, V](supplier: WindowBytesStoreSupplier)(implicit keySerde: Serde[K],
valueSerde: Serde[V]): MaterializedJ[K, V, ByteArrayWindowStore] =
MaterializedJ.as(supplier).withKeySerde(keySerde).withValueSerde(valueSerde)
/**
* Materialize a [[org.apache.kafka.streams.state.SessionStore]] using the provided [[SessionBytesStoreSupplier]].
*
* Important: Custom subclasses are allowed here, but they should respect the retention contract:
* Session stores are required to retain windows at least as long as (session inactivity gap + session grace period).
* Stores constructed via [[org.apache.kafka.streams.state.Stores]] already satisfy this contract.
*
* @tparam K key type of the store
* @tparam V value type of the store
* @param supplier the [[SessionBytesStoreSupplier]] used to materialize the store
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @return a new [[Materialized]] instance with the given supplier
*/
def as[K, V](supplier: SessionBytesStoreSupplier)(implicit keySerde: Serde[K],
valueSerde: Serde[V]): MaterializedJ[K, V, ByteArraySessionStore] =
MaterializedJ.as(supplier).withKeySerde(keySerde).withValueSerde(valueSerde)
/**
* Materialize a [[org.apache.kafka.streams.state.KeyValueStore]] using the provided [[KeyValueBytesStoreSupplier]].
*
* @tparam K key type of the store
* @tparam V value type of the store
* @param supplier the [[KeyValueBytesStoreSupplier]] used to materialize the store
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @return a new [[Materialized]] instance with the given supplier
*/
def as[K, V](
supplier: KeyValueBytesStoreSupplier
)(implicit keySerde: Serde[K], valueSerde: Serde[V]): MaterializedJ[K, V, ByteArrayKeyValueStore] =
MaterializedJ.as(supplier).withKeySerde(keySerde).withValueSerde(valueSerde)
}

View File

@@ -0,0 +1,59 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.common.serialization.Serde
import org.apache.kafka.streams.kstream.{Produced => ProducedJ}
import org.apache.kafka.streams.processor.StreamPartitioner
object Produced {
/**
* Create a Produced instance with provided keySerde and valueSerde.
*
* @tparam K key type
* @tparam V value type
* @param keySerde Serde to use for serializing the key
* @param valueSerde Serde to use for serializing the value
* @return A new [[Produced]] instance configured with keySerde and valueSerde
* @see KStream#through(String, Produced)
* @see KStream#to(String, Produced)
*/
def `with`[K, V](implicit keySerde: Serde[K], valueSerde: Serde[V]): ProducedJ[K, V] =
ProducedJ.`with`(keySerde, valueSerde)
/**
* Create a Produced instance with provided keySerde, valueSerde, and partitioner.
*
* @tparam K key type
* @tparam V value type
* @param partitioner the function used to determine how records are distributed among partitions of the topic,
* if not specified and `keySerde` provides a
* [[org.apache.kafka.streams.kstream.internals.WindowedSerializer]] for the key
* [[org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner]] will be
* used&mdash;otherwise [[org.apache.kafka.clients.producer.internals.DefaultPartitioner]]
* will be used
* @param keySerde Serde to use for serializing the key
* @param valueSerde Serde to use for serializing the value
* @return A new [[Produced]] instance configured with keySerde, valueSerde, and partitioner
* @see KStream#through(String, Produced)
* @see KStream#to(String, Produced)
*/
def `with`[K, V](partitioner: StreamPartitioner[K, V])(implicit keySerde: Serde[K],
valueSerde: Serde[V]): ProducedJ[K, V] =
ProducedJ.`with`(keySerde, valueSerde, partitioner)
}

View File

@@ -0,0 +1,48 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.streams.kstream.{SessionWindowedCogroupedKStream => SessionWindowedCogroupedKStreamJ, Windowed}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{InitializerFromFunction, MergerFromFunction}
/**
* Wraps the Java class SessionWindowedCogroupedKStream and delegates method calls to the underlying Java object.
*
* @tparam K Type of keys
* @tparam V Type of values
* @param inner The underlying Java abstraction for SessionWindowedCogroupedKStream
*
* @see `org.apache.kafka.streams.kstream.SessionWindowedCogroupedKStream`
*/
class SessionWindowedCogroupedKStream[K, V](val inner: SessionWindowedCogroupedKStreamJ[K, V]) {
/**
* Aggregate the values of records in this stream by the grouped key and defined `SessionWindows`.
*
* @param initializer the initializer function
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a windowed [[KTable]] that contains "update" records with unmodified keys, and values that represent
* the latest (rolling) aggregate for each key within a window
* @see `org.apache.kafka.streams.kstream.SessionWindowedCogroupedKStream#aggregate`
*/
def aggregate(initializer: => V, merger: (K, V, V) => V)(
implicit materialized: Materialized[K, V, ByteArraySessionStore]
): KTable[Windowed[K], V] =
new KTable(inner.aggregate((() => initializer).asInitializer, merger.asMerger, materialized))
}

View File

@@ -0,0 +1,95 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.streams.kstream.internals.KTableImpl
import org.apache.kafka.streams.kstream.{KTable => KTableJ, SessionWindowedKStream => SessionWindowedKStreamJ, Windowed}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{
AggregatorFromFunction,
InitializerFromFunction,
MergerFromFunction,
ReducerFromFunction,
ValueMapperFromFunction
}
/**
* Wraps the Java class SessionWindowedKStream and delegates method calls to the underlying Java object.
*
* @tparam K Type of keys
* @tparam V Type of values
* @param inner The underlying Java abstraction for SessionWindowedKStream
*
* @see `org.apache.kafka.streams.kstream.SessionWindowedKStream`
*/
class SessionWindowedKStream[K, V](val inner: SessionWindowedKStreamJ[K, V]) {
/**
* Aggregate the values of records in this stream by the grouped key and defined `SessionWindows`.
*
* @param initializer the initializer function
* @param aggregator the aggregator function
* @param merger the merger function
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a windowed [[KTable]] that contains "update" records with unmodified keys, and values that represent
* the latest (rolling) aggregate for each key within a window
* @see `org.apache.kafka.streams.kstream.SessionWindowedKStream#aggregate`
*/
def aggregate[VR](initializer: => VR)(aggregator: (K, V, VR) => VR, merger: (K, VR, VR) => VR)(
implicit materialized: Materialized[K, VR, ByteArraySessionStore]
): KTable[Windowed[K], VR] =
new KTable(
inner.aggregate((() => initializer).asInitializer, aggregator.asAggregator, merger.asMerger, materialized)
)
/**
* Count the number of records in this stream by the grouped key into `SessionWindows`.
*
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a windowed [[KTable]] that contains "update" records with unmodified keys and `Long` values
* that represent the latest (rolling) count (i.e., number of records) for each key within a window
* @see `org.apache.kafka.streams.kstream.SessionWindowedKStream#count`
*/
def count()(implicit materialized: Materialized[K, Long, ByteArraySessionStore]): KTable[Windowed[K], Long] = {
val javaCountTable: KTableJ[Windowed[K], java.lang.Long] =
inner.count(materialized.asInstanceOf[Materialized[K, java.lang.Long, ByteArraySessionStore]])
val tableImpl = javaCountTable.asInstanceOf[KTableImpl[Windowed[K], ByteArraySessionStore, java.lang.Long]]
new KTable(
javaCountTable.mapValues[Long](
((l: java.lang.Long) => Long2long(l)).asValueMapper,
Materialized.`with`[Windowed[K], Long, ByteArrayKeyValueStore](tableImpl.keySerde(), Serdes.Long)
)
)
}
/**
* Combine values of this stream by the grouped key into `SessionWindows`.
*
* @param reducer a reducer function that computes a new aggregate result.
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a windowed [[KTable]] that contains "update" records with unmodified keys, and values that represent
* the latest (rolling) aggregate for each key within a window
* @see `org.apache.kafka.streams.kstream.SessionWindowedKStream#reduce`
*/
def reduce(reducer: (V, V) => V)(
implicit materialized: Materialized[K, V, ByteArraySessionStore]
): KTable[Windowed[K], V] =
new KTable(inner.reduce(reducer.asReducer, materialized))
}

View File

@@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.common.serialization.Serde
import org.apache.kafka.streams.kstream.{StreamJoined => StreamJoinedJ}
import org.apache.kafka.streams.state.WindowBytesStoreSupplier
object StreamJoined {
/**
* Create an instance of [[StreamJoined]] with key, value, and otherValue [[Serde]]
* instances.
* `null` values are accepted and will be replaced by the default serdes as defined in config.
*
* @tparam K key type
* @tparam V value type
* @tparam VO other value type
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @param otherValueSerde the otherValue serde to use. If `null` the default value serde from config will be used
* @return new [[StreamJoined]] instance with the provided serdes
*/
def `with`[K, V, VO](implicit keySerde: Serde[K],
valueSerde: Serde[V],
otherValueSerde: Serde[VO]): StreamJoinedJ[K, V, VO] =
StreamJoinedJ.`with`(keySerde, valueSerde, otherValueSerde)
/**
* Create an instance of [[StreamJoined]] with store suppliers for the calling stream
* and the other stream. Also adds the key, value, and otherValue [[Serde]]
* instances.
* `null` values are accepted and will be replaced by the default serdes as defined in config.
*
* @tparam K key type
* @tparam V value type
* @tparam VO other value type
* @param supplier store supplier to use
* @param otherSupplier other store supplier to use
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @param otherValueSerde the otherValue serde to use. If `null` the default value serde from config will be used
* @return new [[StreamJoined]] instance with the provided store suppliers and serdes
*/
def `with`[K, V, VO](
supplier: WindowBytesStoreSupplier,
otherSupplier: WindowBytesStoreSupplier
)(implicit keySerde: Serde[K], valueSerde: Serde[V], otherValueSerde: Serde[VO]): StreamJoinedJ[K, V, VO] =
StreamJoinedJ
.`with`(supplier, otherSupplier)
.withKeySerde(keySerde)
.withValueSerde(valueSerde)
.withOtherValueSerde(otherValueSerde)
/**
* Create an instance of [[StreamJoined]] with the name used for naming
* the state stores involved in the join. Also adds the key, value, and otherValue [[Serde]]
* instances.
* `null` values are accepted and will be replaced by the default serdes as defined in config.
*
* @tparam K key type
* @tparam V value type
* @tparam VO other value type
* @param storeName the name to use as a base name for the state stores of the join
* @param keySerde the key serde to use.
* @param valueSerde the value serde to use.
* @param otherValueSerde the otherValue serde to use. If `null` the default value serde from config will be used
* @return new [[StreamJoined]] instance with the provided store suppliers and serdes
*/
def as[K, V, VO](
storeName: String
)(implicit keySerde: Serde[K], valueSerde: Serde[V], otherValueSerde: Serde[VO]): StreamJoinedJ[K, V, VO] =
StreamJoinedJ.as(storeName).withKeySerde(keySerde).withValueSerde(valueSerde).withOtherValueSerde(otherValueSerde)
}

View File

@@ -0,0 +1,128 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import java.time.Duration
import org.apache.kafka.streams.kstream.{Windowed, Suppressed => SupressedJ}
import org.apache.kafka.streams.kstream.Suppressed.{
EagerBufferConfig,
StrictBufferConfig,
BufferConfig => BufferConfigJ
}
import org.apache.kafka.streams.kstream.internals.suppress.{
EagerBufferConfigImpl,
FinalResultsSuppressionBuilder,
StrictBufferConfigImpl,
SuppressedInternal
}
/**
* Duplicates the static factory methods inside the Java interface [[org.apache.kafka.streams.kstream.Suppressed]].
*
* This was required for compatibility w/ Scala 2.11 + Java 1.8 because the Scala 2.11 compiler doesn't support the use
* of static methods inside Java interfaces. We have since dropped Scala 2.11 support.
*/
@deprecated(message = "Use org.apache.kafka.streams.kstream.Suppressed", since = "2.5")
object Suppressed {
/**
* Configure the suppression to emit only the "final results" from the window.
*
* By default all Streams operators emit results whenever new results are available.
* This includes windowed operations.
*
* This configuration will instead emit just one result per key for each window, guaranteeing
* to deliver only the final result. This option is suitable for use cases in which the business logic
* requires a hard guarantee that only the final result is propagated. For example, sending alerts.
*
* To accomplish this, the operator will buffer events from the window until the window close (that is,
* until the end-time passes, and additionally until the grace period expires). Since windowed operators
* are required to reject late events for a window whose grace period is expired, there is an additional
* guarantee that the final results emitted from this suppression will match any queriable state upstream.
*
* @param bufferConfig A configuration specifying how much space to use for buffering intermediate results.
* This is required to be a "strict" config, since it would violate the "final results"
* property to emit early and then issue an update later.
* @tparam K The [[Windowed]] key type for the KTable to apply this suppression to.
* @return a "final results" mode suppression configuration
* @see [[org.apache.kafka.streams.kstream.Suppressed.untilTimeLimit]]
*/
def untilWindowCloses[K](bufferConfig: StrictBufferConfig): SupressedJ[Windowed[K]] =
new FinalResultsSuppressionBuilder[Windowed[K]](null, bufferConfig)
/**
* Configure the suppression to wait `timeToWaitForMoreEvents` amount of time after receiving a record
* before emitting it further downstream. If another record for the same key arrives in the mean time, it replaces
* the first record in the buffer but does <em>not</em> re-start the timer.
*
* @param timeToWaitForMoreEvents The amount of time to wait, per record, for new events.
* @param bufferConfig A configuration specifying how much space to use for buffering intermediate results.
* @tparam K The key type for the KTable to apply this suppression to.
* @return a suppression configuration
* @see [[org.apache.kafka.streams.kstream.Suppressed.untilTimeLimit]]
*/
def untilTimeLimit[K](timeToWaitForMoreEvents: Duration, bufferConfig: BufferConfigJ[_]): SupressedJ[K] =
new SuppressedInternal[K](null, timeToWaitForMoreEvents, bufferConfig, null, false)
/**
* Duplicates the static factory methods inside the Java interface
* [[org.apache.kafka.streams.kstream.Suppressed.BufferConfig]].
*/
object BufferConfig {
/**
* Create a size-constrained buffer in terms of the maximum number of keys it will store.
*
* @param recordLimit maximum number of keys to buffer.
* @return size-constrained buffer in terms of the maximum number of keys it will store.
* @see [[org.apache.kafka.streams.kstream.Suppressed.BufferConfig.maxRecords]]
*/
def maxRecords(recordLimit: Long): EagerBufferConfig =
new EagerBufferConfigImpl(recordLimit, Long.MaxValue)
/**
* Create a size-constrained buffer in terms of the maximum number of bytes it will use.
*
* @param byteLimit maximum number of bytes to buffer.
* @return size-constrained buffer in terms of the maximum number of bytes it will use.
* @see [[org.apache.kafka.streams.kstream.Suppressed.BufferConfig.maxBytes]]
*/
def maxBytes(byteLimit: Long): EagerBufferConfig =
new EagerBufferConfigImpl(Long.MaxValue, byteLimit)
/**
* Create a buffer unconstrained by size (either keys or bytes).
*
* As a result, the buffer will consume as much memory as it needs, dictated by the time bound.
*
* If there isn't enough heap available to meet the demand, the application will encounter an
* [[OutOfMemoryError]] and shut down (not guaranteed to be a graceful exit). Also, note that
* JVM processes under extreme memory pressure may exhibit poor GC behavior.
*
* This is a convenient option if you doubt that your buffer will be that large, but also don't
* wish to pick particular constraints, such as in testing.
*
* This buffer is "strict" in the sense that it will enforce the time bound or crash.
* It will never emit early.
*
* @return a buffer unconstrained by size (either keys or bytes).
* @see [[org.apache.kafka.streams.kstream.Suppressed.BufferConfig.unbounded]]
*/
def unbounded(): StrictBufferConfig = new StrictBufferConfigImpl()
}
}

View File

@@ -0,0 +1,48 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.streams.kstream.{TimeWindowedCogroupedKStream => TimeWindowedCogroupedKStreamJ, Windowed}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.InitializerFromFunction
/**
* Wraps the Java class TimeWindowedCogroupedKStream and delegates method calls to the underlying Java object.
*
* @tparam K Type of keys
* @tparam V Type of values
* @param inner The underlying Java abstraction for TimeWindowedCogroupedKStream
*
* @see `org.apache.kafka.streams.kstream.TimeWindowedCogroupedKStream`
*/
class TimeWindowedCogroupedKStream[K, V](val inner: TimeWindowedCogroupedKStreamJ[K, V]) {
/**
* Aggregate the values of records in these streams by the grouped key and defined window.
*
* @param initializer an initializer function that computes an initial intermediate aggregation result
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the latest
* (rolling) aggregate for each key
* @see `org.apache.kafka.streams.kstream.TimeWindowedCogroupedKStream#aggregate`
*/
def aggregate(initializer: => V)(
implicit materialized: Materialized[K, V, ByteArrayWindowStore]
): KTable[Windowed[K], V] =
new KTable(inner.aggregate((() => initializer).asInitializer, materialized))
}

View File

@@ -0,0 +1,91 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
package kstream
import org.apache.kafka.streams.kstream.internals.KTableImpl
import org.apache.kafka.streams.kstream.{KTable => KTableJ, TimeWindowedKStream => TimeWindowedKStreamJ, Windowed}
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{
AggregatorFromFunction,
InitializerFromFunction,
ReducerFromFunction,
ValueMapperFromFunction
}
/**
* Wraps the Java class TimeWindowedKStream and delegates method calls to the underlying Java object.
*
* @tparam K Type of keys
* @tparam V Type of values
* @param inner The underlying Java abstraction for TimeWindowedKStream
*
* @see `org.apache.kafka.streams.kstream.TimeWindowedKStream`
*/
class TimeWindowedKStream[K, V](val inner: TimeWindowedKStreamJ[K, V]) {
/**
* Aggregate the values of records in this stream by the grouped key.
*
* @param initializer an initializer function that computes an initial intermediate aggregation result
* @param aggregator an aggregator function that computes a new aggregate result
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the
* latest (rolling) aggregate for each key
* @see `org.apache.kafka.streams.kstream.TimeWindowedKStream#aggregate`
*/
def aggregate[VR](initializer: => VR)(aggregator: (K, V, VR) => VR)(
implicit materialized: Materialized[K, VR, ByteArrayWindowStore]
): KTable[Windowed[K], VR] =
new KTable(inner.aggregate((() => initializer).asInitializer, aggregator.asAggregator, materialized))
/**
* Count the number of records in this stream by the grouped key and the defined windows.
*
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys and `Long` values that
* represent the latest (rolling) count (i.e., number of records) for each key
* @see `org.apache.kafka.streams.kstream.TimeWindowedKStream#count`
*/
def count()(implicit materialized: Materialized[K, Long, ByteArrayWindowStore]): KTable[Windowed[K], Long] = {
val javaCountTable: KTableJ[Windowed[K], java.lang.Long] =
inner.count(materialized.asInstanceOf[Materialized[K, java.lang.Long, ByteArrayWindowStore]])
val tableImpl = javaCountTable.asInstanceOf[KTableImpl[Windowed[K], ByteArrayWindowStore, java.lang.Long]]
new KTable(
javaCountTable.mapValues[Long](
((l: java.lang.Long) => Long2long(l)).asValueMapper,
Materialized.`with`[Windowed[K], Long, ByteArrayKeyValueStore](tableImpl.keySerde(), Serdes.Long)
)
)
}
/**
* Combine the values of records in this stream by the grouped key.
*
* @param reducer a function that computes a new aggregate result
* @param materialized an instance of `Materialized` used to materialize a state store.
* @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the
* latest (rolling) aggregate for each key
* @see `org.apache.kafka.streams.kstream.TimeWindowedKStream#reduce`
*/
def reduce(reducer: (V, V) => V)(
implicit materialized: Materialized[K, V, ByteArrayWindowStore]
): KTable[Windowed[K], V] =
new KTable(inner.reduce(reducer.asReducer, materialized))
}

View File

@@ -0,0 +1,28 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import org.apache.kafka.streams.processor.StateStore
package object kstream {
type Materialized[K, V, S <: StateStore] = org.apache.kafka.streams.kstream.Materialized[K, V, S]
type Grouped[K, V] = org.apache.kafka.streams.kstream.Grouped[K, V]
type Consumed[K, V] = org.apache.kafka.streams.kstream.Consumed[K, V]
type Produced[K, V] = org.apache.kafka.streams.kstream.Produced[K, V]
type Joined[K, V, VO] = org.apache.kafka.streams.kstream.Joined[K, V, VO]
type StreamJoined[K, V, VO] = org.apache.kafka.streams.kstream.StreamJoined[K, V, VO]
}

View File

@@ -0,0 +1,29 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams
import org.apache.kafka.streams.state.{KeyValueStore, SessionStore, WindowStore}
import org.apache.kafka.common.utils.Bytes
package object scala {
type ByteArrayKeyValueStore = KeyValueStore[Bytes, Array[Byte]]
type ByteArraySessionStore = SessionStore[Bytes, Array[Byte]]
type ByteArrayWindowStore = WindowStore[Bytes, Array[Byte]]
}

View File

@@ -0,0 +1,34 @@
# Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
# Copyright (C) 2017-2018 Alexis Seigneurin.
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# Set root logger level to DEBUG and its only appender to A1.
log4j.rootLogger=INFO, R
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
log4j.appender.R=org.apache.log4j.RollingFileAppender
log4j.appender.R.File=logs/kafka-streams-scala.log
log4j.appender.R.MaxFileSize=100KB
# Keep one backup file
log4j.appender.R.MaxBackupIndex=1
# A1 uses PatternLayout.
log4j.appender.R.layout=org.apache.log4j.PatternLayout
log4j.appender.R.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n

View File

@@ -0,0 +1,174 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import java.util.Properties
import org.apache.kafka.streams.{KafkaStreams, KeyValue, StreamsConfig}
import org.apache.kafka.streams.scala.ImplicitConversions._
import org.apache.kafka.streams.scala.kstream._
import org.apache.kafka.streams.scala.utils.StreamToTableJoinScalaIntegrationTestBase
import org.apache.kafka.test.IntegrationTest
import org.junit._
import org.junit.experimental.categories.Category
/**
* Test suite that does an example to demonstrate stream-table joins in Kafka Streams
* <p>
* The suite contains the test case using Scala APIs `testShouldCountClicksPerRegion` and the same test case using the
* Java APIs `testShouldCountClicksPerRegionJava`. The idea is to demonstrate that both generate the same result.
*/
@Category(Array(classOf[IntegrationTest]))
class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends StreamToTableJoinScalaIntegrationTestBase {
@Test def testShouldCountClicksPerRegion(): Unit = {
// DefaultSerdes brings into scope implicit serdes (mostly for primitives) that will set up all Grouped, Produced,
// Consumed and Joined instances. So all APIs below that accept Grouped, Produced, Consumed or Joined will
// get these instances automatically
import Serdes._
val streamsConfiguration: Properties = getStreamsConfiguration()
val builder = new StreamsBuilder()
val userClicksStream: KStream[String, Long] = builder.stream(userClicksTopic)
val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic)
// Compute the total per region by summing the individual click counts per region.
val clicksPerRegion: KTable[String, Long] =
userClicksStream
// Join the stream against the table.
.leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks))
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks>
.map((_, regionWithClicks) => regionWithClicks)
// Compute the total per region by summing the individual click counts per region.
.groupByKey
.reduce(_ + _)
// Write the (continuously updating) results to the output topic.
clicksPerRegion.toStream.to(outputTopic)
val streams: KafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration)
streams.start()
val actualClicksPerRegion: java.util.List[KeyValue[String, Long]] =
produceNConsume(userClicksTopic, userRegionsTopic, outputTopic)
Assert.assertTrue("Expected to process some data", !actualClicksPerRegion.isEmpty)
streams.close()
}
@Test
def testShouldCountClicksPerRegionWithNamedRepartitionTopic(): Unit = {
// DefaultSerdes brings into scope implicit serdes (mostly for primitives) that will set up all Grouped, Produced,
// Consumed and Joined instances. So all APIs below that accept Grouped, Produced, Consumed or Joined will
// get these instances automatically
import Serdes._
val streamsConfiguration: Properties = getStreamsConfiguration()
val builder = new StreamsBuilder()
val userClicksStream: KStream[String, Long] = builder.stream(userClicksTopic)
val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic)
// Compute the total per region by summing the individual click counts per region.
val clicksPerRegion: KTable[String, Long] =
userClicksStream
// Join the stream against the table.
.leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks))
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks>
.map((_, regionWithClicks) => regionWithClicks)
// Compute the total per region by summing the individual click counts per region.
.groupByKey
.reduce(_ + _)
// Write the (continuously updating) results to the output topic.
clicksPerRegion.toStream.to(outputTopic)
val streams: KafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration)
streams.start()
val actualClicksPerRegion: java.util.List[KeyValue[String, Long]] =
produceNConsume(userClicksTopic, userRegionsTopic, outputTopic)
Assert.assertTrue("Expected to process some data", !actualClicksPerRegion.isEmpty)
streams.close()
}
@Test
def testShouldCountClicksPerRegionJava(): Unit = {
import java.lang.{Long => JLong}
import org.apache.kafka.streams.kstream.{KStream => KStreamJ, KTable => KTableJ, _}
import org.apache.kafka.streams.{KafkaStreams => KafkaStreamsJ, StreamsBuilder => StreamsBuilderJ}
val streamsConfiguration: Properties = getStreamsConfiguration()
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String.getClass.getName)
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String.getClass.getName)
val builder: StreamsBuilderJ = new StreamsBuilderJ()
val userClicksStream: KStreamJ[String, JLong] =
builder.stream[String, JLong](userClicksTopicJ, Consumed.`with`(Serdes.String, Serdes.JavaLong))
val userRegionsTable: KTableJ[String, String] =
builder.table[String, String](userRegionsTopicJ, Consumed.`with`(Serdes.String, Serdes.String))
// Join the stream against the table.
val valueJoinerJ: ValueJoiner[JLong, String, (String, JLong)] =
(clicks: JLong, region: String) => (if (region == null) "UNKNOWN" else region, clicks)
val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream.leftJoin(
userRegionsTable,
valueJoinerJ,
Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String)
)
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks>
val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion.map { (_, regionWithClicks) =>
new KeyValue(regionWithClicks._1, regionWithClicks._2)
}
// Compute the total per region by summing the individual click counts per region.
val clicksPerRegion: KTableJ[String, JLong] = clicksByRegion
.groupByKey(Grouped.`with`(Serdes.String, Serdes.JavaLong))
.reduce((v1, v2) => v1 + v2)
// Write the (continuously updating) results to the output topic.
clicksPerRegion.toStream.to(outputTopicJ, Produced.`with`(Serdes.String, Serdes.JavaLong))
val streams = new KafkaStreamsJ(builder.build(), streamsConfiguration)
streams.start()
produceNConsume(userClicksTopicJ, userRegionsTopicJ, outputTopicJ)
streams.close()
}
}

View File

@@ -0,0 +1,467 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import java.time.Duration
import java.util
import java.util.{Locale, Properties}
import java.util.regex.Pattern
import org.apache.kafka.common.serialization.{Serdes => SerdesJ}
import org.apache.kafka.streams.kstream.{
Aggregator,
Initializer,
JoinWindows,
KeyValueMapper,
KGroupedStream => KGroupedStreamJ,
KStream => KStreamJ,
KTable => KTableJ,
Materialized => MaterializedJ,
Reducer,
StreamJoined => StreamJoinedJ,
Transformer,
ValueJoiner,
ValueMapper
}
import org.apache.kafka.streams.processor.{AbstractProcessor, ProcessorContext, ProcessorSupplier}
import org.apache.kafka.streams.scala.ImplicitConversions._
import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.scala.kstream._
import org.apache.kafka.streams.{KeyValue, StreamsConfig, TopologyDescription, StreamsBuilder => StreamsBuilderJ}
import org.junit.Assert._
import org.junit._
import _root_.scala.collection.JavaConverters._
/**
* Test suite that verifies that the topology built by the Java and Scala APIs match.
*/
class TopologyTest {
private val inputTopic = "input-topic"
private val userClicksTopic = "user-clicks-topic"
private val userRegionsTopic = "user-regions-topic"
private val pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS)
@Test
def shouldBuildIdenticalTopologyInJavaNScalaSimple(): Unit = {
// build the Scala topology
def getTopologyScala: TopologyDescription = {
import Serdes._
val streamBuilder = new StreamsBuilder
val textLines = streamBuilder.stream[String, String](inputTopic)
val _: KStream[String, String] = textLines.flatMapValues(v => pattern.split(v.toLowerCase))
streamBuilder.build().describe()
}
// build the Java topology
def getTopologyJava: TopologyDescription = {
val streamBuilder = new StreamsBuilderJ
val textLines = streamBuilder.stream[String, String](inputTopic)
val _: KStreamJ[String, String] = textLines.flatMapValues(s => pattern.split(s.toLowerCase).toIterable.asJava)
streamBuilder.build().describe()
}
// should match
assertEquals(getTopologyScala, getTopologyJava)
}
@Test
def shouldBuildIdenticalTopologyInJavaNScalaAggregate(): Unit = {
// build the Scala topology
def getTopologyScala: TopologyDescription = {
import org.apache.kafka.streams.scala.Serdes._
val streamBuilder = new StreamsBuilder
val textLines = streamBuilder.stream[String, String](inputTopic)
textLines
.flatMapValues(v => pattern.split(v.toLowerCase))
.groupBy((_, v) => v)
.count()
streamBuilder.build().describe()
}
// build the Java topology
def getTopologyJava: TopologyDescription = {
val streamBuilder = new StreamsBuilderJ
val textLines: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopic)
val splits: KStreamJ[String, String] =
textLines.flatMapValues(s => pattern.split(s.toLowerCase).toIterable.asJava)
val grouped: KGroupedStreamJ[String, String] = splits.groupBy((_, v) => v)
grouped.count()
streamBuilder.build().describe()
}
// should match
assertEquals(getTopologyScala, getTopologyJava)
}
@Test def shouldBuildIdenticalTopologyInJavaNScalaCogroupSimple(): Unit = {
// build the Scala topology
def getTopologyScala: TopologyDescription = {
import org.apache.kafka.streams.scala.Serdes._
val streamBuilder = new StreamsBuilder
val textLines = streamBuilder.stream[String, String](inputTopic)
textLines
.mapValues(v => v.length)
.groupByKey
.cogroup((_, v1, v2: Long) => v1 + v2)
.aggregate(0L)
streamBuilder.build().describe()
}
// build the Java topology
def getTopologyJava: TopologyDescription = {
val streamBuilder = new StreamsBuilderJ
val textLines: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopic)
val splits: KStreamJ[String, Int] = textLines.mapValues(
new ValueMapper[String, Int] {
def apply(s: String): Int = s.length
}
)
splits.groupByKey
.cogroup((k: String, v: Int, a: Long) => a + v)
.aggregate(() => 0L)
streamBuilder.build().describe()
}
// should match
assertEquals(getTopologyScala, getTopologyJava)
}
@Test def shouldBuildIdenticalTopologyInJavaNScalaCogroup(): Unit = {
// build the Scala topology
def getTopologyScala: TopologyDescription = {
import org.apache.kafka.streams.scala.Serdes._
val streamBuilder = new StreamsBuilder
val textLines1 = streamBuilder.stream[String, String](inputTopic)
val textLines2 = streamBuilder.stream[String, String]("inputTopic2")
textLines1
.mapValues(v => v.length)
.groupByKey
.cogroup((_, v1, v2: Long) => v1 + v2)
.cogroup(textLines2.groupByKey, (_, v: String, a) => v.length + a)
.aggregate(0L)
streamBuilder.build().describe()
}
// build the Java topology
def getTopologyJava: TopologyDescription = {
val streamBuilder = new StreamsBuilderJ
val textLines1: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopic)
val textLines2: KStreamJ[String, String] = streamBuilder.stream[String, String]("inputTopic2")
val splits: KStreamJ[String, Int] = textLines1.mapValues(
new ValueMapper[String, Int] {
def apply(s: String): Int = s.length
}
)
splits.groupByKey
.cogroup((k: String, v: Int, a: Long) => a + v)
.cogroup(textLines2.groupByKey(), (k: String, v: String, a: Long) => v.length + a)
.aggregate(() => 0L)
streamBuilder.build().describe()
}
// should match
assertEquals(getTopologyScala, getTopologyJava)
}
@Test def shouldBuildIdenticalTopologyInJavaNScalaJoin(): Unit = {
// build the Scala topology
def getTopologyScala: TopologyDescription = {
import Serdes._
val builder = new StreamsBuilder()
val userClicksStream: KStream[String, Long] = builder.stream(userClicksTopic)
val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic)
// clicks per region
userClicksStream
.leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks))
.map((_, regionWithClicks) => regionWithClicks)
.groupByKey
.reduce(_ + _)
builder.build().describe()
}
// build the Java topology
def getTopologyJava: TopologyDescription = {
import java.lang.{Long => JLong}
val builder: StreamsBuilderJ = new StreamsBuilderJ()
val userClicksStream: KStreamJ[String, JLong] =
builder.stream[String, JLong](userClicksTopic, Consumed.`with`[String, JLong])
val userRegionsTable: KTableJ[String, String] =
builder.table[String, String](userRegionsTopic, Consumed.`with`[String, String])
// Join the stream against the table.
val valueJoinerJ: ValueJoiner[JLong, String, (String, JLong)] =
(clicks: JLong, region: String) => (if (region == null) "UNKNOWN" else region, clicks)
val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream.leftJoin(
userRegionsTable,
valueJoinerJ,
Joined.`with`[String, JLong, String]
)
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks>
val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion.map { (_, regionWithClicks) =>
new KeyValue(regionWithClicks._1, regionWithClicks._2)
}
// Compute the total per region by summing the individual click counts per region.
clicksByRegion
.groupByKey(Grouped.`with`[String, JLong])
.reduce((v1, v2) => v1 + v2)
builder.build().describe()
}
// should match
assertEquals(getTopologyScala, getTopologyJava)
}
@Test
def shouldBuildIdenticalTopologyInJavaNScalaTransform(): Unit = {
// build the Scala topology
def getTopologyScala: TopologyDescription = {
import Serdes._
val streamBuilder = new StreamsBuilder
val textLines = streamBuilder.stream[String, String](inputTopic)
val _: KTable[String, Long] = textLines
.transform(
() =>
new Transformer[String, String, KeyValue[String, String]] {
override def init(context: ProcessorContext): Unit = ()
override def transform(key: String, value: String): KeyValue[String, String] =
new KeyValue(key, value.toLowerCase)
override def close(): Unit = ()
}
)
.groupBy((_, v) => v)
.count()
streamBuilder.build().describe()
}
// build the Java topology
def getTopologyJava: TopologyDescription = {
val streamBuilder = new StreamsBuilderJ
val textLines: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopic)
val lowered: KStreamJ[String, String] = textLines.transform(
() =>
new Transformer[String, String, KeyValue[String, String]] {
override def init(context: ProcessorContext): Unit = ()
override def transform(key: String, value: String): KeyValue[String, String] =
new KeyValue(key, value.toLowerCase)
override def close(): Unit = ()
}
)
val grouped: KGroupedStreamJ[String, String] = lowered.groupBy((_, v) => v)
// word counts
grouped.count()
streamBuilder.build().describe()
}
// should match
assertEquals(getTopologyScala, getTopologyJava)
}
@Test
def shouldBuildIdenticalTopologyInJavaNScalaProperties(): Unit = {
val props = new Properties()
props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE)
val propsNoOptimization = new Properties()
propsNoOptimization.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.NO_OPTIMIZATION)
val AGGREGATION_TOPIC = "aggregationTopic"
val REDUCE_TOPIC = "reduceTopic"
val JOINED_TOPIC = "joinedTopic"
// build the Scala topology
def getTopologyScala: StreamsBuilder = {
val aggregator = (_: String, v: String, agg: Int) => agg + v.length
val reducer = (v1: String, v2: String) => v1 + ":" + v2
val processorValueCollector: util.List[String] = new util.ArrayList[String]
val builder: StreamsBuilder = new StreamsBuilder
val sourceStream: KStream[String, String] =
builder.stream(inputTopic)(Consumed.`with`(Serdes.String, Serdes.String))
val mappedStream: KStream[String, String] =
sourceStream.map((k: String, v: String) => (k.toUpperCase(Locale.getDefault), v))
mappedStream
.filter((k: String, _: String) => k == "B")
.mapValues((v: String) => v.toUpperCase(Locale.getDefault))
.process(() => new SimpleProcessor(processorValueCollector))
val stream2 = mappedStream.groupByKey
.aggregate(0)(aggregator)(Materialized.`with`(Serdes.String, Serdes.Integer))
.toStream
stream2.to(AGGREGATION_TOPIC)(Produced.`with`(Serdes.String, Serdes.Integer))
// adding operators for case where the repartition node is further downstream
val stream3 = mappedStream
.filter((_: String, _: String) => true)
.peek((k: String, v: String) => System.out.println(k + ":" + v))
.groupByKey
.reduce(reducer)(Materialized.`with`(Serdes.String, Serdes.String))
.toStream
stream3.to(REDUCE_TOPIC)(Produced.`with`(Serdes.String, Serdes.String))
mappedStream
.filter((k: String, _: String) => k == "A")
.join(stream2)((v1: String, v2: Int) => v1 + ":" + v2.toString, JoinWindows.of(Duration.ofMillis(5000)))(
StreamJoined.`with`(Serdes.String, Serdes.String, Serdes.Integer)
)
.to(JOINED_TOPIC)
mappedStream
.filter((k: String, _: String) => k == "A")
.join(stream3)((v1: String, v2: String) => v1 + ":" + v2.toString, JoinWindows.of(Duration.ofMillis(5000)))(
StreamJoined.`with`(Serdes.String, Serdes.String, Serdes.String)
)
.to(JOINED_TOPIC)
builder
}
// build the Java topology
def getTopologyJava: StreamsBuilderJ = {
val keyValueMapper: KeyValueMapper[String, String, KeyValue[String, String]] =
(key, value) => KeyValue.pair(key.toUpperCase(Locale.getDefault), value)
val initializer: Initializer[Integer] = () => 0
val aggregator: Aggregator[String, String, Integer] = (_, value, aggregate) => aggregate + value.length
val reducer: Reducer[String] = (v1, v2) => v1 + ":" + v2
val valueMapper: ValueMapper[String, String] = v => v.toUpperCase(Locale.getDefault)
val processorValueCollector = new util.ArrayList[String]
val processorSupplier: ProcessorSupplier[String, String] = () => new SimpleProcessor(processorValueCollector)
val valueJoiner2: ValueJoiner[String, Integer, String] = (value1, value2) => value1 + ":" + value2.toString
val valueJoiner3: ValueJoiner[String, String, String] = (value1, value2) => value1 + ":" + value2
val builder = new StreamsBuilderJ
val sourceStream = builder.stream(inputTopic, Consumed.`with`(Serdes.String, Serdes.String))
val mappedStream: KStreamJ[String, String] =
sourceStream.map(keyValueMapper)
mappedStream
.filter((key, _) => key == "B")
.mapValues[String](valueMapper)
.process(processorSupplier)
val stream2: KStreamJ[String, Integer] = mappedStream.groupByKey
.aggregate(initializer, aggregator, MaterializedJ.`with`(Serdes.String, SerdesJ.Integer))
.toStream
stream2.to(AGGREGATION_TOPIC, Produced.`with`(Serdes.String, SerdesJ.Integer))
// adding operators for case where the repartition node is further downstream
val stream3 = mappedStream
.filter((_, _) => true)
.peek((k, v) => System.out.println(k + ":" + v))
.groupByKey
.reduce(reducer, MaterializedJ.`with`(Serdes.String, Serdes.String))
.toStream
stream3.to(REDUCE_TOPIC, Produced.`with`(Serdes.String, Serdes.String))
mappedStream
.filter((key, _) => key == "A")
.join[Integer, String](stream2,
valueJoiner2,
JoinWindows.of(Duration.ofMillis(5000)),
StreamJoinedJ.`with`(Serdes.String, Serdes.String, SerdesJ.Integer))
.to(JOINED_TOPIC)
mappedStream
.filter((key, _) => key == "A")
.join(stream3,
valueJoiner3,
JoinWindows.of(Duration.ofMillis(5000)),
StreamJoinedJ.`with`(Serdes.String, Serdes.String, SerdesJ.String))
.to(JOINED_TOPIC)
builder
}
assertNotEquals(getTopologyScala.build(props).describe.toString,
getTopologyScala.build(propsNoOptimization).describe.toString)
assertEquals(getTopologyScala.build(propsNoOptimization).describe.toString,
getTopologyJava.build(propsNoOptimization).describe.toString)
assertEquals(getTopologyScala.build(props).describe.toString, getTopologyJava.build(props).describe.toString)
}
private class SimpleProcessor private[TopologyTest] (val valueList: util.List[String])
extends AbstractProcessor[String, String] {
override def process(key: String, value: String): Unit =
valueList.add(value)
}
}

View File

@@ -0,0 +1,248 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala
import java.util.Properties
import java.util.regex.Pattern
import org.junit.Assert._
import org.junit._
import org.junit.rules.TemporaryFolder
import org.apache.kafka.streams.{KafkaStreams, KeyValue, StreamsConfig}
import org.apache.kafka.streams.scala.kstream._
import org.apache.kafka.streams.integration.utils.{EmbeddedKafkaCluster, IntegrationTestUtils}
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.clients.producer.ProducerConfig
import org.apache.kafka.common.utils.MockTime
import org.apache.kafka.test.{IntegrationTest, TestUtils}
import ImplicitConversions._
import org.apache.kafka.common.serialization.{LongDeserializer, StringDeserializer, StringSerializer}
import org.junit.experimental.categories.Category
/**
* Test suite that does a classic word count example.
* <p>
* The suite contains the test case using Scala APIs `testShouldCountWords` and the same test case using the
* Java APIs `testShouldCountWordsJava`. The idea is to demonstrate that both generate the same result.
*/
@Category(Array(classOf[IntegrationTest]))
class WordCountTest extends WordCountTestData {
private val privateCluster: EmbeddedKafkaCluster = new EmbeddedKafkaCluster(1)
@Rule def cluster: EmbeddedKafkaCluster = privateCluster
final val alignedTime = (System.currentTimeMillis() / 1000 + 1) * 1000
val mockTime: MockTime = cluster.time
mockTime.setCurrentTimeMs(alignedTime)
val tFolder: TemporaryFolder = new TemporaryFolder(TestUtils.tempDirectory())
@Rule def testFolder: TemporaryFolder = tFolder
@Before
def startKafkaCluster(): Unit = {
cluster.createTopic(inputTopic)
cluster.createTopic(outputTopic)
cluster.createTopic(inputTopicJ)
cluster.createTopic(outputTopicJ)
}
@Test
def testShouldCountWords(): Unit = {
import Serdes._
val streamsConfiguration = getStreamsConfiguration()
val streamBuilder = new StreamsBuilder
val textLines = streamBuilder.stream[String, String](inputTopic)
val pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS)
// generate word counts
val wordCounts: KTable[String, Long] =
textLines
.flatMapValues(v => pattern.split(v.toLowerCase))
.groupBy((_, v) => v)
.count()
// write to output topic
wordCounts.toStream.to(outputTopic)
val streams = new KafkaStreams(streamBuilder.build(), streamsConfiguration)
streams.start()
// produce and consume synchronously
val actualWordCounts: java.util.List[KeyValue[String, Long]] = produceNConsume(inputTopic, outputTopic)
streams.close()
import collection.JavaConverters._
assertEquals(actualWordCounts.asScala.take(expectedWordCounts.size).sortBy(_.key), expectedWordCounts.sortBy(_.key))
}
@Test
def testShouldCountWordsMaterialized(): Unit = {
import Serdes._
val streamsConfiguration = getStreamsConfiguration()
val streamBuilder = new StreamsBuilder
val textLines = streamBuilder.stream[String, String](inputTopic)
val pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS)
// generate word counts
val wordCounts: KTable[String, Long] =
textLines
.flatMapValues(v => pattern.split(v.toLowerCase))
.groupBy((k, v) => v)
.count()(Materialized.as("word-count"))
// write to output topic
wordCounts.toStream.to(outputTopic)
val streams = new KafkaStreams(streamBuilder.build(), streamsConfiguration)
streams.start()
// produce and consume synchronously
val actualWordCounts: java.util.List[KeyValue[String, Long]] = produceNConsume(inputTopic, outputTopic)
streams.close()
import collection.JavaConverters._
assertEquals(actualWordCounts.asScala.take(expectedWordCounts.size).sortBy(_.key), expectedWordCounts.sortBy(_.key))
}
@Test
def testShouldCountWordsJava(): Unit = {
import org.apache.kafka.streams.{KafkaStreams => KafkaStreamsJ, StreamsBuilder => StreamsBuilderJ}
import org.apache.kafka.streams.kstream.{
KTable => KTableJ,
KStream => KStreamJ,
KGroupedStream => KGroupedStreamJ,
_
}
import collection.JavaConverters._
val streamsConfiguration = getStreamsConfiguration()
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String.getClass.getName)
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String.getClass.getName)
val streamBuilder = new StreamsBuilderJ
val textLines: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopicJ)
val pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS)
val splits: KStreamJ[String, String] = textLines.flatMapValues { line =>
pattern.split(line.toLowerCase).toIterable.asJava
}
val grouped: KGroupedStreamJ[String, String] = splits.groupBy { (_, v) =>
v
}
val wordCounts: KTableJ[String, java.lang.Long] = grouped.count()
wordCounts.toStream.to(outputTopicJ, Produced.`with`(Serdes.String, Serdes.JavaLong))
val streams: KafkaStreamsJ = new KafkaStreamsJ(streamBuilder.build(), streamsConfiguration)
streams.start()
val actualWordCounts: java.util.List[KeyValue[String, Long]] = produceNConsume(inputTopicJ, outputTopicJ)
streams.close()
assertEquals(actualWordCounts.asScala.take(expectedWordCounts.size).sortBy(_.key), expectedWordCounts.sortBy(_.key))
}
private def getStreamsConfiguration(): Properties = {
val streamsConfiguration: Properties = new Properties()
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-test")
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, "10000")
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot.getPath)
streamsConfiguration
}
private def getProducerConfig(): Properties = {
val p = new Properties()
p.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
p.put(ProducerConfig.ACKS_CONFIG, "all")
p.put(ProducerConfig.RETRIES_CONFIG, "0")
p.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer])
p.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer])
p
}
private def getConsumerConfig(): Properties = {
val p = new Properties()
p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
p.put(ConsumerConfig.GROUP_ID_CONFIG, "wordcount-scala-integration-test-standard-consumer")
p.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer])
p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[LongDeserializer])
p
}
private def produceNConsume(inputTopic: String, outputTopic: String): java.util.List[KeyValue[String, Long]] = {
val linesProducerConfig: Properties = getProducerConfig()
import collection.JavaConverters._
IntegrationTestUtils.produceValuesSynchronously(inputTopic, inputValues.asJava, linesProducerConfig, mockTime)
val consumerConfig = getConsumerConfig()
IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(consumerConfig, outputTopic, expectedWordCounts.size)
}
}
trait WordCountTestData {
val inputTopic = s"inputTopic"
val outputTopic = s"outputTopic"
val inputTopicJ = s"inputTopicJ"
val outputTopicJ = s"outputTopicJ"
val inputValues = List(
"Hello Kafka Streams",
"All streams lead to Kafka",
"Join Kafka Summit",
"И теперь пошли русские слова"
)
val expectedWordCounts: List[KeyValue[String, Long]] = List(
new KeyValue("hello", 1L),
new KeyValue("all", 1L),
new KeyValue("streams", 2L),
new KeyValue("lead", 1L),
new KeyValue("to", 1L),
new KeyValue("join", 1L),
new KeyValue("kafka", 3L),
new KeyValue("summit", 1L),
new KeyValue("и", 1L),
new KeyValue("теперь", 1L),
new KeyValue("пошли", 1L),
new KeyValue("русские", 1L),
new KeyValue("слова", 1L)
)
}

View File

@@ -0,0 +1,73 @@
/*
* Copyright (C) 2018 Joan Goyeau.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.streams.Topology
import org.apache.kafka.streams.kstream.internals.ConsumedInternal
import org.apache.kafka.streams.processor.FailOnInvalidTimestamp
import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.scala.Serdes
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@RunWith(classOf[JUnitRunner])
class ConsumedTest extends FlatSpec with Matchers {
"Create a Consumed" should "create a Consumed with Serdes" in {
val consumed: Consumed[String, Long] = Consumed.`with`[String, Long]
val internalConsumed = new ConsumedInternal(consumed)
internalConsumed.keySerde.getClass shouldBe Serdes.String.getClass
internalConsumed.valueSerde.getClass shouldBe Serdes.Long.getClass
}
"Create a Consumed with timestampExtractor and resetPolicy" should "create a Consumed with Serdes, timestampExtractor and resetPolicy" in {
val timestampExtractor = new FailOnInvalidTimestamp()
val resetPolicy = Topology.AutoOffsetReset.LATEST
val consumed: Consumed[String, Long] =
Consumed.`with`[String, Long](timestampExtractor, resetPolicy)
val internalConsumed = new ConsumedInternal(consumed)
internalConsumed.keySerde.getClass shouldBe Serdes.String.getClass
internalConsumed.valueSerde.getClass shouldBe Serdes.Long.getClass
internalConsumed.timestampExtractor shouldBe timestampExtractor
internalConsumed.offsetResetPolicy shouldBe resetPolicy
}
"Create a Consumed with timestampExtractor" should "create a Consumed with Serdes and timestampExtractor" in {
val timestampExtractor = new FailOnInvalidTimestamp()
val consumed: Consumed[String, Long] = Consumed.`with`[String, Long](timestampExtractor)
val internalConsumed = new ConsumedInternal(consumed)
internalConsumed.keySerde.getClass shouldBe Serdes.String.getClass
internalConsumed.valueSerde.getClass shouldBe Serdes.Long.getClass
internalConsumed.timestampExtractor shouldBe timestampExtractor
}
"Create a Consumed with resetPolicy" should "create a Consumed with Serdes and resetPolicy" in {
val resetPolicy = Topology.AutoOffsetReset.LATEST
val consumed: Consumed[String, Long] = Consumed.`with`[String, Long](resetPolicy)
val internalConsumed = new ConsumedInternal(consumed)
internalConsumed.keySerde.getClass shouldBe Serdes.String.getClass
internalConsumed.valueSerde.getClass shouldBe Serdes.Long.getClass
internalConsumed.offsetResetPolicy shouldBe resetPolicy
}
}

View File

@@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.streams.kstream.internals.GroupedInternal
import org.apache.kafka.streams.scala.Serdes
import org.apache.kafka.streams.scala.Serdes._
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@RunWith(classOf[JUnitRunner])
class GroupedTest extends FlatSpec with Matchers {
"Create a Grouped" should "create a Grouped with Serdes" in {
val grouped: Grouped[String, Long] = Grouped.`with`[String, Long]
val internalGrouped = new GroupedInternal[String, Long](grouped)
internalGrouped.keySerde.getClass shouldBe Serdes.String.getClass
internalGrouped.valueSerde.getClass shouldBe Serdes.Long.getClass
}
"Create a Grouped with repartition topic name" should "create a Grouped with Serdes, and repartition topic name" in {
val repartitionTopicName = "repartition-topic"
val grouped: Grouped[String, Long] = Grouped.`with`(repartitionTopicName)
val internalGrouped = new GroupedInternal[String, Long](grouped)
internalGrouped.keySerde.getClass shouldBe Serdes.String.getClass
internalGrouped.valueSerde.getClass shouldBe Serdes.Long.getClass
internalGrouped.name() shouldBe repartitionTopicName
}
}

View File

@@ -0,0 +1,46 @@
/*
* Copyright (C) 2018 Joan Goyeau.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.streams.scala.Serdes
import org.apache.kafka.streams.scala.Serdes._
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@RunWith(classOf[JUnitRunner])
class JoinedTest extends FlatSpec with Matchers {
"Create a Joined" should "create a Joined with Serdes" in {
val joined: Joined[String, Long, Int] = Joined.`with`[String, Long, Int]
joined.keySerde.getClass shouldBe Serdes.String.getClass
joined.valueSerde.getClass shouldBe Serdes.Long.getClass
joined.otherValueSerde.getClass shouldBe Serdes.Integer.getClass
}
"Create a Joined" should "create a Joined with Serdes and repartition topic name" in {
val repartitionTopicName = "repartition-topic"
val joined: Joined[String, Long, Int] = Joined.`with`(repartitionTopicName)
joined.keySerde.getClass shouldBe Serdes.String.getClass
joined.valueSerde.getClass shouldBe Serdes.Long.getClass
joined.otherValueSerde.getClass shouldBe Serdes.Integer.getClass
}
}

View File

@@ -0,0 +1,340 @@
/*
* Copyright (C) 2018 Joan Goyeau.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import java.time.Duration.ofSeconds
import java.time.Instant
import org.apache.kafka.streams.KeyValue
import org.apache.kafka.streams.kstream.{
JoinWindows,
Transformer,
ValueTransformer,
ValueTransformerSupplier,
ValueTransformerWithKey,
ValueTransformerWithKeySupplier
}
import org.apache.kafka.streams.processor.ProcessorContext
import org.apache.kafka.streams.scala.ImplicitConversions._
import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.scala.StreamsBuilder
import org.apache.kafka.streams.scala.utils.TestDriver
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@RunWith(classOf[JUnitRunner])
class KStreamTest extends FlatSpec with Matchers with TestDriver {
"filter a KStream" should "filter records satisfying the predicate" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
builder.stream[String, String](sourceTopic).filter((_, value) => value != "value2").to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, String](sinkTopic)
testInput.pipeInput("1", "value1")
testOutput.readValue shouldBe "value1"
testInput.pipeInput("2", "value2")
testOutput.isEmpty shouldBe true
testInput.pipeInput("3", "value3")
testOutput.readValue shouldBe "value3"
testOutput.isEmpty shouldBe true
testDriver.close()
}
"filterNot a KStream" should "filter records not satisfying the predicate" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
builder.stream[String, String](sourceTopic).filterNot((_, value) => value == "value2").to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, String](sinkTopic)
testInput.pipeInput("1", "value1")
testOutput.readValue shouldBe "value1"
testInput.pipeInput("2", "value2")
testOutput.isEmpty shouldBe true
testInput.pipeInput("3", "value3")
testOutput.readValue shouldBe "value3"
testOutput.isEmpty shouldBe true
testDriver.close()
}
"foreach a KStream" should "run foreach actions on records" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
var acc = ""
builder.stream[String, String](sourceTopic).foreach((_, value) => acc += value)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
testInput.pipeInput("1", "value1")
acc shouldBe "value1"
testInput.pipeInput("2", "value2")
acc shouldBe "value1value2"
testDriver.close()
}
"peek a KStream" should "run peek actions on records" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
var acc = ""
builder.stream[String, String](sourceTopic).peek((_, v) => acc += v).to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, String](sinkTopic)
testInput.pipeInput("1", "value1")
acc shouldBe "value1"
testOutput.readValue shouldBe "value1"
testInput.pipeInput("2", "value2")
acc shouldBe "value1value2"
testOutput.readValue shouldBe "value2"
testDriver.close()
}
"selectKey a KStream" should "select a new key" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
builder.stream[String, String](sourceTopic).selectKey((_, value) => value).to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, String](sinkTopic)
testInput.pipeInput("1", "value1")
testOutput.readKeyValue.key shouldBe "value1"
testInput.pipeInput("1", "value2")
testOutput.readKeyValue.key shouldBe "value2"
testOutput.isEmpty shouldBe true
testDriver.close()
}
"join 2 KStreams" should "join correctly records" in {
val builder = new StreamsBuilder()
val sourceTopic1 = "source1"
val sourceTopic2 = "source2"
val sinkTopic = "sink"
val stream1 = builder.stream[String, String](sourceTopic1)
val stream2 = builder.stream[String, String](sourceTopic2)
stream1.join(stream2)((a, b) => s"$a-$b", JoinWindows.of(ofSeconds(1))).to(sinkTopic)
val now = Instant.now()
val testDriver = createTestDriver(builder, now)
val testInput1 = testDriver.createInput[String, String](sourceTopic1)
val testInput2 = testDriver.createInput[String, String](sourceTopic2)
val testOutput = testDriver.createOutput[String, String](sinkTopic)
testInput1.pipeInput("1", "topic1value1", now)
testInput2.pipeInput("1", "topic2value1", now)
testOutput.readValue shouldBe "topic1value1-topic2value1"
testOutput.isEmpty shouldBe true
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 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 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()
}
"join 2 KStreamToTables" should "join correctly records" in {
val builder = new StreamsBuilder()
val sourceTopic1 = "source1"
val sourceTopic2 = "source2"
val sinkTopic = "sink"
val table1 = builder.stream[String, String](sourceTopic1).toTable
val table2 = builder.stream[String, String](sourceTopic2).toTable
table1.join(table2)((a, b) => a + b).toStream.to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput1 = testDriver.createInput[String, String](sourceTopic1)
val testInput2 = testDriver.createInput[String, String](sourceTopic2)
val testOutput = testDriver.createOutput[String, String](sinkTopic)
testInput1.pipeInput("1", "topic1value1")
testInput2.pipeInput("1", "topic2value1")
testOutput.readValue shouldBe "topic1value1topic2value1"
testOutput.isEmpty shouldBe true
testDriver.close()
}
}

View File

@@ -0,0 +1,396 @@
/*
* Copyright (C) 2018 Joan Goyeau.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import java.time.Duration
import org.apache.kafka.streams.kstream.{SessionWindows, Suppressed => JSuppressed, TimeWindows, Windowed}
import org.apache.kafka.streams.kstream.Suppressed.BufferConfig
import org.apache.kafka.streams.scala.ImplicitConversions._
import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.scala.utils.TestDriver
import org.apache.kafka.streams.scala.{ByteArrayKeyValueStore, StreamsBuilder}
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@RunWith(classOf[JUnitRunner])
class KTableTest extends FlatSpec with Matchers with TestDriver {
"filter a KTable" should "filter records satisfying the predicate" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
val table = builder.stream[String, String](sourceTopic).groupBy((key, _) => key).count()
table.filter((_, value) => value > 1).toStream.to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, Long](sinkTopic)
{
testInput.pipeInput("1", "value1")
val record = testOutput.readKeyValue
record.key shouldBe "1"
record.value shouldBe (null: java.lang.Long)
}
{
testInput.pipeInput("1", "value2")
val record = testOutput.readKeyValue
record.key shouldBe "1"
record.value shouldBe 2
}
{
testInput.pipeInput("2", "value1")
val record = testOutput.readKeyValue
record.key shouldBe "2"
record.value shouldBe (null: java.lang.Long)
}
testOutput.isEmpty shouldBe true
testDriver.close()
}
"filterNot a KTable" should "filter records not satisfying the predicate" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
val table = builder.stream[String, String](sourceTopic).groupBy((key, _) => key).count()
table.filterNot((_, value) => value > 1).toStream.to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, Long](sinkTopic)
{
testInput.pipeInput("1", "value1")
val record = testOutput.readKeyValue
record.key shouldBe "1"
record.value shouldBe 1
}
{
testInput.pipeInput("1", "value2")
val record = testOutput.readKeyValue
record.key shouldBe "1"
record.value shouldBe (null: java.lang.Long)
}
{
testInput.pipeInput("2", "value1")
val record = testOutput.readKeyValue
record.key shouldBe "2"
record.value shouldBe 1
}
testOutput.isEmpty shouldBe true
testDriver.close()
}
"join 2 KTables" should "join correctly records" in {
val builder = new StreamsBuilder()
val sourceTopic1 = "source1"
val sourceTopic2 = "source2"
val sinkTopic = "sink"
val table1 = builder.stream[String, String](sourceTopic1).groupBy((key, _) => key).count()
val table2 = builder.stream[String, String](sourceTopic2).groupBy((key, _) => key).count()
table1.join(table2)((a, b) => a + b).toStream.to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput1 = testDriver.createInput[String, String](sourceTopic1)
val testInput2 = testDriver.createInput[String, String](sourceTopic2)
val testOutput = testDriver.createOutput[String, Long](sinkTopic)
testInput1.pipeInput("1", "topic1value1")
testInput2.pipeInput("1", "topic2value1")
testOutput.readValue shouldBe 2
testOutput.isEmpty shouldBe true
testDriver.close()
}
"join 2 KTables with a Materialized" should "join correctly records and state store" in {
val builder = new StreamsBuilder()
val sourceTopic1 = "source1"
val sourceTopic2 = "source2"
val sinkTopic = "sink"
val stateStore = "store"
val materialized = Materialized.as[String, Long, ByteArrayKeyValueStore](stateStore)
val table1 = builder.stream[String, String](sourceTopic1).groupBy((key, _) => key).count()
val table2 = builder.stream[String, String](sourceTopic2).groupBy((key, _) => key).count()
table1.join(table2, materialized)((a, b) => a + b).toStream.to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput1 = testDriver.createInput[String, String](sourceTopic1)
val testInput2 = testDriver.createInput[String, String](sourceTopic2)
val testOutput = testDriver.createOutput[String, Long](sinkTopic)
testInput1.pipeInput("1", "topic1value1")
testInput2.pipeInput("1", "topic2value1")
testOutput.readValue shouldBe 2
testDriver.getKeyValueStore[String, Long](stateStore).get("1") shouldBe 2
testOutput.isEmpty shouldBe true
testDriver.close()
}
"windowed KTable#suppress" should "correctly suppress results using Suppressed.untilTimeLimit" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
val window = TimeWindows.of(Duration.ofSeconds(1L))
val suppression = JSuppressed.untilTimeLimit[Windowed[String]](Duration.ofSeconds(2L), BufferConfig.unbounded())
val table: KTable[Windowed[String], Long] = builder
.stream[String, String](sourceTopic)
.groupByKey
.windowedBy(window)
.count
.suppress(suppression)
table.toStream((k, _) => s"${k.window().start()}:${k.window().end()}:${k.key()}").to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, Long](sinkTopic)
{
// publish key=1 @ time 0 => count==1
testInput.pipeInput("1", "value1", 0L)
testOutput.isEmpty shouldBe true
}
{
// publish key=1 @ time 1 => count==2
testInput.pipeInput("1", "value2", 1L)
testOutput.isEmpty shouldBe true
}
{
// move event time past the first window, but before the suppression window
testInput.pipeInput("2", "value1", 1001L)
testOutput.isEmpty shouldBe true
}
{
// move event time riiiight before suppression window ends
testInput.pipeInput("2", "value2", 1999L)
testOutput.isEmpty shouldBe true
}
{
// publish a late event before suppression window terminates => count==3
testInput.pipeInput("1", "value3", 999L)
testOutput.isEmpty shouldBe true
}
{
// move event time right past the suppression window of the first window.
testInput.pipeInput("2", "value3", 2001L)
val record = testOutput.readKeyValue
record.key shouldBe "0:1000:1"
record.value shouldBe 3L
}
testOutput.isEmpty shouldBe true
testDriver.close()
}
"windowed KTable#suppress" should "correctly suppress results using Suppressed.untilWindowCloses" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
val window = TimeWindows.of(Duration.ofSeconds(1L)).grace(Duration.ofSeconds(1L))
val suppression = JSuppressed.untilWindowCloses(BufferConfig.unbounded())
val table: KTable[Windowed[String], Long] = builder
.stream[String, String](sourceTopic)
.groupByKey
.windowedBy(window)
.count
.suppress(suppression)
table.toStream((k, _) => s"${k.window().start()}:${k.window().end()}:${k.key()}").to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, Long](sinkTopic)
{
// publish key=1 @ time 0 => count==1
testInput.pipeInput("1", "value1", 0L)
testOutput.isEmpty shouldBe true
}
{
// publish key=1 @ time 1 => count==2
testInput.pipeInput("1", "value2", 1L)
testOutput.isEmpty shouldBe true
}
{
// move event time past the window, but before the grace period
testInput.pipeInput("2", "value1", 1001L)
testOutput.isEmpty shouldBe true
}
{
// move event time riiiight before grace period ends
testInput.pipeInput("2", "value2", 1999L)
testOutput.isEmpty shouldBe true
}
{
// publish a late event before grace period terminates => count==3
testInput.pipeInput("1", "value3", 999L)
testOutput.isEmpty shouldBe true
}
{
// move event time right past the grace period of the first window.
testInput.pipeInput("2", "value3", 2001L)
val record = testOutput.readKeyValue
record.key shouldBe "0:1000:1"
record.value shouldBe 3L
}
testOutput.isEmpty shouldBe true
testDriver.close()
}
"session windowed KTable#suppress" should "correctly suppress results using Suppressed.untilWindowCloses" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
// Very similar to SuppressScenarioTest.shouldSupportFinalResultsForSessionWindows
val window = SessionWindows.`with`(Duration.ofMillis(5L)).grace(Duration.ofMillis(10L))
val suppression = JSuppressed.untilWindowCloses(BufferConfig.unbounded())
val table: KTable[Windowed[String], Long] = builder
.stream[String, String](sourceTopic)
.groupByKey
.windowedBy(window)
.count
.suppress(suppression)
table.toStream((k, _) => s"${k.window().start()}:${k.window().end()}:${k.key()}").to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, Long](sinkTopic)
{
// first window
testInput.pipeInput("k1", "v1", 0L)
testOutput.isEmpty shouldBe true
}
{
// first window
testInput.pipeInput("k1", "v1", 1L)
testOutput.isEmpty shouldBe true
}
{
// new window, but grace period hasn't ended for first window
testInput.pipeInput("k1", "v1", 8L)
testOutput.isEmpty shouldBe true
}
{
// out-of-order event for first window, included since grade period hasn't passed
testInput.pipeInput("k1", "v1", 2L)
testOutput.isEmpty shouldBe true
}
{
// add to second window
testInput.pipeInput("k1", "v1", 13L)
testOutput.isEmpty shouldBe true
}
{
// add out-of-order to second window
testInput.pipeInput("k1", "v1", 10L)
testOutput.isEmpty shouldBe true
}
{
// push stream time forward to flush other events through
testInput.pipeInput("k1", "v1", 30L)
// late event should get dropped from the stream
testInput.pipeInput("k1", "v1", 3L)
// should now have to results
val r1 = testOutput.readRecord
r1.key shouldBe "0:2:k1"
r1.value shouldBe 3L
r1.timestamp shouldBe 2L
val r2 = testOutput.readRecord
r2.key shouldBe "8:13:k1"
r2.value shouldBe 3L
r2.timestamp shouldBe 13L
}
testOutput.isEmpty shouldBe true
testDriver.close()
}
"non-windowed KTable#suppress" should "correctly suppress results using Suppressed.untilTimeLimit" in {
val builder = new StreamsBuilder()
val sourceTopic = "source"
val sinkTopic = "sink"
val suppression = JSuppressed.untilTimeLimit[String](Duration.ofSeconds(2L), BufferConfig.unbounded())
val table: KTable[String, Long] = builder
.stream[String, String](sourceTopic)
.groupByKey
.count
.suppress(suppression)
table.toStream.to(sinkTopic)
val testDriver = createTestDriver(builder)
val testInput = testDriver.createInput[String, String](sourceTopic)
val testOutput = testDriver.createOutput[String, Long](sinkTopic)
{
// publish key=1 @ time 0 => count==1
testInput.pipeInput("1", "value1", 0L)
testOutput.isEmpty shouldBe true
}
{
// publish key=1 @ time 1 => count==2
testInput.pipeInput("1", "value2", 1L)
testOutput.isEmpty shouldBe true
}
{
// move event time past the window, but before the grace period
testInput.pipeInput("2", "value1", 1001L)
testOutput.isEmpty shouldBe true
}
{
// move event time right before grace period ends
testInput.pipeInput("2", "value2", 1999L)
testOutput.isEmpty shouldBe true
}
{
// publish a late event before grace period terminates => count==3
testInput.pipeInput("1", "value3", 999L)
testOutput.isEmpty shouldBe true
}
{
// move event time right past the grace period of the first window.
testInput.pipeInput("2", "value3", 2001L)
val record = testOutput.readKeyValue
record.key shouldBe "1"
record.value shouldBe 3L
}
testOutput.isEmpty shouldBe true
testDriver.close()
}
}

View File

@@ -0,0 +1,86 @@
/*
* Copyright (C) 2018 Joan Goyeau.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import java.time.Duration
import org.apache.kafka.streams.kstream.internals.MaterializedInternal
import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.scala._
import org.apache.kafka.streams.state.Stores
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@RunWith(classOf[JUnitRunner])
class MaterializedTest extends FlatSpec with Matchers {
"Create a Materialized" should "create a Materialized with Serdes" in {
val materialized: Materialized[String, Long, ByteArrayKeyValueStore] =
Materialized.`with`[String, Long, ByteArrayKeyValueStore]
val internalMaterialized = new MaterializedInternal(materialized)
internalMaterialized.keySerde.getClass shouldBe Serdes.String.getClass
internalMaterialized.valueSerde.getClass shouldBe Serdes.Long.getClass
}
"Create a Materialize with a store name" should "create a Materialized with Serdes and a store name" in {
val storeName = "store"
val materialized: Materialized[String, Long, ByteArrayKeyValueStore] =
Materialized.as[String, Long, ByteArrayKeyValueStore](storeName)
val internalMaterialized = new MaterializedInternal(materialized)
internalMaterialized.keySerde.getClass shouldBe Serdes.String.getClass
internalMaterialized.valueSerde.getClass shouldBe Serdes.Long.getClass
internalMaterialized.storeName shouldBe storeName
}
"Create a Materialize with a window store supplier" should "create a Materialized with Serdes and a store supplier" in {
val storeSupplier = Stores.persistentWindowStore("store", Duration.ofMillis(1), Duration.ofMillis(1), true)
val materialized: Materialized[String, Long, ByteArrayWindowStore] =
Materialized.as[String, Long](storeSupplier)
val internalMaterialized = new MaterializedInternal(materialized)
internalMaterialized.keySerde.getClass shouldBe Serdes.String.getClass
internalMaterialized.valueSerde.getClass shouldBe Serdes.Long.getClass
internalMaterialized.storeSupplier shouldBe storeSupplier
}
"Create a Materialize with a key value store supplier" should "create a Materialized with Serdes and a store supplier" in {
val storeSupplier = Stores.persistentKeyValueStore("store")
val materialized: Materialized[String, Long, ByteArrayKeyValueStore] =
Materialized.as[String, Long](storeSupplier)
val internalMaterialized = new MaterializedInternal(materialized)
internalMaterialized.keySerde.getClass shouldBe Serdes.String.getClass
internalMaterialized.valueSerde.getClass shouldBe Serdes.Long.getClass
internalMaterialized.storeSupplier shouldBe storeSupplier
}
"Create a Materialize with a session store supplier" should "create a Materialized with Serdes and a store supplier" in {
val storeSupplier = Stores.persistentSessionStore("store", Duration.ofMillis(1))
val materialized: Materialized[String, Long, ByteArraySessionStore] =
Materialized.as[String, Long](storeSupplier)
val internalMaterialized = new MaterializedInternal(materialized)
internalMaterialized.keySerde.getClass shouldBe Serdes.String.getClass
internalMaterialized.valueSerde.getClass shouldBe Serdes.Long.getClass
internalMaterialized.storeSupplier shouldBe storeSupplier
}
}

View File

@@ -0,0 +1,51 @@
/*
* Copyright (C) 2018 Joan Goyeau.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import org.apache.kafka.streams.kstream.internals.ProducedInternal
import org.apache.kafka.streams.processor.StreamPartitioner
import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.scala.Serdes
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@RunWith(classOf[JUnitRunner])
class ProducedTest extends FlatSpec with Matchers {
"Create a Produced" should "create a Produced with Serdes" in {
val produced: Produced[String, Long] = Produced.`with`[String, Long]
val internalProduced = new ProducedInternal(produced)
internalProduced.keySerde.getClass shouldBe Serdes.String.getClass
internalProduced.valueSerde.getClass shouldBe Serdes.Long.getClass
}
"Create a Produced with timestampExtractor and resetPolicy" should "create a Consumed with Serdes, timestampExtractor and resetPolicy" in {
val partitioner = new StreamPartitioner[String, Long] {
override def partition(topic: String, key: String, value: Long, numPartitions: Int): Integer = 0
}
val produced: Produced[String, Long] = Produced.`with`(partitioner)
val internalConsumed = new ProducedInternal(produced)
internalConsumed.keySerde.getClass shouldBe Serdes.String.getClass
internalConsumed.valueSerde.getClass shouldBe Serdes.Long.getClass
internalConsumed.streamPartitioner shouldBe partitioner
}
}

View File

@@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import java.time.Duration
import org.apache.kafka.streams.kstream.internals.StreamJoinedInternal
import org.apache.kafka.streams.scala.Serdes
import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.state.Stores
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@RunWith(classOf[JUnitRunner])
class StreamJoinedTest extends FlatSpec with Matchers {
"Create a StreamJoined" should "create a StreamJoined with Serdes" in {
val streamJoined: StreamJoined[String, String, Long] = StreamJoined.`with`[String, String, Long]
val streamJoinedInternal = new StreamJoinedInternal[String, String, Long](streamJoined)
streamJoinedInternal.keySerde().getClass shouldBe Serdes.String.getClass
streamJoinedInternal.valueSerde().getClass shouldBe Serdes.String.getClass
streamJoinedInternal.otherValueSerde().getClass shouldBe Serdes.Long.getClass
}
"Create a StreamJoined" should "create a StreamJoined with Serdes and Store Suppliers" in {
val storeSupplier = Stores.inMemoryWindowStore("myStore", Duration.ofMillis(500), Duration.ofMillis(250), false)
val otherStoreSupplier =
Stores.inMemoryWindowStore("otherStore", Duration.ofMillis(500), Duration.ofMillis(250), false)
val streamJoined: StreamJoined[String, String, Long] =
StreamJoined.`with`[String, String, Long](storeSupplier, otherStoreSupplier)
val streamJoinedInternal = new StreamJoinedInternal[String, String, Long](streamJoined)
streamJoinedInternal.keySerde().getClass shouldBe Serdes.String.getClass
streamJoinedInternal.valueSerde().getClass shouldBe Serdes.String.getClass
streamJoinedInternal.otherValueSerde().getClass shouldBe Serdes.Long.getClass
streamJoinedInternal.otherStoreSupplier().equals(otherStoreSupplier)
streamJoinedInternal.thisStoreSupplier().equals(storeSupplier)
}
"Create a StreamJoined" should "create a StreamJoined with Serdes and a State Store name" in {
val streamJoined: StreamJoined[String, String, Long] = StreamJoined.as[String, String, Long]("myStoreName")
val streamJoinedInternal = new StreamJoinedInternal[String, String, Long](streamJoined)
streamJoinedInternal.keySerde().getClass shouldBe Serdes.String.getClass
streamJoinedInternal.valueSerde().getClass shouldBe Serdes.String.getClass
streamJoinedInternal.otherValueSerde().getClass shouldBe Serdes.Long.getClass
streamJoinedInternal.storeName().equals("myStoreName")
}
}

View File

@@ -0,0 +1,98 @@
/*
* Copyright (C) 2018 Joan Goyeau.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.kstream
import java.time.Duration
import org.apache.kafka.streams.kstream.internals.suppress.{
BufferFullStrategy,
EagerBufferConfigImpl,
FinalResultsSuppressionBuilder,
StrictBufferConfigImpl,
SuppressedInternal
}
import org.apache.kafka.streams.scala.kstream.Suppressed.BufferConfig
import org.junit.runner.RunWith
import org.scalatest.{FlatSpec, Matchers}
import org.scalatestplus.junit.JUnitRunner
@deprecated(message = "org.apache.kafka.streams.scala.kstream.Suppressed has been deprecated", since = "2.5")
@RunWith(classOf[JUnitRunner])
class SuppressedTest extends FlatSpec with Matchers {
"Suppressed.untilWindowCloses" should "produce the correct suppression" in {
val bufferConfig = BufferConfig.unbounded()
val suppression = Suppressed.untilWindowCloses[String](bufferConfig)
suppression shouldEqual new FinalResultsSuppressionBuilder(null, bufferConfig)
suppression.withName("soup") shouldEqual new FinalResultsSuppressionBuilder("soup", bufferConfig)
}
"Suppressed.untilTimeLimit" should "produce the correct suppression" in {
val bufferConfig = BufferConfig.unbounded()
val duration = Duration.ofMillis(1)
Suppressed.untilTimeLimit[String](duration, bufferConfig) shouldEqual
new SuppressedInternal[String](null, duration, bufferConfig, null, false)
}
"BufferConfig.maxRecords" should "produce the correct buffer config" in {
BufferConfig.maxRecords(4) shouldEqual new EagerBufferConfigImpl(4, Long.MaxValue)
BufferConfig.maxRecords(4).withMaxBytes(5) shouldEqual new EagerBufferConfigImpl(4, 5)
}
"BufferConfig.maxBytes" should "produce the correct buffer config" in {
BufferConfig.maxBytes(4) shouldEqual new EagerBufferConfigImpl(Long.MaxValue, 4)
BufferConfig.maxBytes(4).withMaxRecords(5) shouldEqual new EagerBufferConfigImpl(5, 4)
}
"BufferConfig.unbounded" should "produce the correct buffer config" in {
BufferConfig.unbounded() shouldEqual
new StrictBufferConfigImpl(Long.MaxValue, Long.MaxValue, BufferFullStrategy.SHUT_DOWN)
}
"BufferConfig" should "support very long chains of factory methods" in {
val bc1 = BufferConfig
.unbounded()
.emitEarlyWhenFull()
.withMaxRecords(3L)
.withMaxBytes(4L)
.withMaxRecords(5L)
.withMaxBytes(6L)
bc1 shouldEqual new EagerBufferConfigImpl(5L, 6L)
bc1.shutDownWhenFull() shouldEqual new StrictBufferConfigImpl(5L, 6L, BufferFullStrategy.SHUT_DOWN)
val bc2 = BufferConfig
.maxBytes(4)
.withMaxRecords(5)
.withMaxBytes(6)
.withNoBound()
.withMaxBytes(7)
.withMaxRecords(8)
bc2 shouldEqual new StrictBufferConfigImpl(8L, 7L, BufferFullStrategy.SHUT_DOWN)
bc2.withNoBound() shouldEqual BufferConfig.unbounded()
val bc3 = BufferConfig
.maxRecords(5L)
.withMaxBytes(10L)
.emitEarlyWhenFull()
.withMaxRecords(11L)
bc3 shouldEqual new EagerBufferConfigImpl(11L, 10L)
}
}

View File

@@ -0,0 +1,136 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.utils
import java.util.Properties
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.clients.producer.ProducerConfig
import org.apache.kafka.common.serialization._
import org.apache.kafka.common.utils.MockTime
import org.apache.kafka.streams._
import org.apache.kafka.streams.integration.utils.{EmbeddedKafkaCluster, IntegrationTestUtils}
import org.apache.kafka.test.{IntegrationTest, TestUtils}
import org.junit._
import org.junit.experimental.categories.Category
import org.junit.rules.TemporaryFolder
/**
* Test suite base that prepares Kafka cluster for stream-table joins in Kafka Streams
* <p>
*/
@Category(Array(classOf[IntegrationTest]))
class StreamToTableJoinScalaIntegrationTestBase extends StreamToTableJoinTestData {
private val privateCluster: EmbeddedKafkaCluster = new EmbeddedKafkaCluster(1)
@Rule def cluster: EmbeddedKafkaCluster = privateCluster
final val alignedTime = (System.currentTimeMillis() / 1000 + 1) * 1000
val mockTime: MockTime = cluster.time
mockTime.setCurrentTimeMs(alignedTime)
val tFolder: TemporaryFolder = new TemporaryFolder(TestUtils.tempDirectory())
@Rule def testFolder: TemporaryFolder = tFolder
@Before
def startKafkaCluster(): Unit = {
cluster.createTopic(userClicksTopic)
cluster.createTopic(userRegionsTopic)
cluster.createTopic(outputTopic)
cluster.createTopic(userClicksTopicJ)
cluster.createTopic(userRegionsTopicJ)
cluster.createTopic(outputTopicJ)
}
def getStreamsConfiguration(): Properties = {
val streamsConfiguration: Properties = new Properties()
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "stream-table-join-scala-integration-test")
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, "1000")
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot.getPath)
streamsConfiguration
}
private def getUserRegionsProducerConfig(): Properties = {
val p = new Properties()
p.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
p.put(ProducerConfig.ACKS_CONFIG, "all")
p.put(ProducerConfig.RETRIES_CONFIG, "0")
p.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer])
p.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer])
p
}
private def getUserClicksProducerConfig(): Properties = {
val p = new Properties()
p.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
p.put(ProducerConfig.ACKS_CONFIG, "all")
p.put(ProducerConfig.RETRIES_CONFIG, "0")
p.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer])
p.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[LongSerializer])
p
}
private def getConsumerConfig(): Properties = {
val p = new Properties()
p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers())
p.put(ConsumerConfig.GROUP_ID_CONFIG, "join-scala-integration-test-standard-consumer")
p.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer])
p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[LongDeserializer])
p
}
def produceNConsume(userClicksTopic: String,
userRegionsTopic: String,
outputTopic: String,
waitTillRecordsReceived: Boolean = true): java.util.List[KeyValue[String, Long]] = {
import collection.JavaConverters._
// Publish user-region information.
val userRegionsProducerConfig: Properties = getUserRegionsProducerConfig()
IntegrationTestUtils.produceKeyValuesSynchronously(userRegionsTopic,
userRegions.asJava,
userRegionsProducerConfig,
mockTime,
false)
// Publish user-click information.
val userClicksProducerConfig: Properties = getUserClicksProducerConfig()
IntegrationTestUtils.produceKeyValuesSynchronously(userClicksTopic,
userClicks.asJava,
userClicksProducerConfig,
mockTime,
false)
if (waitTillRecordsReceived) {
// consume and verify result
val consumerConfig = getConsumerConfig()
IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived(consumerConfig,
outputTopic,
expectedClicksPerRegion.asJava)
} else {
java.util.Collections.emptyList()
}
}
}

View File

@@ -0,0 +1,60 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2017-2018 Alexis Seigneurin.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.utils
import org.apache.kafka.streams.KeyValue
trait StreamToTableJoinTestData {
val brokers = "localhost:9092"
val userClicksTopic = s"user-clicks"
val userRegionsTopic = s"user-regions"
val outputTopic = s"output-topic"
val userClicksTopicJ = s"user-clicks-j"
val userRegionsTopicJ = s"user-regions-j"
val outputTopicJ = s"output-topic-j"
// Input 1: Clicks per user (multiple records allowed per user).
val userClicks: Seq[KeyValue[String, Long]] = Seq(
new KeyValue("alice", 13L),
new KeyValue("bob", 4L),
new KeyValue("chao", 25L),
new KeyValue("bob", 19L),
new KeyValue("dave", 56L),
new KeyValue("eve", 78L),
new KeyValue("alice", 40L),
new KeyValue("fang", 99L)
)
// Input 2: Region per user (multiple records allowed per user).
val userRegions: Seq[KeyValue[String, String]] = Seq(
new KeyValue("alice", "asia"), /* Alice lived in Asia originally... */
new KeyValue("bob", "americas"),
new KeyValue("chao", "asia"),
new KeyValue("dave", "europe"),
new KeyValue("alice", "europe"), /* ...but moved to Europe some time later. */
new KeyValue("eve", "americas"),
new KeyValue("fang", "asia")
)
val expectedClicksPerRegion: Seq[KeyValue[String, Long]] = Seq(
new KeyValue("americas", 101L),
new KeyValue("europe", 109L),
new KeyValue("asia", 124L)
)
}

View File

@@ -0,0 +1,46 @@
/*
* Copyright (C) 2018 Joan Goyeau.
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.scala.utils
import java.time.Instant
import java.util.{Properties, UUID}
import org.apache.kafka.common.serialization.Serde
import org.apache.kafka.streams.scala.StreamsBuilder
import org.apache.kafka.streams.{StreamsConfig, TestInputTopic, TestOutputTopic, TopologyTestDriver}
import org.scalatest.Suite
trait TestDriver { this: Suite =>
def createTestDriver(builder: StreamsBuilder, initialWallClockTime: Instant = Instant.now()): TopologyTestDriver = {
val config = new Properties()
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "test")
config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234")
config.put(StreamsConfig.STATE_DIR_CONFIG, s"out/state-store-${UUID.randomUUID()}")
new TopologyTestDriver(builder.build(), config, initialWallClockTime)
}
implicit class TopologyTestDriverOps(inner: TopologyTestDriver) {
def createInput[K, V](topic: String)(implicit serdeKey: Serde[K], serdeValue: Serde[V]): TestInputTopic[K, V] =
inner.createInputTopic(topic, serdeKey.serializer, serdeValue.serializer)
def createOutput[K, V](topic: String)(implicit serdeKey: Serde[K], serdeValue: Serde[V]): TestOutputTopic[K, V] =
inner.createOutputTopic(topic, serdeKey.deserializer, serdeValue.deserializer)
}
}