MINOR: Add Scalafmt to Streams Scala API (#4965)

Reviewers: Guozhang Wang <wangguoz@gmail.com>
This commit is contained in:
Joan Goyeau 2018-07-10 01:48:34 +02:00 committed by Guozhang Wang
parent 9a18f92935
commit 05c5854d1f
16 changed files with 318 additions and 282 deletions

View File

@ -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')
}
}

20
checkstyle/.scalafmt.conf Normal file
View File

@ -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]

View File

@ -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)
}

View File

@ -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)
}

View File

@ -38,7 +38,8 @@ object Serdes {
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(

View File

@ -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)
/**

View File

@ -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.
*

View File

@ -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)
}
/**
@ -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]] = {
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] = {
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()
}
}
}
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

View File

@ -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)
/**
@ -187,9 +182,8 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
* @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).
@ -213,9 +207,8 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
*/
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
}

View File

@ -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)

View File

@ -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)
@ -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)
}
}

View File

@ -58,4 +58,3 @@ trait StreamToTableJoinTestData {
new KeyValue("asia", 124L)
)
}

View File

@ -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)
}
}

View File

@ -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)
)
}