mirror of https://github.com/apache/kafka.git
MINOR: Add Scalafmt to Streams Scala API (#4965)
Reviewers: Guozhang Wang <wangguoz@gmail.com>
This commit is contained in:
parent
9a18f92935
commit
05c5854d1f
|
@ -29,6 +29,15 @@ buildscript {
|
|||
classpath 'org.scoverage:gradle-scoverage:2.3.0'
|
||||
classpath 'com.github.jengelman.gradle.plugins:shadow:2.0.4'
|
||||
classpath 'org.owasp:dependency-check-gradle:3.2.1'
|
||||
classpath "com.diffplug.spotless:spotless-plugin-gradle:3.10.0"
|
||||
}
|
||||
}
|
||||
|
||||
apply plugin: "com.diffplug.gradle.spotless"
|
||||
spotless {
|
||||
scala {
|
||||
target 'streams/**/*.scala'
|
||||
scalafmt('1.5.1').configFile('checkstyle/.scalafmt.conf')
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,20 @@
|
|||
// 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.
|
||||
docstrings = JavaDoc
|
||||
maxColumn = 120
|
||||
continuationIndent.defnSite = 2
|
||||
assumeStandardLibraryStripMargin = true
|
||||
danglingParentheses = true
|
||||
rewrite.rules = [SortImports, RedundantBraces, RedundantParens, SortModifiers]
|
|
@ -40,7 +40,7 @@ object FunctionConversions {
|
|||
}
|
||||
}
|
||||
|
||||
implicit class MapperFromFunction[T, U, VR](val f:(T,U) => VR) extends AnyVal {
|
||||
implicit class MapperFromFunction[T, U, VR](val f: (T, U) => VR) extends AnyVal {
|
||||
def asKeyValueMapper: KeyValueMapper[T, U, VR] = new KeyValueMapper[T, U, VR] {
|
||||
override def apply(key: T, value: U): VR = f(key, value)
|
||||
}
|
||||
|
@ -49,7 +49,7 @@ object FunctionConversions {
|
|||
}
|
||||
}
|
||||
|
||||
implicit class KeyValueMapperFromFunction[K, V, KR, VR](val f:(K,V) => (KR, VR)) extends AnyVal {
|
||||
implicit class KeyValueMapperFromFunction[K, V, KR, VR](val f: (K, V) => (KR, VR)) extends AnyVal {
|
||||
def asKeyValueMapper: KeyValueMapper[K, V, KeyValue[KR, VR]] = new KeyValueMapper[K, V, KeyValue[KR, VR]] {
|
||||
override def apply(key: K, value: V): KeyValue[KR, VR] = {
|
||||
val (kr, vr) = f(key, value)
|
||||
|
@ -88,7 +88,7 @@ object FunctionConversions {
|
|||
}
|
||||
}
|
||||
|
||||
implicit class MergerFromFunction[K,VR](val f: (K, VR, VR) => VR) extends AnyVal {
|
||||
implicit class MergerFromFunction[K, VR](val f: (K, VR, VR) => VR) extends AnyVal {
|
||||
def asMerger: Merger[K, VR] = new Merger[K, VR] {
|
||||
override def apply(aggKey: K, aggOne: VR, aggTwo: VR): VR = f(aggKey, aggOne, aggTwo)
|
||||
}
|
||||
|
|
|
@ -77,7 +77,8 @@ object ImplicitConversions {
|
|||
valueSerde: Serde[V]): Materialized[K, V, S] =
|
||||
Materialized.`with`[K, V, S](keySerde, valueSerde)
|
||||
|
||||
implicit def joinedFromKeyValueOtherSerde[K, V, VO]
|
||||
(implicit keySerde: Serde[K], valueSerde: Serde[V], otherValueSerde: Serde[VO]): Joined[K, V, VO] =
|
||||
implicit def joinedFromKeyValueOtherSerde[K, V, VO](implicit keySerde: Serde[K],
|
||||
valueSerde: Serde[V],
|
||||
otherValueSerde: Serde[VO]): Joined[K, V, VO] =
|
||||
Joined.`with`(keySerde, valueSerde, otherValueSerde)
|
||||
}
|
||||
|
|
|
@ -25,47 +25,48 @@ import org.apache.kafka.common.serialization.{Deserializer, Serde, Serializer, S
|
|||
import org.apache.kafka.streams.kstream.WindowedSerdes
|
||||
|
||||
object Serdes {
|
||||
implicit val String: Serde[String] = JSerdes.String()
|
||||
implicit val Long: Serde[Long] = JSerdes.Long().asInstanceOf[Serde[Long]]
|
||||
implicit val JavaLong: Serde[java.lang.Long] = JSerdes.Long()
|
||||
implicit val ByteArray: Serde[Array[Byte]] = JSerdes.ByteArray()
|
||||
implicit val String: Serde[String] = JSerdes.String()
|
||||
implicit val Long: Serde[Long] = JSerdes.Long().asInstanceOf[Serde[Long]]
|
||||
implicit val JavaLong: Serde[java.lang.Long] = JSerdes.Long()
|
||||
implicit val ByteArray: Serde[Array[Byte]] = JSerdes.ByteArray()
|
||||
implicit val Bytes: Serde[org.apache.kafka.common.utils.Bytes] = JSerdes.Bytes()
|
||||
implicit val Float: Serde[Float] = JSerdes.Float().asInstanceOf[Serde[Float]]
|
||||
implicit val JavaFloat: Serde[java.lang.Float] = JSerdes.Float()
|
||||
implicit val Double: Serde[Double] = JSerdes.Double().asInstanceOf[Serde[Double]]
|
||||
implicit val JavaDouble: Serde[java.lang.Double] = JSerdes.Double()
|
||||
implicit val Integer: Serde[Int] = JSerdes.Integer().asInstanceOf[Serde[Int]]
|
||||
implicit val JavaInteger: Serde[java.lang.Integer] = JSerdes.Integer()
|
||||
implicit val Float: Serde[Float] = JSerdes.Float().asInstanceOf[Serde[Float]]
|
||||
implicit val JavaFloat: Serde[java.lang.Float] = JSerdes.Float()
|
||||
implicit val Double: Serde[Double] = JSerdes.Double().asInstanceOf[Serde[Double]]
|
||||
implicit val JavaDouble: Serde[java.lang.Double] = JSerdes.Double()
|
||||
implicit val Integer: Serde[Int] = JSerdes.Integer().asInstanceOf[Serde[Int]]
|
||||
implicit val JavaInteger: Serde[java.lang.Integer] = JSerdes.Integer()
|
||||
|
||||
implicit def timeWindowedSerde[T]: WindowedSerdes.TimeWindowedSerde[T] = new WindowedSerdes.TimeWindowedSerde[T]()
|
||||
implicit def sessionWindowedSerde[T]: WindowedSerdes.SessionWindowedSerde[T] = new WindowedSerdes.SessionWindowedSerde[T]()
|
||||
implicit def sessionWindowedSerde[T]: WindowedSerdes.SessionWindowedSerde[T] =
|
||||
new WindowedSerdes.SessionWindowedSerde[T]()
|
||||
|
||||
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 serialize(topic: String, data: T): Array[Byte] = serializer(data)
|
||||
override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = ()
|
||||
override def close(): Unit = ()
|
||||
override def close(): Unit = ()
|
||||
},
|
||||
new Deserializer[T] {
|
||||
override def deserialize(topic: String, data: Array[Byte]): T = deserializer(data).orNull
|
||||
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 = ()
|
||||
override def close(): Unit = ()
|
||||
}
|
||||
)
|
||||
|
||||
def fromFn[T >: Null](serializer: (String, T) => Array[Byte],
|
||||
deserializer: (String, Array[Byte]) => Option[T]): Serde[T] =
|
||||
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 serialize(topic: String, data: T): Array[Byte] = serializer(topic, data)
|
||||
override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = ()
|
||||
override def close(): Unit = ()
|
||||
override def close(): Unit = ()
|
||||
},
|
||||
new Deserializer[T] {
|
||||
override def deserialize(topic: String, data: Array[Byte]): T = deserializer(topic, data).orNull
|
||||
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 = ()
|
||||
override def close(): Unit = ()
|
||||
}
|
||||
)
|
||||
}
|
||||
|
|
|
@ -31,8 +31,8 @@ import ImplicitConversions._
|
|||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* Wraps the Java class StreamsBuilder and delegates method calls to the underlying Java object.
|
||||
*/
|
||||
* Wraps the Java class StreamsBuilder and delegates method calls to the underlying Java object.
|
||||
*/
|
||||
class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) {
|
||||
|
||||
/**
|
||||
|
@ -123,8 +123,9 @@ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) {
|
|||
* @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] =
|
||||
def table[K, V](topic: String, materialized: Materialized[K, V, ByteArrayKeyValueStore])(
|
||||
implicit consumed: Consumed[K, V]
|
||||
): KTable[K, V] =
|
||||
inner.table[K, V](topic, consumed, materialized)
|
||||
|
||||
/**
|
||||
|
@ -148,8 +149,9 @@ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) {
|
|||
* @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] =
|
||||
def globalTable[K, V](topic: String, materialized: Materialized[K, V, ByteArrayKeyValueStore])(
|
||||
implicit consumed: Consumed[K, V]
|
||||
): GlobalKTable[K, V] =
|
||||
inner.globalTable(topic, consumed, materialized)
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.kafka.streams.kstream.{KGroupedStream => KGroupedStreamJ, _}
|
|||
import org.apache.kafka.streams.scala.ImplicitConversions._
|
||||
import org.apache.kafka.streams.scala.FunctionConversions._
|
||||
|
||||
|
||||
/**
|
||||
* Wraps the Java class KGroupedStream and delegates method calls to the underlying Java object.
|
||||
*
|
||||
|
|
|
@ -125,7 +125,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
*/
|
||||
def flatMap[KR, VR](mapper: (K, V) => Iterable[(KR, VR)]): KStream[KR, VR] = {
|
||||
val kvMapper = mapper.tupled andThen (iter => iter.map(tuple2ToKeyValue).asJava)
|
||||
inner.flatMap[KR, VR](((k: K, v: V) => kvMapper(k , v)).asKeyValueMapper)
|
||||
inner.flatMap[KR, VR](((k: K, v: V) => kvMapper(k, v)).asKeyValueMapper)
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -250,34 +250,34 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
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 (implicit) produced the instance of Produced that gives the serdes and `StreamPartitioner`
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#to`
|
||||
*/
|
||||
* 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 (implicit) 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)
|
||||
|
||||
|
@ -293,24 +293,22 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @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](transformer: Transformer[K, V, (K1, V1)],
|
||||
stateStoreNames: String*): KStream[K1, V1] = {
|
||||
val transformerSupplierJ: TransformerSupplier[K, V, KeyValue[K1, V1]] = new TransformerSupplier[K, V, KeyValue[K1, V1]] {
|
||||
override def get(): Transformer[K, V, KeyValue[K1, V1]] = {
|
||||
new Transformer[K, V, KeyValue[K1, V1]] {
|
||||
override def transform(key: K, value: V): KeyValue[K1, V1] = {
|
||||
transformer.transform(key, value) match {
|
||||
case (k1, v1) => KeyValue.pair(k1, v1)
|
||||
case _ => null
|
||||
}
|
||||
def transform[K1, V1](transformer: Transformer[K, V, (K1, V1)], stateStoreNames: String*): KStream[K1, V1] = {
|
||||
val transformerSupplierJ: TransformerSupplier[K, V, KeyValue[K1, V1]] =
|
||||
new TransformerSupplier[K, V, KeyValue[K1, V1]] {
|
||||
override def get(): Transformer[K, V, KeyValue[K1, V1]] =
|
||||
new Transformer[K, V, KeyValue[K1, V1]] {
|
||||
override def transform(key: K, value: V): KeyValue[K1, V1] =
|
||||
transformer.transform(key, value) match {
|
||||
case (k1, v1) => KeyValue.pair(k1, v1)
|
||||
case _ => null
|
||||
}
|
||||
|
||||
override def init(context: ProcessorContext): Unit = transformer.init(context)
|
||||
|
||||
override def close(): Unit = transformer.close()
|
||||
}
|
||||
|
||||
override def init(context: ProcessorContext): Unit = transformer.init(context)
|
||||
|
||||
override def close(): Unit = transformer.close()
|
||||
}
|
||||
}
|
||||
}
|
||||
inner.transform(transformerSupplierJ, stateStoreNames: _*)
|
||||
}
|
||||
|
||||
|
@ -343,9 +341,8 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
def transformValues[VR](valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, VR],
|
||||
stateStoreNames: String*): KStream[K, VR] = {
|
||||
stateStoreNames: String*): KStream[K, VR] =
|
||||
inner.transformValues[VR](valueTransformerSupplier, stateStoreNames: _*)
|
||||
}
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a `Processor` (provided by the given
|
||||
|
|
|
@ -45,9 +45,8 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
|
|||
* @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] = {
|
||||
def filter(predicate: (K, V) => Boolean): KTable[K, V] =
|
||||
inner.filter(predicate(_, _))
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new [[KTable]] that consists all records of this [[KTable]] which satisfies the given
|
||||
|
@ -59,8 +58,7 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
|
|||
* @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] =
|
||||
def filter(predicate: (K, V) => Boolean, materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
|
||||
inner.filter(predicate.asPredicate, materialized)
|
||||
|
||||
/**
|
||||
|
@ -84,8 +82,7 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
|
|||
* @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] =
|
||||
def filterNot(predicate: (K, V) => Boolean, materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
|
||||
inner.filterNot(predicate.asPredicate, materialized)
|
||||
|
||||
/**
|
||||
|
@ -113,8 +110,7 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
|
|||
* @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] =
|
||||
def mapValues[VR](mapper: V => VR, materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] =
|
||||
inner.mapValues[VR](mapper.asValueMapper, materialized)
|
||||
|
||||
/**
|
||||
|
@ -142,8 +138,7 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
|
|||
* @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] =
|
||||
def mapValues[VR](mapper: (K, V) => VR, materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] =
|
||||
inner.mapValues[VR](mapper.asValueMapperWithKey)
|
||||
|
||||
/**
|
||||
|
@ -165,57 +160,55 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
|
|||
inner.toStream[KR](mapper.asKeyValueMapper)
|
||||
|
||||
/**
|
||||
* 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`
|
||||
*/
|
||||
* 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] = {
|
||||
stateStoreNames: String*): KTable[K, VR] =
|
||||
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`
|
||||
*/
|
||||
* 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] = {
|
||||
stateStoreNames: String*): KTable[K, VR] =
|
||||
inner.transformValues[VR](valueTransformerWithKeySupplier, materialized, stateStoreNames: _*)
|
||||
}
|
||||
|
||||
/**
|
||||
* Re-groups the records of this [[KTable]] using the provided key/value mapper
|
||||
|
@ -323,4 +316,3 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
|
|||
*/
|
||||
def queryableStoreName: String = inner.queryableStoreName
|
||||
}
|
||||
|
||||
|
|
|
@ -46,8 +46,7 @@ class SessionWindowedKStream[K, V](val inner: SessionWindowedKStreamJ[K, V]) {
|
|||
* 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)(
|
||||
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] =
|
||||
inner.aggregate((() => initializer).asInitializer, aggregator.asAggregator, merger.asMerger, materialized)
|
||||
|
|
|
@ -41,8 +41,7 @@ import org.scalatest.junit.JUnitSuite
|
|||
* Note: In the current project settings SAM type conversion is turned off as it's experimental in Scala 2.11.
|
||||
* Hence the native Java API based version is more verbose.
|
||||
*/
|
||||
class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite
|
||||
with StreamToTableJoinTestData {
|
||||
class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite with StreamToTableJoinTestData {
|
||||
|
||||
private val privateCluster: EmbeddedKafkaCluster = new EmbeddedKafkaCluster(1)
|
||||
|
||||
|
@ -84,7 +83,7 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite
|
|||
val clicksPerRegion: KTable[String, Long] =
|
||||
userClicksStream
|
||||
|
||||
// Join the stream against the table.
|
||||
// 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>
|
||||
|
@ -100,7 +99,6 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite
|
|||
val streams: KafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration)
|
||||
streams.start()
|
||||
|
||||
|
||||
val actualClicksPerRegion: java.util.List[KeyValue[String, Long]] =
|
||||
produceNConsume(userClicksTopic, userRegionsTopic, outputTopic)
|
||||
|
||||
|
@ -134,18 +132,21 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite
|
|||
|
||||
// Join the stream against the table.
|
||||
val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream
|
||||
.leftJoin(userRegionsTable,
|
||||
.leftJoin(
|
||||
userRegionsTable,
|
||||
new ValueJoiner[JLong, String, (String, JLong)] {
|
||||
def apply(clicks: JLong, region: String): (String, JLong) =
|
||||
(if (region == null) "UNKNOWN" else region, clicks)
|
||||
},
|
||||
Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String))
|
||||
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
|
||||
val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion
|
||||
.map {
|
||||
new KeyValueMapper[String, (String, JLong), KeyValue[String, JLong]] {
|
||||
def apply(k: String, regionWithClicks: (String, JLong)) = new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2)
|
||||
def apply(k: String, regionWithClicks: (String, JLong)) =
|
||||
new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -214,17 +215,27 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite
|
|||
p
|
||||
}
|
||||
|
||||
private def produceNConsume(userClicksTopic: String, userRegionsTopic: String, outputTopic: String): java.util.List[KeyValue[String, Long]] = {
|
||||
private def produceNConsume(userClicksTopic: String,
|
||||
userRegionsTopic: String,
|
||||
outputTopic: String): 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)
|
||||
IntegrationTestUtils.produceKeyValuesSynchronously(userRegionsTopic,
|
||||
userRegions.asJava,
|
||||
userRegionsProducerConfig,
|
||||
mockTime,
|
||||
false)
|
||||
|
||||
// Publish user-click information.
|
||||
val userClicksProducerConfig: Properties = getUserClicksProducerConfig()
|
||||
IntegrationTestUtils.produceKeyValuesSynchronously(userClicksTopic, userClicks.asJava, userClicksProducerConfig, mockTime, false)
|
||||
IntegrationTestUtils.produceKeyValuesSynchronously(userClicksTopic,
|
||||
userClicks.asJava,
|
||||
userClicksProducerConfig,
|
||||
mockTime,
|
||||
false)
|
||||
|
||||
// consume and verify result
|
||||
val consumerConfig = getConsumerConfig()
|
||||
|
@ -232,4 +243,3 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite
|
|||
IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(consumerConfig, outputTopic, expectedClicksPerRegion.size)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -58,4 +58,3 @@ trait StreamToTableJoinTestData {
|
|||
new KeyValue("asia", 124L)
|
||||
)
|
||||
}
|
||||
|
||||
|
|
|
@ -89,7 +89,8 @@ class TopologyTest extends JUnitSuite {
|
|||
val textLines = streamBuilder.stream[String, String](inputTopic)
|
||||
|
||||
val _: KTable[String, Long] =
|
||||
textLines.flatMapValues(v => pattern.split(v.toLowerCase))
|
||||
textLines
|
||||
.flatMapValues(v => pattern.split(v.toLowerCase))
|
||||
.groupBy((k, v) => v)
|
||||
.count()
|
||||
|
||||
|
@ -160,18 +161,21 @@ class TopologyTest extends JUnitSuite {
|
|||
|
||||
// Join the stream against the table.
|
||||
val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream
|
||||
.leftJoin(userRegionsTable,
|
||||
.leftJoin(
|
||||
userRegionsTable,
|
||||
new ValueJoiner[JLong, String, (String, JLong)] {
|
||||
def apply(clicks: JLong, region: String): (String, JLong) =
|
||||
(if (region == null) "UNKNOWN" else region, clicks)
|
||||
},
|
||||
Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String))
|
||||
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
|
||||
val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion
|
||||
.map {
|
||||
new KeyValueMapper[String, (String, JLong), KeyValue[String, JLong]] {
|
||||
def apply(k: String, regionWithClicks: (String, JLong)) = new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2)
|
||||
def apply(k: String, regionWithClicks: (String, JLong)) =
|
||||
new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -61,11 +61,8 @@ class WordCountTest extends JUnitSuite with WordCountTestData {
|
|||
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)
|
||||
|
@ -86,7 +83,8 @@ class WordCountTest extends JUnitSuite with WordCountTestData {
|
|||
|
||||
// generate word counts
|
||||
val wordCounts: KTable[String, Long] =
|
||||
textLines.flatMapValues(v => pattern.split(v.toLowerCase))
|
||||
textLines
|
||||
.flatMapValues(v => pattern.split(v.toLowerCase))
|
||||
.groupBy((_, v) => v)
|
||||
.count()
|
||||
|
||||
|
@ -117,7 +115,8 @@ class WordCountTest extends JUnitSuite with WordCountTestData {
|
|||
|
||||
// generate word counts
|
||||
val wordCounts: KTable[String, Long] =
|
||||
textLines.flatMapValues(v => pattern.split(v.toLowerCase))
|
||||
textLines
|
||||
.flatMapValues(v => pattern.split(v.toLowerCase))
|
||||
.groupBy((k, v) => v)
|
||||
.count()(Materialized.as("word-count"))
|
||||
|
||||
|
@ -139,7 +138,12 @@ class WordCountTest extends JUnitSuite with WordCountTestData {
|
|||
@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 org.apache.kafka.streams.kstream.{
|
||||
KTable => KTableJ,
|
||||
KStream => KStreamJ,
|
||||
KGroupedStream => KGroupedStreamJ,
|
||||
_
|
||||
}
|
||||
import collection.JavaConverters._
|
||||
|
||||
val streamsConfiguration = getStreamsConfiguration()
|
||||
|
@ -250,4 +254,3 @@ trait WordCountTestData {
|
|||
new KeyValue("слова", 1L)
|
||||
)
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue