You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by gu...@apache.org on 2018/04/17 20:13:19 UTC
[kafka] branch trunk updated: KAFKA-6376: preliminary cleanup
(#4872)
This is an automated email from the ASF dual-hosted git repository.
guozhang pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new ac9c3ed KAFKA-6376: preliminary cleanup (#4872)
ac9c3ed is described below
commit ac9c3ed0b43ee848e6e555a01c55ea2eee78540a
Author: John Roesler <vv...@users.noreply.github.com>
AuthorDate: Tue Apr 17 15:13:15 2018 -0500
KAFKA-6376: preliminary cleanup (#4872)
General cleanup of Streams code, mostly resolving compiler warnings and re-formatting.
The regular testing suite should be sufficient.
Reviewers: Matthias J. Sax <ma...@confluent.io>, Guozhang Wang <wa...@gmail.com>
---
.../java/org/apache/kafka/common/utils/Utils.java | 60 +++
.../kstream/internals/KStreamAggregate.java | 10 +-
.../kstream/internals/KStreamKStreamJoin.java | 6 +-
.../streams/kstream/internals/KStreamReduce.java | 14 +-
.../internals/KStreamSessionWindowAggregate.java | 17 +-
.../kstream/internals/KStreamWindowAggregate.java | 9 +-
.../kstream/internals/KStreamWindowReduce.java | 9 +-
.../streams/kstream/internals/KTableAggregate.java | 8 +-
.../kstream/internals/KTableKTableInnerJoin.java | 13 +-
.../kstream/internals/KTableKTableLeftJoin.java | 25 +-
.../kstream/internals/KTableKTableOuterJoin.java | 25 +-
.../kstream/internals/KTableKTableRightJoin.java | 22 +-
.../streams/kstream/internals/KTableReduce.java | 11 +-
.../kstream/internals/KTableRepartitionMap.java | 13 +-
.../streams/kstream/internals/KTableSource.java | 11 +-
.../streams/processor/internals/AbstractTask.java | 22 +-
.../processor/internals/GlobalStateUpdateTask.java | 4 +-
.../processor/internals/GlobalStreamThread.java | 64 +--
.../streams/processor/internals/ProcessorNode.java | 87 ++--
.../processor/internals/RecordCollectorImpl.java | 82 ++--
.../processor/internals/RecordDeserializer.java | 12 +-
.../streams/processor/internals/SourceNode.java | 23 +-
.../streams/processor/internals/StandbyTask.java | 2 +-
.../streams/processor/internals/StreamTask.java | 69 +--
.../streams/processor/internals/StreamThread.java | 368 ++++++++-------
.../processor/internals/StreamsMetricsImpl.java | 101 +++--
.../kafka/streams/state/internals/NamedCache.java | 79 ++--
.../kstream/internals/GlobalKTableJoinsTest.java | 22 +-
.../kstream/internals/KGroupedStreamImplTest.java | 150 ++++---
.../kstream/internals/KStreamKStreamJoinTest.java | 210 ++++-----
.../kstream/internals/KStreamKTableJoinTest.java | 12 +-
...KStreamSessionWindowAggregateProcessorTest.java | 83 ++--
.../internals/KStreamWindowAggregateTest.java | 140 +++---
.../internals/KTableKTableInnerJoinTest.java | 26 +-
.../internals/KTableKTableLeftJoinTest.java | 96 ++--
.../internals/KTableKTableOuterJoinTest.java | 70 ++-
.../kstream/internals/KTableSourceTest.java | 26 +-
.../processor/internals/GlobalStateTaskTest.java | 42 +-
.../processor/internals/MockStreamsMetrics.java | 5 +-
.../processor/internals/PartitionGroupTest.java | 10 +-
.../processor/internals/ProcessorNodeTest.java | 41 +-
.../processor/internals/RecordCollectorTest.java | 158 +++----
.../internals/RecordDeserializerTest.java | 9 +-
.../processor/internals/RecordQueueTest.java | 51 +--
.../streams/processor/internals/SinkNodeTest.java | 15 +-
.../processor/internals/StreamTaskTest.java | 492 +++++++++++----------
.../processor/internals/StreamThreadTest.java | 357 ++++++++-------
.../internals/StreamsMetricsImplTest.java | 70 +--
.../streams/state/KeyValueStoreTestDriver.java | 29 +-
.../state/internals/RocksDBWindowStoreTest.java | 389 +++++++++-------
.../state/internals/StoreChangeLoggerTest.java | 42 +-
.../StreamThreadStateStoreProviderTest.java | 68 ++-
.../org/apache/kafka/test/KStreamTestDriver.java | 6 +-
.../kafka/test/ProcessorTopologyTestDriver.java | 103 ++---
.../org/apache/kafka/test/StreamsTestUtils.java | 2 +-
.../apache/kafka/streams/TopologyTestDriver.java | 36 +-
.../streams/processor/MockProcessorContext.java | 16 +-
57 files changed, 2134 insertions(+), 1808 deletions(-)
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index ba621ab..f026308 100755
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -51,6 +51,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@@ -629,6 +630,65 @@ public final class Utils {
}
/**
+ * Creates a map entry (for use with {@link Utils#mkMap(java.util.Map.Entry[])})
+ *
+ * @param k The key
+ * @param v The value
+ * @param <K> The key type
+ * @param <V> The value type
+ * @return An entry
+ */
+ public static <K, V> Map.Entry<K, V> mkEntry(final K k, final V v) {
+ return new Map.Entry<K, V>() {
+ @Override
+ public K getKey() {
+ return k;
+ }
+
+ @Override
+ public V getValue() {
+ return v;
+ }
+
+ @Override
+ public V setValue(final V value) {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+
+ /**
+ * Creates a map from a sequence of entries
+ *
+ * @param entries The entries to map
+ * @param <K> The key type
+ * @param <V> The value type
+ * @return A map
+ */
+ @SafeVarargs
+ public static <K, V> Map<K, V> mkMap(final Map.Entry<K, V>... entries) {
+ final LinkedHashMap<K, V> result = new LinkedHashMap<>();
+ for (final Map.Entry<K, V> entry : entries) {
+ result.put(entry.getKey(), entry.getValue());
+ }
+ return result;
+ }
+
+ /**
+ * Creates a {@link Properties} from a map
+ *
+ * @param properties A map of properties to add
+ * @return The properties object
+ */
+ public static Properties mkProperties(final Map<String, String> properties) {
+ final Properties result = new Properties();
+ for (final Map.Entry<String, String> entry : properties.entrySet()) {
+ result.setProperty(entry.getKey(), entry.getValue());
+ }
+ return result;
+ }
+
+ /**
* Recursively delete the given file/directory and any subfiles (if any exist)
*
* @param file The root file at which to begin deleting
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java
index 95ad78e..1d2d173 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java
@@ -32,7 +32,7 @@ public class KStreamAggregate<K, V, T> implements KStreamAggProcessorSupplier<K,
private boolean sendOldValues = false;
- public KStreamAggregate(String storeName, Initializer<T> initializer, Aggregator<? super K, ? super V, T> aggregator) {
+ KStreamAggregate(final String storeName, final Initializer<T> initializer, final Aggregator<? super K, ? super V, T> aggregator) {
this.storeName = storeName;
this.initializer = initializer;
this.aggregator = aggregator;
@@ -55,7 +55,7 @@ public class KStreamAggregate<K, V, T> implements KStreamAggProcessorSupplier<K,
@SuppressWarnings("unchecked")
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
store = (KeyValueStore<K, T>) context.getStateStore(storeName);
tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues);
@@ -63,7 +63,7 @@ public class KStreamAggregate<K, V, T> implements KStreamAggProcessorSupplier<K,
@Override
- public void process(K key, V value) {
+ public void process(final K key, final V value) {
// If the key or value is null we don't need to proceed
if (key == null || value == null) {
return;
@@ -108,12 +108,12 @@ public class KStreamAggregate<K, V, T> implements KStreamAggProcessorSupplier<K,
@SuppressWarnings("unchecked")
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
store = (KeyValueStore<K, T>) context.getStateStore(storeName);
}
@Override
- public T get(K key) {
+ public T get(final K key) {
return store.get(key);
}
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
index 4f26767..6af4972 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
@@ -34,7 +34,7 @@ class KStreamKStreamJoin<K, R, V1, V2> implements ProcessorSupplier<K, V1> {
private final ValueJoiner<? super V1, ? super V2, ? extends R> joiner;
private final boolean outer;
- KStreamKStreamJoin(String otherWindowName, long joinBeforeMs, long joinAfterMs, ValueJoiner<? super V1, ? super V2, ? extends R> joiner, boolean outer) {
+ KStreamKStreamJoin(final String otherWindowName, final long joinBeforeMs, final long joinAfterMs, final ValueJoiner<? super V1, ? super V2, ? extends R> joiner, final boolean outer) {
this.otherWindowName = otherWindowName;
this.joinBeforeMs = joinBeforeMs;
this.joinAfterMs = joinAfterMs;
@@ -53,7 +53,7 @@ class KStreamKStreamJoin<K, R, V1, V2> implements ProcessorSupplier<K, V1> {
@SuppressWarnings("unchecked")
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
otherWindow = (WindowStore<K, V2>) context.getStateStore(otherWindowName);
@@ -77,7 +77,7 @@ class KStreamKStreamJoin<K, R, V1, V2> implements ProcessorSupplier<K, V1> {
final long timeFrom = Math.max(0L, context().timestamp() - joinBeforeMs);
final long timeTo = Math.max(0L, context().timestamp() + joinAfterMs);
- try (WindowStoreIterator<V2> iter = otherWindow.fetch(key, timeFrom, timeTo)) {
+ try (final WindowStoreIterator<V2> iter = otherWindow.fetch(key, timeFrom, timeTo)) {
while (iter.hasNext()) {
needOuterJoin = false;
context().forward(key, joiner.apply(value, iter.next().value));
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java
index 0fd8f75..bd0598a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java
@@ -29,7 +29,7 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
private boolean sendOldValues = false;
- public KStreamReduce(String storeName, Reducer<V> reducer) {
+ KStreamReduce(final String storeName, final Reducer<V> reducer) {
this.storeName = storeName;
this.reducer = reducer;
}
@@ -51,7 +51,7 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
@SuppressWarnings("unchecked")
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
store = (KeyValueStore<K, V>) context.getStateStore(storeName);
@@ -60,13 +60,13 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
@Override
- public void process(K key, V value) {
+ public void process(final K key, final V value) {
// If the key or value is null we don't need to proceed
if (key == null || value == null) {
return;
}
- V oldAgg = store.get(key);
+ final V oldAgg = store.get(key);
V newAgg = oldAgg;
// try to add the new value
@@ -75,7 +75,7 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
} else {
newAgg = reducer.apply(newAgg, value);
}
-
+
// update the store with the new value
store.put(key, newAgg);
tupleForwarder.maybeForward(key, newAgg, oldAgg);
@@ -104,12 +104,12 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
@SuppressWarnings("unchecked")
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
store = (KeyValueStore<K, V>) context.getStateStore(storeName);
}
@Override
- public V get(K key) {
+ public V get(final K key) {
return store.get(key);
}
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
index 166d8a8..d96468a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
@@ -71,9 +71,9 @@ class KStreamSessionWindowAggregate<K, V, T> implements KStreamAggProcessorSuppl
@SuppressWarnings("unchecked")
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
- store = (SessionStore<K, T>) context.getStateStore(storeName);
+ store = (SessionStore<K, T>) context.getStateStore(storeName);
tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues);
}
@@ -91,8 +91,13 @@ class KStreamSessionWindowAggregate<K, V, T> implements KStreamAggProcessorSuppl
SessionWindow mergedWindow = newSessionWindow;
T agg = initializer.apply();
- try (final KeyValueIterator<Windowed<K>, T> iterator = store.findSessions(key, timestamp - windows.inactivityGap(),
- timestamp + windows.inactivityGap())) {
+ try (
+ final KeyValueIterator<Windowed<K>, T> iterator = store.findSessions(
+ key,
+ timestamp - windows.inactivityGap(),
+ timestamp + windows.inactivityGap()
+ )
+ ) {
while (iterator.hasNext()) {
final KeyValue<Windowed<K>, T> next = iterator.next();
merged.add(next);
@@ -132,7 +137,7 @@ class KStreamSessionWindowAggregate<K, V, T> implements KStreamAggProcessorSuppl
@Override
public String[] storeNames() {
- return new String[] {storeName};
+ return new String[]{storeName};
}
};
}
@@ -148,7 +153,7 @@ class KStreamSessionWindowAggregate<K, V, T> implements KStreamAggProcessorSuppl
@Override
public T get(final Windowed<K> key) {
- try (KeyValueIterator<Windowed<K>, T> iter = store.findSessions(key.key(), key.window().end(), key.window().end())) {
+ try (final KeyValueIterator<Windowed<K>, T> iter = store.findSessions(key.key(), key.window().end(), key.window().end())) {
if (!iter.hasNext()) {
return null;
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
index 6953f7c..6a94543 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
@@ -18,9 +18,9 @@ package org.apache.kafka.streams.kstream.internals;
import org.apache.kafka.streams.kstream.Aggregator;
import org.apache.kafka.streams.kstream.Initializer;
-import org.apache.kafka.streams.kstream.Windows;
import org.apache.kafka.streams.kstream.Window;
import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.Windows;
import org.apache.kafka.streams.processor.AbstractProcessor;
import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.ProcessorContext;
@@ -75,8 +75,9 @@ public class KStreamWindowAggregate<K, V, T, W extends Window> implements KStrea
public void process(final K key, final V value) {
// if the key is null, we do not need proceed aggregating the record
// the record with the table
- if (key == null)
+ if (key == null) {
return;
+ }
// first get the matching windows
final long timestamp = context().timestamp();
@@ -128,8 +129,8 @@ public class KStreamWindowAggregate<K, V, T, W extends Window> implements KStrea
@SuppressWarnings("unchecked")
@Override
public T get(final Windowed<K> windowedKey) {
- K key = windowedKey.key();
- W window = (W) windowedKey.window();
+ final K key = windowedKey.key();
+ final W window = (W) windowedKey.window();
return windowStore.fetch(key, window.start());
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java
index 1d8b32b..10c3d8a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java
@@ -70,8 +70,9 @@ public class KStreamWindowReduce<K, V, W extends Window> implements KStreamAggPr
public void process(final K key, final V value) {
// if the key is null, we do not need proceed aggregating
// the record with the table
- if (key == null)
+ if (key == null) {
return;
+ }
// first get the matching windows
final long timestamp = context().timestamp();
@@ -81,7 +82,7 @@ public class KStreamWindowReduce<K, V, W extends Window> implements KStreamAggPr
for (final Map.Entry<Long, W> entry : matchedWindows.entrySet()) {
final V oldAgg = windowStore.fetch(key, entry.getKey());
- V newAgg;
+ final V newAgg;
if (oldAgg == null) {
newAgg = value;
} else {
@@ -124,8 +125,8 @@ public class KStreamWindowReduce<K, V, W extends Window> implements KStreamAggPr
@SuppressWarnings("unchecked")
@Override
public V get(final Windowed<K> windowedKey) {
- K key = windowedKey.key();
- W window = (W) windowedKey.window();
+ final K key = windowedKey.key();
+ final W window = (W) windowedKey.window();
return windowStore.fetch(key, window.start());
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java
index dd34735..b60f9ab 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java
@@ -69,15 +69,17 @@ public class KTableAggregate<K, V, T> implements KTableProcessorSupplier<K, V, T
* @throws StreamsException if key is null
*/
@Override
- public void process(K key, Change<V> value) {
+ public void process(final K key, final Change<V> value) {
// the keys should never be null
- if (key == null)
+ if (key == null) {
throw new StreamsException("Record key for KTable aggregate operator with state " + storeName + " should not be null.");
+ }
T oldAgg = store.get(key);
- if (oldAgg == null)
+ if (oldAgg == null) {
oldAgg = initializer.apply();
+ }
T newAgg = oldAgg;
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java
index e170175..78f3517 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java
@@ -31,7 +31,9 @@ class KTableKTableInnerJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
}
};
- KTableKTableInnerJoin(KTableImpl<K, ?, V1> table1, KTableImpl<K, ?, V2> table2, ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
+ KTableKTableInnerJoin(final KTableImpl<K, ?, V1> table1,
+ final KTableImpl<K, ?, V2> table2,
+ final ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
super(table1, table2, joiner);
}
@@ -47,7 +49,8 @@ class KTableKTableInnerJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private class KTableKTableInnerJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier<K, R, V1, V2> {
- KTableKTableInnerJoinValueGetterSupplier(KTableValueGetterSupplier<K, V1> valueGetterSupplier1, KTableValueGetterSupplier<K, V2> valueGetterSupplier2) {
+ KTableKTableInnerJoinValueGetterSupplier(final KTableValueGetterSupplier<K, V1> valueGetterSupplier1,
+ final KTableValueGetterSupplier<K, V2> valueGetterSupplier2) {
super(valueGetterSupplier1, valueGetterSupplier2);
}
@@ -60,12 +63,12 @@ class KTableKTableInnerJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V2> valueGetter;
- KTableKTableJoinProcessor(KTableValueGetter<K, V2> valueGetter) {
+ KTableKTableJoinProcessor(final KTableValueGetter<K, V2> valueGetter) {
this.valueGetter = valueGetter;
}
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
valueGetter.init(context);
}
@@ -119,7 +122,7 @@ class KTableKTableInnerJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
final V1 value1 = valueGetter1.get(key);
if (value1 != null) {
- V2 value2 = valueGetter2.get(keyValueMapper.apply(key, value1));
+ final V2 value2 = valueGetter2.get(keyValueMapper.apply(key, value1));
if (value2 != null) {
return joiner.apply(value1, value2);
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java
index bb3e652..6b170da 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java
@@ -23,7 +23,9 @@ import org.apache.kafka.streams.processor.ProcessorContext;
class KTableKTableLeftJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R, V1, V2> {
- KTableKTableLeftJoin(KTableImpl<K, ?, V1> table1, KTableImpl<K, ?, V2> table2, ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
+ KTableKTableLeftJoin(final KTableImpl<K, ?, V1> table1,
+ final KTableImpl<K, ?, V2> table2,
+ final ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
super(table1, table2, joiner);
}
@@ -39,7 +41,8 @@ class KTableKTableLeftJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private class KTableKTableLeftJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier<K, R, V1, V2> {
- KTableKTableLeftJoinValueGetterSupplier(KTableValueGetterSupplier<K, V1> valueGetterSupplier1, KTableValueGetterSupplier<K, V2> valueGetterSupplier2) {
+ KTableKTableLeftJoinValueGetterSupplier(final KTableValueGetterSupplier<K, V1> valueGetterSupplier1,
+ final KTableValueGetterSupplier<K, V2> valueGetterSupplier2) {
super(valueGetterSupplier1, valueGetterSupplier2);
}
@@ -53,12 +56,12 @@ class KTableKTableLeftJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V2> valueGetter;
- KTableKTableLeftJoinProcessor(KTableValueGetter<K, V2> valueGetter) {
+ KTableKTableLeftJoinProcessor(final KTableValueGetter<K, V2> valueGetter) {
this.valueGetter = valueGetter;
}
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
valueGetter.init(context);
}
@@ -82,8 +85,9 @@ class KTableKTableLeftJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
newValue = joiner.apply(change.newValue, value2);
}
- if (sendOldValues && change.oldValue != null)
+ if (sendOldValues && change.oldValue != null) {
oldValue = joiner.apply(change.oldValue, value2);
+ }
context().forward(key, new Change<>(newValue, oldValue));
}
@@ -94,23 +98,24 @@ class KTableKTableLeftJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V1> valueGetter1;
private final KTableValueGetter<K, V2> valueGetter2;
- KTableKTableLeftJoinValueGetter(KTableValueGetter<K, V1> valueGetter1, KTableValueGetter<K, V2> valueGetter2) {
+ KTableKTableLeftJoinValueGetter(final KTableValueGetter<K, V1> valueGetter1,
+ final KTableValueGetter<K, V2> valueGetter2) {
this.valueGetter1 = valueGetter1;
this.valueGetter2 = valueGetter2;
}
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
valueGetter1.init(context);
valueGetter2.init(context);
}
@Override
- public R get(K key) {
- V1 value1 = valueGetter1.get(key);
+ public R get(final K key) {
+ final V1 value1 = valueGetter1.get(key);
if (value1 != null) {
- V2 value2 = valueGetter2.get(key);
+ final V2 value2 = valueGetter2.get(key);
return joiner.apply(value1, value2);
} else {
return null;
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java
index e7c170e..97f00e3 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java
@@ -23,7 +23,9 @@ import org.apache.kafka.streams.processor.ProcessorContext;
class KTableKTableOuterJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R, V1, V2> {
- KTableKTableOuterJoin(KTableImpl<K, ?, V1> table1, KTableImpl<K, ?, V2> table2, ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
+ KTableKTableOuterJoin(final KTableImpl<K, ?, V1> table1,
+ final KTableImpl<K, ?, V2> table2,
+ final ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
super(table1, table2, joiner);
}
@@ -39,7 +41,8 @@ class KTableKTableOuterJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private class KTableKTableOuterJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier<K, R, V1, V2> {
- KTableKTableOuterJoinValueGetterSupplier(KTableValueGetterSupplier<K, V1> valueGetterSupplier1, KTableValueGetterSupplier<K, V2> valueGetterSupplier2) {
+ KTableKTableOuterJoinValueGetterSupplier(final KTableValueGetterSupplier<K, V1> valueGetterSupplier1,
+ final KTableValueGetterSupplier<K, V2> valueGetterSupplier2) {
super(valueGetterSupplier1, valueGetterSupplier2);
}
@@ -52,12 +55,12 @@ class KTableKTableOuterJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V2> valueGetter;
- KTableKTableOuterJoinProcessor(KTableValueGetter<K, V2> valueGetter) {
+ KTableKTableOuterJoinProcessor(final KTableValueGetter<K, V2> valueGetter) {
this.valueGetter = valueGetter;
}
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
valueGetter.init(context);
}
@@ -94,25 +97,27 @@ class KTableKTableOuterJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V1> valueGetter1;
private final KTableValueGetter<K, V2> valueGetter2;
- KTableKTableOuterJoinValueGetter(KTableValueGetter<K, V1> valueGetter1, KTableValueGetter<K, V2> valueGetter2) {
+ KTableKTableOuterJoinValueGetter(final KTableValueGetter<K, V1> valueGetter1,
+ final KTableValueGetter<K, V2> valueGetter2) {
this.valueGetter1 = valueGetter1;
this.valueGetter2 = valueGetter2;
}
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
valueGetter1.init(context);
valueGetter2.init(context);
}
@Override
- public R get(K key) {
+ public R get(final K key) {
R newValue = null;
- V1 value1 = valueGetter1.get(key);
- V2 value2 = valueGetter2.get(key);
+ final V1 value1 = valueGetter1.get(key);
+ final V2 value2 = valueGetter2.get(key);
- if (value1 != null || value2 != null)
+ if (value1 != null || value2 != null) {
newValue = joiner.apply(value1, value2);
+ }
return newValue;
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java
index c540cf9..61798cd 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java
@@ -24,7 +24,9 @@ import org.apache.kafka.streams.processor.ProcessorContext;
class KTableKTableRightJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R, V1, V2> {
- KTableKTableRightJoin(KTableImpl<K, ?, V1> table1, KTableImpl<K, ?, V2> table2, ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
+ KTableKTableRightJoin(final KTableImpl<K, ?, V1> table1,
+ final KTableImpl<K, ?, V2> table2,
+ final ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
super(table1, table2, joiner);
}
@@ -40,7 +42,8 @@ class KTableKTableRightJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private class KTableKTableRightJoinValueGetterSupplier extends KTableKTableAbstractJoinValueGetterSupplier<K, R, V1, V2> {
- KTableKTableRightJoinValueGetterSupplier(KTableValueGetterSupplier<K, V1> valueGetterSupplier1, KTableValueGetterSupplier<K, V2> valueGetterSupplier2) {
+ KTableKTableRightJoinValueGetterSupplier(final KTableValueGetterSupplier<K, V1> valueGetterSupplier1,
+ final KTableValueGetterSupplier<K, V2> valueGetterSupplier2) {
super(valueGetterSupplier1, valueGetterSupplier2);
}
@@ -53,12 +56,12 @@ class KTableKTableRightJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V2> valueGetter;
- KTableKTableRightJoinProcessor(KTableValueGetter<K, V2> valueGetter) {
+ KTableKTableRightJoinProcessor(final KTableValueGetter<K, V2> valueGetter) {
this.valueGetter = valueGetter;
}
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
valueGetter.init(context);
}
@@ -94,23 +97,24 @@ class KTableKTableRightJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V1> valueGetter1;
private final KTableValueGetter<K, V2> valueGetter2;
- KTableKTableRightJoinValueGetter(KTableValueGetter<K, V1> valueGetter1, KTableValueGetter<K, V2> valueGetter2) {
+ KTableKTableRightJoinValueGetter(final KTableValueGetter<K, V1> valueGetter1,
+ final KTableValueGetter<K, V2> valueGetter2) {
this.valueGetter1 = valueGetter1;
this.valueGetter2 = valueGetter2;
}
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
valueGetter1.init(context);
valueGetter2.init(context);
}
@Override
- public R get(K key) {
- V2 value2 = valueGetter2.get(key);
+ public R get(final K key) {
+ final V2 value2 = valueGetter2.get(key);
if (value2 != null) {
- V1 value1 = valueGetter1.get(key);
+ final V1 value1 = valueGetter1.get(key);
return joiner.apply(value1, value2);
} else {
return null;
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java
index 11b7d55..069b360 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java
@@ -57,19 +57,20 @@ public class KTableReduce<K, V> implements KTableProcessorSupplier<K, V, V> {
public void init(final ProcessorContext context) {
super.init(context);
store = (KeyValueStore<K, V>) context.getStateStore(storeName);
- tupleForwarder = new TupleForwarder<K, V>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues);
+ tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues);
}
/**
* @throws StreamsException if key is null
*/
@Override
- public void process(K key, Change<V> value) {
+ public void process(final K key, final Change<V> value) {
// the keys should never be null
- if (key == null)
+ if (key == null) {
throw new StreamsException("Record key for KTable reduce operator with state " + storeName + " should not be null.");
+ }
- V oldAgg = store.get(key);
+ final V oldAgg = store.get(key);
V newAgg = oldAgg;
// first try to add the new value
@@ -94,6 +95,6 @@ public class KTableReduce<K, V> implements KTableProcessorSupplier<K, V, V> {
@Override
public KTableValueGetterSupplier<K, V> view() {
- return new KTableMaterializedValueGetterSupplier<K, V>(storeName);
+ return new KTableMaterializedValueGetterSupplier<>(storeName);
}
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java
index 5aa3f2f..20d943e 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java
@@ -25,7 +25,7 @@ import org.apache.kafka.streams.processor.ProcessorContext;
/**
* KTable repartition map functions are not exposed to public APIs, but only used for keyed aggregations.
- *
+ * <p>
* Given the input, it can output at most two records (one mapped from old value and one mapped from new value).
*/
public class KTableRepartitionMap<K, V, K1, V1> implements KTableProcessorSupplier<K, V, KeyValue<K1, V1>> {
@@ -75,14 +75,15 @@ public class KTableRepartitionMap<K, V, K1, V1> implements KTableProcessorSuppli
* @throws StreamsException if key is null
*/
@Override
- public void process(K key, Change<V> change) {
+ public void process(final K key, final Change<V> change) {
// the original key should never be null
- if (key == null)
+ if (key == null) {
throw new StreamsException("Record key for the grouping KTable should not be null.");
+ }
// if the value is null, we do not need to forward its selected key-value further
- KeyValue<? extends K1, ? extends V1> newPair = change.newValue == null ? null : mapper.apply(key, change.newValue);
- KeyValue<? extends K1, ? extends V1> oldPair = change.oldValue == null ? null : mapper.apply(key, change.oldValue);
+ final KeyValue<? extends K1, ? extends V1> newPair = change.newValue == null ? null : mapper.apply(key, change.newValue);
+ final KeyValue<? extends K1, ? extends V1> oldPair = change.oldValue == null ? null : mapper.apply(key, change.oldValue);
// if the selected repartition key or value is null, skip
// forward oldPair first, to be consistent with reduce and aggregate
@@ -93,7 +94,7 @@ public class KTableRepartitionMap<K, V, K1, V1> implements KTableProcessorSuppli
if (newPair != null && newPair.key != null && newPair.value != null) {
context().forward(newPair.key, new Change<>(newPair.value, null));
}
-
+
}
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java
index de8bf2e..273e255 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java
@@ -28,7 +28,7 @@ public class KTableSource<K, V> implements ProcessorSupplier<K, V> {
private boolean sendOldValues = false;
- public KTableSource(String storeName) {
+ public KTableSource(final String storeName) {
this.storeName = storeName;
}
@@ -48,18 +48,19 @@ public class KTableSource<K, V> implements ProcessorSupplier<K, V> {
@SuppressWarnings("unchecked")
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
store = (KeyValueStore<K, V>) context.getStateStore(storeName);
tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues);
}
@Override
- public void process(K key, V value) {
+ public void process(final K key, final V value) {
// if the key is null, then ignore the record
- if (key == null)
+ if (key == null) {
return;
- V oldValue = store.get(key);
+ }
+ final V oldValue = store.get(key);
store.put(key, value);
tupleForwarder.maybeForward(key, value, oldValue);
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
index a8f7e65..bbdb349 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
@@ -76,7 +76,7 @@ public abstract class AbstractTask implements Task {
this.eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
this.stateDirectory = stateDirectory;
- this.logPrefix = String.format("%s [%s] ", isStandby ? "standby-task" : "task", id());
+ this.logPrefix = String.format("%s [%s] ", isStandby ? "standby-task" : "task", id);
this.logContext = new LogContext(logPrefix);
this.log = logContext.logger(getClass());
@@ -129,6 +129,7 @@ public abstract class AbstractTask implements Task {
/**
* Produces a string representation containing useful information about a StreamTask.
* This is useful in debugging scenarios.
+ *
* @return A string representation of the StreamTask instance.
*/
@Override
@@ -139,6 +140,7 @@ public abstract class AbstractTask implements Task {
/**
* Produces a string representation containing useful information about a StreamTask starting with the given indent.
* This is useful in debugging scenarios.
+ *
* @return A string representation of the StreamTask instance.
*/
public String toString(final String indent) {
@@ -197,9 +199,9 @@ public abstract class AbstractTask implements Task {
}
/**
- * @throws StreamsException if the store's change log does not contain the partition
- *
* Package-private for testing only
+ *
+ * @throws StreamsException If the store's change log does not contain the partition
*/
void registerStateStores() {
if (topology.stateStores().isEmpty()) {
@@ -208,12 +210,12 @@ public abstract class AbstractTask implements Task {
try {
if (!stateDirectory.lock(id)) {
- throw new LockException(String.format("%sFailed to lock the state directory for task %s",
- logPrefix, id));
+ throw new LockException(String.format("%sFailed to lock the state directory for task %s", logPrefix, id));
}
- } catch (IOException e) {
- throw new StreamsException(String.format("%sFatal error while trying to lock the state directory for task %s",
- logPrefix, id));
+ } catch (final IOException e) {
+ throw new StreamsException(
+ String.format("%sFatal error while trying to lock the state directory for task %s",
+ logPrefix, id));
}
log.trace("Initializing state stores");
@@ -232,8 +234,8 @@ public abstract class AbstractTask implements Task {
}
/**
- * @throws ProcessorStateException if there is an error while closing the state manager
* @param writeCheckpoint boolean indicating if a checkpoint file should be written
+ * @throws ProcessorStateException if there is an error while closing the state manager
*/
// visible for testing
void closeStateManager(final boolean writeCheckpoint) throws ProcessorStateException {
@@ -246,7 +248,7 @@ public abstract class AbstractTask implements Task {
} finally {
try {
stateDirectory.unlock(id);
- } catch (IOException e) {
+ } catch (final IOException e) {
if (exception == null) {
exception = new ProcessorStateException(String.format("%sFailed to release state dir lock", logPrefix), e);
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java
index c18f3c7..c455275 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java
@@ -54,7 +54,7 @@ public class GlobalStateUpdateTask implements GlobalStateMaintainer {
/**
* @throws IllegalStateException If store gets registered after initialized is already finished
- * @throws StreamsException if the store's change log does not contain the partition
+ * @throws StreamsException If the store's change log does not contain the partition
*/
@Override
public Map<TopicPartition, Long> initialize() {
@@ -100,7 +100,7 @@ public class GlobalStateUpdateTask implements GlobalStateMaintainer {
}
private void initTopology() {
- for (ProcessorNode node : this.topology.processors()) {
+ for (final ProcessorNode node : this.topology.processors()) {
processorContext.setCurrentNode(node);
try {
node.init(this.processorContext);
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
index 1cc5c85..bb8bc1d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
@@ -86,11 +86,10 @@ public class GlobalStreamThread extends Thread {
* </pre>
*
* Note the following:
- * - Any state can go to PENDING_SHUTDOWN.
- * That is because streams can be closed at any time.
- * - State PENDING_SHUTDOWN may want to transit itself.
- * In this case we will forbid the transition but will not treat as an error.
- *
+ * <ul>
+ * <li>Any state can go to PENDING_SHUTDOWN. That is because streams can be closed at any time.</li>
+ * <li>State PENDING_SHUTDOWN may want to transit itself. In this case we will forbid the transition but will not treat as an error.</li>
+ * </ul>
*/
public enum State implements ThreadStateTransitionValidator {
CREATED(1, 2), RUNNING(2), PENDING_SHUTDOWN(3), DEAD;
@@ -107,7 +106,7 @@ public class GlobalStreamThread extends Thread {
@Override
public boolean isValidTransition(final ThreadStateTransitionValidator newState) {
- State tmpState = (State) newState;
+ final State tmpState = (State) newState;
return validTransitions.contains(tmpState.ordinal());
}
}
@@ -136,20 +135,21 @@ public class GlobalStreamThread extends Thread {
/**
* Sets the state
+ *
* @param newState New state
*/
- private boolean setState(final State newState) {
+ private void setState(final State newState) {
final State oldState = state;
synchronized (stateLock) {
if (state == State.PENDING_SHUTDOWN && newState == State.PENDING_SHUTDOWN) {
// when the state is already in PENDING_SHUTDOWN, its transition to itself
// will be refused but we do not throw exception here
- return false;
+ return;
} else if (state == State.DEAD) {
// when the state is already in NOT_RUNNING, all its transitions
// will be refused but we do not throw exception here
- return false;
+ return;
} else if (!state.isValidTransition(newState)) {
log.error("Unexpected state transition from {} to {}", oldState, newState);
throw new StreamsException(logPrefix + "Unexpected state transition from " + oldState + " to " + newState);
@@ -163,8 +163,6 @@ public class GlobalStreamThread extends Thread {
if (stateListener != null) {
stateListener.onChange(this, state, oldState);
}
-
- return true;
}
public boolean stillRunning() {
@@ -194,7 +192,6 @@ public class GlobalStreamThread extends Thread {
this.log = logContext.logger(getClass());
this.cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics);
this.stateRestoreListener = stateRestoreListener;
-
}
static class StateConsumer {
@@ -223,12 +220,12 @@ public class GlobalStreamThread extends Thread {
/**
* @throws IllegalStateException If store gets registered after initialized is already finished
- * @throws StreamsException if the store's change log does not contain the partition
+ * @throws StreamsException if the store's change log does not contain the partition
*/
void initialize() {
final Map<TopicPartition, Long> partitionOffsets = stateMaintainer.initialize();
globalConsumer.assign(partitionOffsets.keySet());
- for (Map.Entry<TopicPartition, Long> entry : partitionOffsets.entrySet()) {
+ for (final Map.Entry<TopicPartition, Long> entry : partitionOffsets.entrySet()) {
globalConsumer.seek(entry.getKey(), entry.getValue());
}
lastFlush = time.milliseconds();
@@ -237,7 +234,7 @@ public class GlobalStreamThread extends Thread {
void pollAndUpdate() {
try {
final ConsumerRecords<byte[], byte[]> received = globalConsumer.poll(pollMs);
- for (ConsumerRecord<byte[], byte[]> record : received) {
+ for (final ConsumerRecord<byte[], byte[]> record : received) {
stateMaintainer.update(record);
}
final long now = time.milliseconds();
@@ -309,12 +306,13 @@ public class GlobalStreamThread extends Thread {
private StateConsumer initialize() {
try {
- final GlobalStateManager stateMgr = new GlobalStateManagerImpl(logContext,
- topology,
- globalConsumer,
- stateDirectory,
- stateRestoreListener,
- config);
+ final GlobalStateManager stateMgr = new GlobalStateManagerImpl(
+ logContext,
+ topology,
+ globalConsumer,
+ stateDirectory,
+ stateRestoreListener,
+ config);
final GlobalProcessorContextImpl globalProcessorContext = new GlobalProcessorContextImpl(
config,
@@ -323,17 +321,19 @@ public class GlobalStreamThread extends Thread {
cache);
stateMgr.setGlobalProcessorContext(globalProcessorContext);
- final StateConsumer stateConsumer
- = new StateConsumer(this.logContext,
- globalConsumer,
- new GlobalStateUpdateTask(topology,
- globalProcessorContext,
- stateMgr,
- config.defaultDeserializationExceptionHandler(),
- logContext),
- time,
- config.getLong(StreamsConfig.POLL_MS_CONFIG),
- config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG));
+ final StateConsumer stateConsumer = new StateConsumer(
+ logContext,
+ globalConsumer,
+ new GlobalStateUpdateTask(
+ topology,
+ globalProcessorContext,
+ stateMgr,
+ config.defaultDeserializationExceptionHandler(),
+ logContext
+ ),
+ time,
+ config.getLong(StreamsConfig.POLL_MS_CONFIG),
+ config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG));
stateConsumer.initialize();
return stateConsumer;
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java
index 94e8640..0411a37 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java
@@ -39,19 +39,19 @@ public class ProcessorNode<K, V> {
private final String name;
private final Processor<K, V> processor;
- NodeMetrics nodeMetrics;
- private Time time;
+ private NodeMetrics nodeMetrics;
+ private final Time time;
private K key;
private V value;
- private Runnable processDelegate = new Runnable() {
+ private final Runnable processDelegate = new Runnable() {
@Override
public void run() {
processor.process(key, value);
}
};
private ProcessorContext context;
- private Runnable initDelegate = new Runnable() {
+ private final Runnable initDelegate = new Runnable() {
@Override
public void run() {
if (processor != null) {
@@ -59,7 +59,7 @@ public class ProcessorNode<K, V> {
}
}
};
- private Runnable closeDelegate = new Runnable() {
+ private final Runnable closeDelegate = new Runnable() {
@Override
public void run() {
if (processor != null) {
@@ -70,12 +70,12 @@ public class ProcessorNode<K, V> {
public final Set<String> stateStores;
- public ProcessorNode(String name) {
+ public ProcessorNode(final String name) {
this(name, null, null);
}
- public ProcessorNode(String name, Processor<K, V> processor, Set<String> stateStores) {
+ public ProcessorNode(final String name, final Processor<K, V> processor, final Set<String> stateStores) {
this.name = name;
this.processor = processor;
this.children = new ArrayList<>();
@@ -97,21 +97,21 @@ public class ProcessorNode<K, V> {
return children;
}
- public final ProcessorNode getChild(final String childName) {
+ final ProcessorNode getChild(final String childName) {
return childByName.get(childName);
}
- public void addChild(ProcessorNode<?, ?> child) {
+ public void addChild(final ProcessorNode<?, ?> child) {
children.add(child);
childByName.put(child.name, child);
}
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
this.context = context;
try {
nodeMetrics = new NodeMetrics(context.metrics(), name, context);
nodeMetrics.metrics.measureLatencyNs(time, initDelegate, nodeMetrics.nodeCreationSensor);
- } catch (Exception e) {
+ } catch (final Exception e) {
throw new StreamsException(String.format("failed to initialize processor %s", name), e);
}
}
@@ -120,7 +120,7 @@ public class ProcessorNode<K, V> {
try {
nodeMetrics.metrics.measureLatencyNs(time, closeDelegate, nodeMetrics.nodeDestructionSensor);
nodeMetrics.removeAllSensors();
- } catch (Exception e) {
+ } catch (final Exception e) {
throw new StreamsException(String.format("failed to close processor %s", name), e);
}
}
@@ -134,7 +134,7 @@ public class ProcessorNode<K, V> {
}
public void punctuate(final long timestamp, final Punctuator punctuator) {
- Runnable punctuateDelegate = new Runnable() {
+ final Runnable punctuateDelegate = new Runnable() {
@Override
public void run() {
punctuator.punctuate(timestamp);
@@ -154,11 +154,11 @@ public class ProcessorNode<K, V> {
/**
* @return a string representation of this node starting with the given indent, useful for debugging.
*/
- public String toString(String indent) {
+ public String toString(final String indent) {
final StringBuilder sb = new StringBuilder(indent + name + ":\n");
if (stateStores != null && !stateStores.isEmpty()) {
sb.append(indent).append("\tstates:\t\t[");
- for (String store : stateStores) {
+ for (final String store : stateStores) {
sb.append(store);
sb.append(", ");
}
@@ -168,39 +168,52 @@ public class ProcessorNode<K, V> {
return sb.toString();
}
- protected static final class NodeMetrics {
- final StreamsMetricsImpl metrics;
+ Sensor sourceNodeForwardSensor() {
+ return nodeMetrics.sourceNodeForwardSensor;
+ }
+
+ Sensor sourceNodeSkippedDueToDeserializationErrorSensor() {
+ return nodeMetrics.sourceNodeSkippedDueToDeserializationError;
+ }
- final Sensor nodeProcessTimeSensor;
- final Sensor nodePunctuateTimeSensor;
- final Sensor sourceNodeForwardSensor;
- final Sensor sourceNodeSkippedDueToDeserializationError;
- final Sensor nodeCreationSensor;
- final Sensor nodeDestructionSensor;
+ private static final class NodeMetrics {
+ private final StreamsMetricsImpl metrics;
+ private final Sensor nodeProcessTimeSensor;
+ private final Sensor nodePunctuateTimeSensor;
+ private final Sensor sourceNodeForwardSensor;
+ private final Sensor sourceNodeSkippedDueToDeserializationError;
+ private final Sensor nodeCreationSensor;
+ private final Sensor nodeDestructionSensor;
- public NodeMetrics(final StreamsMetrics metrics, final String name, final ProcessorContext context) {
+ private NodeMetrics(final StreamsMetrics metrics, final String name, final ProcessorContext context) {
final String scope = "processor-node";
final String tagKey = "task-id";
final String tagValue = context.taskId().toString();
this.metrics = (StreamsMetricsImpl) metrics;
// these are all latency metrics
- this.nodeProcessTimeSensor = metrics.addLatencyAndThroughputSensor(scope, name, "process",
- Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
- this.nodePunctuateTimeSensor = metrics.addLatencyAndThroughputSensor(scope, name, "punctuate",
- Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
- this.nodeCreationSensor = metrics.addLatencyAndThroughputSensor(scope, name, "create",
- Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
- this.nodeDestructionSensor = metrics.addLatencyAndThroughputSensor(scope, name, "destroy",
- Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
- this.sourceNodeForwardSensor = metrics.addThroughputSensor(scope, name, "forward",
- Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
- this.sourceNodeSkippedDueToDeserializationError = metrics.addThroughputSensor(scope, name, "skippedDueToDeserializationError",
- Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
+ this.nodeProcessTimeSensor = metrics.addLatencyAndThroughputSensor(
+ scope, name, "process", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
+ );
+ this.nodePunctuateTimeSensor = metrics.addLatencyAndThroughputSensor(
+ scope, name, "punctuate", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
+ );
+ this.nodeCreationSensor = metrics.addLatencyAndThroughputSensor(
+ scope, name, "create", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
+ );
+ this.nodeDestructionSensor = metrics.addLatencyAndThroughputSensor(
+ scope, name, "destroy", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
+ );
+ this.sourceNodeForwardSensor = metrics.addThroughputSensor(
+ scope, name, "forward", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
+ );
+ this.sourceNodeSkippedDueToDeserializationError = metrics.addThroughputSensor(
+ scope, name, "skippedDueToDeserializationError", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
+ );
}
- public void removeAllSensors() {
+ private void removeAllSensors() {
metrics.removeSensor(nodeProcessTimeSensor);
metrics.removeSensor(nodePunctuateTimeSensor);
metrics.removeSensor(sourceNodeForwardSensor);
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
index f568048..0d13758 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
@@ -23,14 +23,14 @@ import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.errors.AuthenticationException;
+import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.RetriableException;
-import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.SecurityDisabledException;
+import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.serialization.Serializer;
@@ -95,11 +95,11 @@ public class RecordCollectorImpl implements RecordCollector {
}
private boolean productionExceptionIsFatal(final Exception exception) {
- boolean securityException = exception instanceof AuthenticationException ||
+ final boolean securityException = exception instanceof AuthenticationException ||
exception instanceof AuthorizationException ||
exception instanceof SecurityDisabledException;
- boolean communicationException = exception instanceof InvalidTopicException ||
+ final boolean communicationException = exception instanceof InvalidTopicException ||
exception instanceof UnknownServerException ||
exception instanceof SerializationException ||
exception instanceof OffsetMetadataTooLarge ||
@@ -123,31 +123,32 @@ public class RecordCollectorImpl implements RecordCollector {
}
log.error(errorLogMessage, key, value, timestamp, topic, exception.toString());
sendException = new StreamsException(
- String.format(errorMessage,
- logPrefix,
- "an error caught",
- key,
- value,
- timestamp,
- topic,
- exception.toString()),
+ String.format(
+ errorMessage,
+ logPrefix,
+ "an error caught",
+ key,
+ value,
+ timestamp,
+ topic,
+ exception.toString()
+ ),
exception);
}
@Override
- public <K, V> void send(final String topic,
- final K key,
- final V value,
- final Integer partition,
- final Long timestamp,
- final Serializer<K> keySerializer,
- final Serializer<V> valueSerializer) {
+ public <K, V> void send(final String topic,
+ final K key,
+ final V value,
+ final Integer partition,
+ final Long timestamp,
+ final Serializer<K> keySerializer,
+ final Serializer<V> valueSerializer) {
checkForException();
final byte[] keyBytes = keySerializer.serialize(topic, key);
final byte[] valBytes = valueSerializer.serialize(topic, value);
- final ProducerRecord<byte[], byte[]> serializedRecord =
- new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes);
+ final ProducerRecord<byte[], byte[]> serializedRecord = new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes);
try {
producer.send(serializedRecord, new Callback() {
@@ -165,14 +166,17 @@ public class RecordCollectorImpl implements RecordCollector {
if (exception instanceof ProducerFencedException) {
log.warn(LOG_MESSAGE, key, value, timestamp, topic, exception.toString());
sendException = new ProducerFencedException(
- String.format(EXCEPTION_MESSAGE,
- logPrefix,
- "producer got fenced",
- key,
- value,
- timestamp,
- topic,
- exception.toString()));
+ String.format(
+ EXCEPTION_MESSAGE,
+ logPrefix,
+ "producer got fenced",
+ key,
+ value,
+ timestamp,
+ topic,
+ exception.toString()
+ )
+ );
} else {
if (productionExceptionIsFatal(exception)) {
recordSendError(key, value, timestamp, topic, exception);
@@ -194,19 +198,21 @@ public class RecordCollectorImpl implements RecordCollector {
throw new StreamsException(String.format("%sFailed to send record to topic %s due to timeout.", logPrefix, topic));
} catch (final Exception uncaughtException) {
throw new StreamsException(
- String.format(EXCEPTION_MESSAGE,
- logPrefix,
- "an error caught",
- key,
- value,
- timestamp,
- topic,
- uncaughtException.toString()),
+ String.format(
+ EXCEPTION_MESSAGE,
+ logPrefix,
+ "an error caught",
+ key,
+ value,
+ timestamp,
+ topic,
+ uncaughtException.toString()
+ ),
uncaughtException);
}
}
- private void checkForException() {
+ private void checkForException() {
if (sendException != null) {
throw sendException;
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java
index 1b5f764..47591d1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java
@@ -41,7 +41,8 @@ class RecordDeserializer {
/**
* @throws StreamsException if a deserialization error occurs and the deserialization callback returns
- * {@link DeserializationExceptionHandler.DeserializationHandlerResponse#FAIL FAIL} or throws an exception itself
+ * {@link DeserializationExceptionHandler.DeserializationHandlerResponse#FAIL FAIL}
+ * or throws an exception itself
*/
@SuppressWarnings("deprecation")
ConsumerRecord<Object, Object> deserialize(final ProcessorContext processorContext,
@@ -64,9 +65,10 @@ class RecordDeserializer {
try {
response = deserializationExceptionHandler.handle(processorContext, rawRecord, deserializationException);
} catch (final Exception fatalUserException) {
- log.error("Deserialization error callback failed after deserialization error for record {}",
- rawRecord,
- deserializationException);
+ log.error(
+ "Deserialization error callback failed after deserialization error for record {}",
+ rawRecord,
+ deserializationException);
throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException);
}
@@ -77,7 +79,7 @@ class RecordDeserializer {
DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.",
deserializationException);
} else {
- sourceNode.nodeMetrics.sourceNodeSkippedDueToDeserializationError.record();
+ sourceNode.sourceNodeSkippedDueToDeserializationErrorSensor().record();
}
}
return null;
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java
index 6d450dd..8df3998 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java
@@ -36,7 +36,11 @@ public class SourceNode<K, V> extends ProcessorNode<K, V> {
private ExtendedDeserializer<V> valDeserializer;
private final TimestampExtractor timestampExtractor;
- public SourceNode(String name, List<String> topics, TimestampExtractor timestampExtractor, Deserializer<K> keyDeserializer, Deserializer<V> valDeserializer) {
+ public SourceNode(final String name,
+ final List<String> topics,
+ final TimestampExtractor timestampExtractor,
+ final Deserializer<K> keyDeserializer,
+ final Deserializer<V> valDeserializer) {
super(name);
this.topics = topics;
this.timestampExtractor = timestampExtractor;
@@ -44,21 +48,24 @@ public class SourceNode<K, V> extends ProcessorNode<K, V> {
this.valDeserializer = ensureExtended(valDeserializer);
}
- public SourceNode(String name, List<String> topics, Deserializer<K> keyDeserializer, Deserializer<V> valDeserializer) {
+ public SourceNode(final String name,
+ final List<String> topics,
+ final Deserializer<K> keyDeserializer,
+ final Deserializer<V> valDeserializer) {
this(name, topics, null, keyDeserializer, valDeserializer);
}
- K deserializeKey(String topic, Headers headers, byte[] data) {
+ K deserializeKey(final String topic, final Headers headers, final byte[] data) {
return keyDeserializer.deserialize(topic, headers, data);
}
- V deserializeValue(String topic, Headers headers, byte[] data) {
+ V deserializeValue(final String topic, final Headers headers, final byte[] data) {
return valDeserializer.deserialize(topic, headers, data);
}
@SuppressWarnings("unchecked")
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
super.init(context);
this.context = context;
@@ -78,7 +85,7 @@ public class SourceNode<K, V> extends ProcessorNode<K, V> {
@Override
public void process(final K key, final V value) {
context.forward(key, value);
- nodeMetrics.sourceNodeForwardSensor.record();
+ sourceNodeForwardSensor().record();
}
/**
@@ -92,10 +99,10 @@ public class SourceNode<K, V> extends ProcessorNode<K, V> {
/**
* @return a string representation of this node starting with the given indent, useful for debugging.
*/
- public String toString(String indent) {
+ public String toString(final String indent) {
final StringBuilder sb = new StringBuilder(super.toString(indent));
sb.append(indent).append("\ttopics:\t\t[");
- for (String topic : topics) {
+ for (final String topic : topics) {
sb.append(topic);
sb.append(", ");
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java
index 861556c..a048407 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java
@@ -166,7 +166,7 @@ public class StandbyTask extends AbstractTask {
return stateMgr.updateStandbyStates(partition, records);
}
- public Map<TopicPartition, Long> checkpointedOffsets() {
+ Map<TopicPartition, Long> checkpointedOffsets() {
return checkpointedOffsets;
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
index 4b2e1b8..f7816d2 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
@@ -70,16 +70,15 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
private final Time time;
private final TaskMetrics metrics;
- protected class TaskMetrics {
+ protected static class TaskMetrics {
final StreamsMetricsImpl metrics;
final Sensor taskCommitTimeSensor;
- TaskMetrics(final StreamsMetrics metrics) {
- final String name = id().toString();
+ TaskMetrics(final TaskId id, final StreamsMetrics metrics) {
+ final String name = id.toString();
this.metrics = (StreamsMetricsImpl) metrics;
- taskCommitTimeSensor = metrics.addLatencyAndThroughputSensor("task", name, "commit",
- Sensor.RecordingLevel.DEBUG);
+ taskCommitTimeSensor = metrics.addLatencyAndThroughputSensor("task", name, "commit", Sensor.RecordingLevel.DEBUG);
}
void removeAllSensors() {
@@ -89,17 +88,18 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
/**
* Create {@link StreamTask} with its assigned partitions
- * @param id the ID of this task
- * @param partitions the collection of assigned {@link TopicPartition}
- * @param topology the instance of {@link ProcessorTopology}
- * @param consumer the instance of {@link Consumer}
- * @param changelogReader the instance of {@link ChangelogReader} used for restoring state
- * @param config the {@link StreamsConfig} specified by the user
- * @param metrics the {@link StreamsMetrics} created by the thread
- * @param stateDirectory the {@link StateDirectory} created by the thread
- * @param cache the {@link ThreadCache} created by the thread
- * @param time the system {@link Time} of the thread
- * @param producer the instance of {@link Producer} used to produce records
+ *
+ * @param id the ID of this task
+ * @param partitions the collection of assigned {@link TopicPartition}
+ * @param topology the instance of {@link ProcessorTopology}
+ * @param consumer the instance of {@link Consumer}
+ * @param changelogReader the instance of {@link ChangelogReader} used for restoring state
+ * @param config the {@link StreamsConfig} specified by the user
+ * @param metrics the {@link StreamsMetrics} created by the thread
+ * @param stateDirectory the {@link StateDirectory} created by the thread
+ * @param cache the {@link ThreadCache} created by the thread
+ * @param time the system {@link Time} of the thread
+ * @param producer the instance of {@link Producer} used to produce records
*/
public StreamTask(final TaskId id,
final Collection<TopicPartition> partitions,
@@ -116,7 +116,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
this.time = time;
this.producer = producer;
- this.metrics = new TaskMetrics(metrics);
+ this.metrics = new TaskMetrics(id, metrics);
final ProductionExceptionHandler productionExceptionHandler = config.defaultProductionExceptionHandler();
@@ -140,7 +140,14 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
for (final TopicPartition partition : partitions) {
final SourceNode source = topology.source(partition.topic());
final TimestampExtractor sourceTimestampExtractor = source.getTimestampExtractor() != null ? source.getTimestampExtractor() : defaultTimestampExtractor;
- final RecordQueue queue = new RecordQueue(partition, source, sourceTimestampExtractor, defaultDeserializationExceptionHandler, processorContext, logContext);
+ final RecordQueue queue = new RecordQueue(
+ partition,
+ source,
+ sourceTimestampExtractor,
+ defaultDeserializationExceptionHandler,
+ processorContext,
+ logContext
+ );
partitionQueues.put(partition, queue);
}
@@ -167,6 +174,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* <pre>
* - (re-)initialize the topology of the task
* </pre>
+ *
* @throws TaskMigratedException if the task producer got fenced (EOS only)
*/
@Override
@@ -272,7 +280,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
} catch (final ProducerFencedException fatal) {
throw new TaskMigratedException(this, fatal);
} catch (final KafkaException e) {
- throw new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e);
+ throw new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e);
} finally {
processorContext.setCurrentNode(null);
}
@@ -289,6 +297,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* - if(!eos) write checkpoint
* - commit offsets and start new transaction
* </pre>
+ *
* @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
*/
@@ -377,9 +386,10 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
Map<TopicPartition, Long> purgableOffsets() {
final Map<TopicPartition, Long> purgableConsumedOffsets = new HashMap<>();
for (final Map.Entry<TopicPartition, Long> entry : consumedOffsets.entrySet()) {
- TopicPartition tp = entry.getKey();
- if (topology.isRepartitionTopic(tp.topic()))
+ final TopicPartition tp = entry.getKey();
+ if (topology.isRepartitionTopic(tp.topic())) {
purgableConsumedOffsets.put(tp, entry.getValue() + 1);
+ }
}
return purgableConsumedOffsets;
@@ -406,6 +416,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* - if (!eos) write checkpoint
* - commit offsets
* </pre>
+ *
* @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
*/
@@ -423,6 +434,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* - if (!eos) write checkpoint
* - commit offsets
* </pre>
+ *
* @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
*/
@@ -523,8 +535,9 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* - if (clean) write checkpoint
* - if (eos) close producer
* </pre>
- * @param clean shut down cleanly (ie, incl. flush and commit) if {@code true} --
- * otherwise, just close open resources
+ *
+ * @param clean shut down cleanly (ie, incl. flush and commit) if {@code true} --
+ * otherwise, just close open resources
* @param isZombie {@code true} is this task is a zombie or not (this will repress {@link TaskMigratedException}
* @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
@@ -553,7 +566,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* and not added to the queue for processing
*
* @param partition the partition
- * @param records the records
+ * @param records the records
* @return the number of added records
*/
@SuppressWarnings("unchecked")
@@ -578,7 +591,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* Schedules a punctuation for the processor
*
* @param interval the interval in milliseconds
- * @param type the punctuation type
+ * @param type the punctuation type
* @throws IllegalStateException if the current node is not null
*/
public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator punctuator) {
@@ -598,8 +611,8 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* Schedules a punctuation for the processor
*
* @param startTime time of the first punctuation
- * @param interval the interval in milliseconds
- * @param type the punctuation type
+ * @param interval the interval in milliseconds
+ * @param type the punctuation type
* @throws IllegalStateException if the current node is not null
*/
Cancellable schedule(final long startTime, final long interval, final PunctuationType type, final Punctuator punctuator) {
@@ -633,6 +646,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* Possibly trigger registered stream-time punctuation functions if
* current partition group timestamp has reached the defined stamp
* Note, this is only called in the presence of new records
+ *
* @throws TaskMigratedException if the task producer got fenced (EOS only)
*/
public boolean maybePunctuateStreamTime() {
@@ -651,6 +665,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* Possibly trigger registered system-time punctuation functions if
* current system timestamp has reached the defined stamp
* Note, this is called irrespective of the presence of new records
+ *
* @throws TaskMigratedException if the task producer got fenced (EOS only)
*/
public boolean maybePunctuateSystemTime() {
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
index a7e3bcd..32cafb4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
@@ -73,7 +73,7 @@ public class StreamThread extends Thread {
* Stream thread states are the possible states that a stream thread can be in.
* A thread must only be in one state at a time
* The expected state transitions with the following defined states is:
- * <p>
+ *
* <pre>
* +-------------+
* +<--- | Created (0) |
@@ -108,17 +108,20 @@ public class StreamThread extends Thread {
* +-------------+
* </pre>
*
- * <p>
* Note the following:
- * - Any state can go to PENDING_SHUTDOWN.
- * That is because streams can be closed at any time.
- * - State PENDING_SHUTDOWN may want to transit to some other states other than DEAD, in the corner case when
- * the shutdown is triggered while the thread is still in the rebalance loop.
- * In this case we will forbid the transition but will not treat as an error.
- * - State PARTITIONS_REVOKED may want transit to itself indefinitely, in the corner case when
- * the coordinator repeatedly fails in-between revoking partitions and assigning new partitions.
- * In this case we will forbid the transition but will not treat as an error.
- *
+ * <ul>
+ * <li>Any state can go to PENDING_SHUTDOWN. That is because streams can be closed at any time.</li>
+ * <li>
+ * State PENDING_SHUTDOWN may want to transit to some other states other than DEAD,
+ * in the corner case when the shutdown is triggered while the thread is still in the rebalance loop.
+ * In this case we will forbid the transition but will not treat as an error.
+ * </li>
+ * <li>
+ * State PARTITIONS_REVOKED may want transit to itself indefinitely, in the corner case when
+ * the coordinator repeatedly fails in-between revoking partitions and assigning new partitions.
+ * In this case we will forbid the transition but will not treat as an error.
+ * </li>
+ * </ul>
*/
public enum State implements ThreadStateTransitionValidator {
CREATED(1, 4), RUNNING(2, 4), PARTITIONS_REVOKED(3, 4), PARTITIONS_ASSIGNED(1, 2, 4), PENDING_SHUTDOWN(5), DEAD;
@@ -135,7 +138,7 @@ public class StreamThread extends Thread {
@Override
public boolean isValidTransition(final ThreadStateTransitionValidator newState) {
- State tmpState = (State) newState;
+ final State tmpState = (State) newState;
return validTransitions.contains(tmpState.ordinal());
}
}
@@ -147,9 +150,10 @@ public class StreamThread extends Thread {
/**
* Called when state changes
- * @param thread thread changing state
- * @param newState current state
- * @param oldState previous state
+ *
+ * @param thread thread changing state
+ * @param newState current state
+ * @param oldState previous state
*/
void onChange(final Thread thread, final ThreadStateTransitionValidator newState, final ThreadStateTransitionValidator oldState);
}
@@ -172,6 +176,7 @@ public class StreamThread extends Thread {
/**
* Sets the state
+ *
* @param newState New state
* @return The state prior to the call to setState, or null if the transition is invalid
*/
@@ -245,12 +250,12 @@ public class StreamThread extends Thread {
@Override
public void onPartitionsAssigned(final Collection<TopicPartition> assignment) {
log.debug("at state {}: partitions {} assigned at the end of consumer rebalance.\n" +
- "\tcurrent suspended active tasks: {}\n" +
- "\tcurrent suspended standby tasks: {}\n",
- streamThread.state,
- assignment,
- taskManager.suspendedActiveTaskIds(),
- taskManager.suspendedStandbyTaskIds());
+ "\tcurrent suspended active tasks: {}\n" +
+ "\tcurrent suspended standby tasks: {}\n",
+ streamThread.state,
+ assignment,
+ taskManager.suspendedActiveTaskIds(),
+ taskManager.suspendedStandbyTaskIds());
final long start = time.milliseconds();
try {
@@ -259,18 +264,21 @@ public class StreamThread extends Thread {
}
taskManager.createTasks(assignment);
} catch (final Throwable t) {
- log.error("Error caught during partition assignment, " +
- "will abort the current process and re-throw at the end of rebalance: {}", t);
+ log.error(
+ "Error caught during partition assignment, " +
+ "will abort the current process and re-throw at the end of rebalance: {}",
+ t
+ );
streamThread.setRebalanceException(t);
} finally {
log.info("partition assignment took {} ms.\n" +
- "\tcurrent active tasks: {}\n" +
- "\tcurrent standby tasks: {}\n" +
- "\tprevious active tasks: {}\n",
- time.milliseconds() - start,
- taskManager.activeTaskIds(),
- taskManager.standbyTaskIds(),
- taskManager.prevActiveTaskIds());
+ "\tcurrent active tasks: {}\n" +
+ "\tcurrent standby tasks: {}\n" +
+ "\tprevious active tasks: {}\n",
+ time.milliseconds() - start,
+ taskManager.activeTaskIds(),
+ taskManager.standbyTaskIds(),
+ taskManager.prevActiveTaskIds());
}
}
@@ -290,18 +298,21 @@ public class StreamThread extends Thread {
// suspend active tasks
taskManager.suspendTasksAndState();
} catch (final Throwable t) {
- log.error("Error caught during partition revocation, " +
- "will abort the current process and re-throw at the end of rebalance: {}", t);
+ log.error(
+ "Error caught during partition revocation, " +
+ "will abort the current process and re-throw at the end of rebalance: {}",
+ t
+ );
streamThread.setRebalanceException(t);
} finally {
streamThread.clearStandbyRecords();
log.info("partition revocation took {} ms.\n" +
- "\tsuspended active tasks: {}\n" +
- "\tsuspended standby tasks: {}",
- time.milliseconds() - start,
- taskManager.suspendedActiveTaskIds(),
- taskManager.suspendedStandbyTaskIds());
+ "\tsuspended active tasks: {}\n" +
+ "\tsuspended standby tasks: {}",
+ time.milliseconds() - start,
+ taskManager.suspendedActiveTaskIds(),
+ taskManager.suspendedStandbyTaskIds());
}
}
}
@@ -346,12 +357,13 @@ public class StreamThread extends Thread {
return stateDirectory;
}
- Collection<T> createTasks(final Consumer<byte[], byte[]> consumer, final Map<TaskId, Set<TopicPartition>> tasksToBeCreated) {
+ Collection<T> createTasks(final Consumer<byte[], byte[]> consumer,
+ final Map<TaskId, Set<TopicPartition>> tasksToBeCreated) {
final List<T> createdTasks = new ArrayList<>();
for (final Map.Entry<TaskId, Set<TopicPartition>> newTaskAndPartitions : tasksToBeCreated.entrySet()) {
final TaskId taskId = newTaskAndPartitions.getKey();
final Set<TopicPartition> partitions = newTaskAndPartitions.getValue();
- T task = createTask(consumer, taskId, partitions);
+ final T task = createTask(consumer, taskId, partitions);
if (task != null) {
log.trace("Created task {} with assigned partitions {}", taskId, partitions);
createdTasks.add(task);
@@ -384,14 +396,15 @@ public class StreamThread extends Thread {
final Producer<byte[], byte[]> threadProducer,
final String threadClientId,
final Logger log) {
- super(builder,
- config,
- streamsMetrics,
- stateDirectory,
- taskCreatedSensor,
- storeChangelogReader,
- time,
- log);
+ super(
+ builder,
+ config,
+ streamsMetrics,
+ stateDirectory,
+ taskCreatedSensor,
+ storeChangelogReader,
+ time,
+ log);
this.cache = cache;
this.clientSupplier = clientSupplier;
this.threadProducer = threadProducer;
@@ -399,21 +412,23 @@ public class StreamThread extends Thread {
}
@Override
- StreamTask createTask(final Consumer<byte[], byte[]> consumer, final TaskId taskId, final Set<TopicPartition> partitions) {
+ StreamTask createTask(final Consumer<byte[], byte[]> consumer,
+ final TaskId taskId,
+ final Set<TopicPartition> partitions) {
taskCreatedSensor.record();
return new StreamTask(
- taskId,
- partitions,
- builder.build(taskId.topicGroupId),
- consumer,
- storeChangelogReader,
- config,
- streamsMetrics,
- stateDirectory,
- cache,
- time,
- createProducer(taskId));
+ taskId,
+ partitions,
+ builder.build(taskId.topicGroupId),
+ consumer,
+ storeChangelogReader,
+ config,
+ streamsMetrics,
+ stateDirectory,
+ cache,
+ time,
+ createProducer(taskId));
}
@@ -450,14 +465,15 @@ public class StreamThread extends Thread {
final ChangelogReader storeChangelogReader,
final Time time,
final Logger log) {
- super(builder,
- config,
- streamsMetrics,
- stateDirectory,
- taskCreatedSensor,
- storeChangelogReader,
- time,
- log);
+ super(
+ builder,
+ config,
+ streamsMetrics,
+ stateDirectory,
+ taskCreatedSensor,
+ storeChangelogReader,
+ time,
+ log);
}
@Override
@@ -469,17 +485,21 @@ public class StreamThread extends Thread {
final ProcessorTopology topology = builder.build(taskId.topicGroupId);
if (!topology.stateStores().isEmpty()) {
- return new StandbyTask(taskId,
- partitions,
- topology,
- consumer,
- storeChangelogReader,
- config,
- streamsMetrics,
- stateDirectory);
+ return new StandbyTask(
+ taskId,
+ partitions,
+ topology,
+ consumer,
+ storeChangelogReader,
+ config,
+ streamsMetrics,
+ stateDirectory);
} else {
- log.trace("Skipped standby task {} with assigned partitions {} " +
- "since it does not have any state stores to materialize", taskId, partitions);
+ log.trace(
+ "Skipped standby task {} with assigned partitions {} " +
+ "since it does not have any state stores to materialize",
+ taskId, partitions
+ );
return null;
}
}
@@ -531,11 +551,22 @@ public class StreamThread extends Thread {
}
- private Meter createMeter(Metrics metrics, SampledStat stat, String baseName, String descriptiveName) {
- MetricName rateMetricName = metrics.metricName(baseName + "-rate", groupName,
- String.format("The average per-second number of %s", descriptiveName), tags);
- MetricName totalMetricName = metrics.metricName(baseName + "-total", groupName,
- String.format("The total number of %s", descriptiveName), tags);
+ private Meter createMeter(final Metrics metrics,
+ final SampledStat stat,
+ final String baseName,
+ final String descriptiveName) {
+ final MetricName rateMetricName = metrics.metricName(
+ baseName + "-rate",
+ groupName,
+ String.format("The average per-second number of %s", descriptiveName),
+ tags
+ );
+ final MetricName totalMetricName = metrics.metricName(
+ baseName + "-total",
+ groupName,
+ String.format("The total number of %s", descriptiveName),
+ tags
+ );
return new Meter(stat, rateMetricName, totalMetricName);
}
@@ -562,7 +593,7 @@ public class StreamThread extends Thread {
private long lastCommitMs;
private long timerStartedMs;
- private String originalReset;
+ private final String originalReset;
private Throwable rebalanceException = null;
private boolean processStandbyRecords = false;
private volatile State state = State.CREATED;
@@ -607,44 +638,47 @@ public class StreamThread extends Thread {
threadProducer = clientSupplier.getProducer(producerConfigs);
}
- StreamsMetricsThreadImpl streamsMetrics = new StreamsMetricsThreadImpl(
- metrics,
- "stream-metrics",
- "thread." + threadClientId,
- Collections.singletonMap("client-id", threadClientId));
+ final StreamsMetricsThreadImpl streamsMetrics = new StreamsMetricsThreadImpl(
+ metrics,
+ "stream-metrics",
+ "thread." + threadClientId,
+ Collections.singletonMap("client-id", threadClientId));
final ThreadCache cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics);
- final AbstractTaskCreator<StreamTask> activeTaskCreator = new TaskCreator(builder,
- config,
- streamsMetrics,
- stateDirectory,
- streamsMetrics.taskCreatedSensor,
- changelogReader,
- cache,
- time,
- clientSupplier,
- threadProducer,
- threadClientId,
- log);
- final AbstractTaskCreator<StandbyTask> standbyTaskCreator = new StandbyTaskCreator(builder,
- config,
- streamsMetrics,
- stateDirectory,
- streamsMetrics.taskCreatedSensor,
- changelogReader,
- time,
- log);
- TaskManager taskManager = new TaskManager(changelogReader,
- processId,
- logPrefix,
- restoreConsumer,
- streamsMetadataState,
- activeTaskCreator,
- standbyTaskCreator,
- adminClient,
- new AssignedStreamsTasks(logContext),
- new AssignedStandbyTasks(logContext));
+ final AbstractTaskCreator<StreamTask> activeTaskCreator = new TaskCreator(
+ builder,
+ config,
+ streamsMetrics,
+ stateDirectory,
+ streamsMetrics.taskCreatedSensor,
+ changelogReader,
+ cache,
+ time,
+ clientSupplier,
+ threadProducer,
+ threadClientId,
+ log);
+ final AbstractTaskCreator<StandbyTask> standbyTaskCreator = new StandbyTaskCreator(
+ builder,
+ config,
+ streamsMetrics,
+ stateDirectory,
+ streamsMetrics.taskCreatedSensor,
+ changelogReader,
+ time,
+ log);
+ final TaskManager taskManager = new TaskManager(
+ changelogReader,
+ processId,
+ logPrefix,
+ restoreConsumer,
+ streamsMetadataState,
+ activeTaskCreator,
+ standbyTaskCreator,
+ adminClient,
+ new AssignedStreamsTasks(logContext),
+ new AssignedStandbyTasks(logContext));
log.info("Creating consumer client");
final String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG);
@@ -658,16 +692,17 @@ public class StreamThread extends Thread {
final Consumer<byte[], byte[]> consumer = clientSupplier.getConsumer(consumerConfigs);
taskManager.setConsumer(consumer);
- return new StreamThread(time,
- config,
- restoreConsumer,
- consumer,
- originalReset,
- taskManager,
- streamsMetrics,
- builder,
- threadClientId,
- logContext);
+ return new StreamThread(
+ time,
+ config,
+ restoreConsumer,
+ consumer,
+ originalReset,
+ taskManager,
+ streamsMetrics,
+ builder,
+ threadClientId,
+ logContext);
}
public StreamThread(final Time time,
@@ -705,8 +740,8 @@ public class StreamThread extends Thread {
/**
* Execute the stream processors
*
- * @throws KafkaException for any Kafka-related exceptions
- * @throws RuntimeException for any other non-Kafka exceptions
+ * @throws KafkaException for any Kafka-related exceptions
+ * @throws RuntimeException for any other non-Kafka exceptions
*/
@Override
public void run() {
@@ -732,14 +767,15 @@ public class StreamThread extends Thread {
}
}
- void setRebalanceException(final Throwable rebalanceException) {
+ private void setRebalanceException(final Throwable rebalanceException) {
this.rebalanceException = rebalanceException;
}
/**
* Main event loop for polling, and processing records through topologies.
+ *
* @throws IllegalStateException If store gets registered after initialized is already finished
- * @throws StreamsException if the store's change log does not contain the partition
+ * @throws StreamsException if the store's change log does not contain the partition
*/
private void runLoop() {
long recordsProcessedBeforeCommit = UNLIMITED_RECORDS;
@@ -750,9 +786,9 @@ public class StreamThread extends Thread {
recordsProcessedBeforeCommit = runOnce(recordsProcessedBeforeCommit);
} catch (final TaskMigratedException ignoreAndRejoinGroup) {
log.warn("Detected task {} that got migrated to another thread. " +
- "This implies that this thread missed a rebalance and dropped out of the consumer group. " +
- "Will try to rejoin the consumer group. Below is the detailed description of the task:\n{}",
- ignoreAndRejoinGroup.migratedTask().id(), ignoreAndRejoinGroup.migratedTask().toString(">"));
+ "This implies that this thread missed a rebalance and dropped out of the consumer group. " +
+ "Will try to rejoin the consumer group. Below is the detailed description of the task:\n{}",
+ ignoreAndRejoinGroup.migratedTask().id(), ignoreAndRejoinGroup.migratedTask().toString(">"));
// re-subscribe to enforce a rebalance in the next poll call
consumer.unsubscribe();
@@ -763,8 +799,8 @@ public class StreamThread extends Thread {
/**
* @throws IllegalStateException If store gets registered after initialized is already finished
- * @throws StreamsException if the store's change log does not contain the partition
- * @throws TaskMigratedException if another thread wrote to the changelog topic that is currently restored
+ * @throws StreamsException If the store's change log does not contain the partition
+ * @throws TaskMigratedException If another thread wrote to the changelog topic that is currently restored
* or if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
*/
@@ -772,7 +808,7 @@ public class StreamThread extends Thread {
long runOnce(final long recordsProcessedBeforeCommit) {
long processedBeforeCommit = recordsProcessedBeforeCommit;
- ConsumerRecords<byte[], byte[]> records;
+ final ConsumerRecords<byte[], byte[]> records;
timerStartedMs = time.milliseconds();
@@ -803,12 +839,12 @@ public class StreamThread extends Thread {
final long totalProcessed = processAndMaybeCommit(recordsProcessedBeforeCommit);
if (totalProcessed > 0) {
final long processLatency = computeLatency();
- streamsMetrics.processTimeSensor.record(processLatency / (double) totalProcessed,
- timerStartedMs);
- processedBeforeCommit = adjustRecordsProcessedBeforeCommit(recordsProcessedBeforeCommit,
- totalProcessed,
- processLatency,
- commitTimeMs);
+ streamsMetrics.processTimeSensor.record(processLatency / (double) totalProcessed, timerStartedMs);
+ processedBeforeCommit = adjustRecordsProcessedBeforeCommit(
+ recordsProcessedBeforeCommit,
+ totalProcessed,
+ processLatency,
+ commitTimeMs);
}
}
@@ -890,6 +926,7 @@ public class StreamThread extends Thread {
/**
* Take records and add them to each respective task
+ *
* @param records Records, can be null
*/
private void addRecordsToTasks(final ConsumerRecords<byte[], byte[]> records) {
@@ -900,7 +937,7 @@ public class StreamThread extends Thread {
if (task.isClosed()) {
log.info("Stream task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
- "Notifying the thread to trigger a new rebalance immediately.", task.id());
+ "Notifying the thread to trigger a new rebalance immediately.", task.id());
throw new TaskMigratedException(task);
}
@@ -912,6 +949,7 @@ public class StreamThread extends Thread {
/**
* Schedule the records processing by selecting which record is processed next. Commits may
* happen as records are processed.
+ *
* @param recordsProcessedBeforeCommit number of records to be processed before commit is called.
* if UNLIMITED_RECORDS, then commit is never called
* @return Number of records processed since last commit.
@@ -961,10 +999,11 @@ public class StreamThread extends Thread {
/**
* Adjust the number of records that should be processed by scheduler. This avoids
* scenarios where the processing time is higher than the commit time.
+ *
* @param prevRecordsProcessedBeforeCommit Previous number of records processed by scheduler.
- * @param totalProcessed Total number of records processed in this last round.
- * @param processLatency Total processing latency in ms processed in this last round.
- * @param commitTime Desired commit time in ms.
+ * @param totalProcessed Total number of records processed in this last round.
+ * @param processLatency Total processing latency in ms processed in this last round.
+ * @param commitTime Desired commit time in ms.
* @return An adjusted number of records to be processed in the next round.
*/
private long adjustRecordsProcessedBeforeCommit(final long prevRecordsProcessedBeforeCommit, final long totalProcessed,
@@ -978,12 +1017,12 @@ public class StreamThread extends Thread {
// push down
recordsProcessedBeforeCommit = Math.max(1, (commitTime * totalProcessed) / processLatency);
log.debug("processing latency {} > commit time {} for {} records. Adjusting down recordsProcessedBeforeCommit={}",
- processLatency, commitTime, totalProcessed, recordsProcessedBeforeCommit);
+ processLatency, commitTime, totalProcessed, recordsProcessedBeforeCommit);
} else if (prevRecordsProcessedBeforeCommit != UNLIMITED_RECORDS && processLatency > 0) {
// push up
recordsProcessedBeforeCommit = Math.max(1, (commitTime * totalProcessed) / processLatency);
log.debug("processing latency {} < commit time {} for {} records. Adjusting up recordsProcessedBeforeCommit={}",
- processLatency, commitTime, totalProcessed, recordsProcessedBeforeCommit);
+ processLatency, commitTime, totalProcessed, recordsProcessedBeforeCommit);
}
return recordsProcessedBeforeCommit;
@@ -991,6 +1030,7 @@ public class StreamThread extends Thread {
/**
* Commit all tasks owned by this thread if specified interval time has elapsed
+ *
* @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
*/
@@ -998,10 +1038,10 @@ public class StreamThread extends Thread {
if (commitTimeMs >= 0 && lastCommitMs + commitTimeMs < now) {
if (log.isTraceEnabled()) {
log.trace("Committing all active tasks {} and standby tasks {} since {}ms has elapsed (commit interval is {}ms)",
- taskManager.activeTaskIds(), taskManager.standbyTaskIds(), now - lastCommitMs, commitTimeMs);
+ taskManager.activeTaskIds(), taskManager.standbyTaskIds(), now - lastCommitMs, commitTimeMs);
}
- int committed = taskManager.commitAll();
+ final int committed = taskManager.commitAll();
if (committed > 0) {
streamsMetrics.commitTimeSensor.record(computeLatency() / (double) committed, timerStartedMs);
@@ -1010,7 +1050,7 @@ public class StreamThread extends Thread {
}
if (log.isDebugEnabled()) {
log.debug("Committed all active tasks {} and standby tasks {} in {}ms",
- taskManager.activeTaskIds(), taskManager.standbyTaskIds(), timerStartedMs - now);
+ taskManager.activeTaskIds(), taskManager.standbyTaskIds(), timerStartedMs - now);
}
lastCommitMs = now;
@@ -1033,7 +1073,7 @@ public class StreamThread extends Thread {
if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
- "Notifying the thread to trigger a new rebalance immediately.", task.id());
+ "Notifying the thread to trigger a new rebalance immediately.", task.id());
throw new TaskMigratedException(task);
}
@@ -1066,7 +1106,7 @@ public class StreamThread extends Thread {
if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
- "Notifying the thread to trigger a new rebalance immediately.", task.id());
+ "Notifying the thread to trigger a new rebalance immediately.", task.id());
throw new TaskMigratedException(task);
}
@@ -1085,7 +1125,7 @@ public class StreamThread extends Thread {
if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
- "Notifying the thread to trigger a new rebalance immediately.", task.id());
+ "Notifying the thread to trigger a new rebalance immediately.", task.id());
throw new TaskMigratedException(task);
}
@@ -1112,13 +1152,13 @@ public class StreamThread extends Thread {
/**
* Shutdown this stream thread.
- *
+ * <p>
* Note that there is nothing to prevent this function from being called multiple times
* (e.g., in testing), hence the state is set only the first time
*/
public void shutdown() {
log.info("Informed to shut down");
- State oldState = setState(State.PENDING_SHUTDOWN);
+ final State oldState = setState(State.PENDING_SHUTDOWN);
if (oldState == State.CREATED) {
// The thread may not have been started. Take responsibility for shutting down
completeShutdown(true);
@@ -1169,11 +1209,11 @@ public class StreamThread extends Thread {
private void updateThreadMetadata(final Map<TaskId, StreamTask> activeTasks, final Map<TaskId, StandbyTask> standbyTasks) {
final Set<TaskMetadata> activeTasksMetadata = new HashSet<>();
- for (Map.Entry<TaskId, StreamTask> task : activeTasks.entrySet()) {
+ for (final Map.Entry<TaskId, StreamTask> task : activeTasks.entrySet()) {
activeTasksMetadata.add(new TaskMetadata(task.getKey().toString(), task.getValue().partitions()));
}
final Set<TaskMetadata> standbyTasksMetadata = new HashSet<>();
- for (Map.Entry<TaskId, StandbyTask> task : standbyTasks.entrySet()) {
+ for (final Map.Entry<TaskId, StandbyTask> task : standbyTasks.entrySet()) {
standbyTasksMetadata.add(new TaskMetadata(task.getKey().toString(), task.getValue().partitions()));
}
@@ -1187,6 +1227,7 @@ public class StreamThread extends Thread {
/**
* Produces a string representation containing useful information about a StreamThread.
* This is useful in debugging scenarios.
+ *
* @return A string representation of the StreamThread instance.
*/
@Override
@@ -1197,14 +1238,11 @@ public class StreamThread extends Thread {
/**
* Produces a string representation containing useful information about a StreamThread, starting with the given indent.
* This is useful in debugging scenarios.
+ *
* @return A string representation of the StreamThread instance.
*/
public String toString(final String indent) {
- final StringBuilder sb = new StringBuilder()
- .append(indent).append("\tStreamsThread threadId: ").append(getName()).append("\n");
-
- sb.append(taskManager.toString(indent));
- return sb.toString();
+ return indent + "\tStreamsThread threadId: " + getName() + "\n" + taskManager.toString(indent);
}
// the following are for testing only
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java
index b2ce2e7..83b4f12 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java
@@ -30,13 +30,11 @@ import org.apache.kafka.streams.StreamsMetrics;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Arrays;
public class StreamsMetricsImpl implements StreamsMetrics {
private static final Logger log = LoggerFactory.getLogger(StreamsMetricsImpl.class);
@@ -44,9 +42,9 @@ public class StreamsMetricsImpl implements StreamsMetrics {
final Metrics metrics;
final String groupName;
final Map<String, String> tags;
- final Map<Sensor, Sensor> parentSensors;
+ private final Map<Sensor, Sensor> parentSensors;
- public StreamsMetricsImpl(Metrics metrics, String groupName, Map<String, String> tags) {
+ public StreamsMetricsImpl(final Metrics metrics, final String groupName, final Map<String, String> tags) {
Objects.requireNonNull(metrics, "Metrics cannot be null");
this.metrics = metrics;
@@ -60,12 +58,12 @@ public class StreamsMetricsImpl implements StreamsMetrics {
}
@Override
- public Sensor addSensor(String name, Sensor.RecordingLevel recordingLevel) {
+ public Sensor addSensor(final String name, final Sensor.RecordingLevel recordingLevel) {
return metrics.sensor(name, recordingLevel);
}
@Override
- public Sensor addSensor(String name, Sensor.RecordingLevel recordingLevel, Sensor... parents) {
+ public Sensor addSensor(final String name, final Sensor.RecordingLevel recordingLevel, final Sensor... parents) {
return metrics.sensor(name, recordingLevel, parents);
}
@@ -75,21 +73,21 @@ public class StreamsMetricsImpl implements StreamsMetrics {
}
@Override
- public void recordLatency(Sensor sensor, long startNs, long endNs) {
+ public void recordLatency(final Sensor sensor, final long startNs, final long endNs) {
sensor.record(endNs - startNs);
}
@Override
- public void recordThroughput(Sensor sensor, long value) {
+ public void recordThroughput(final Sensor sensor, final long value) {
sensor.record(value);
}
- private String groupNameFromScope(String scopeName) {
+ private String groupNameFromScope(final String scopeName) {
return "stream-" + scopeName + "-metrics";
}
- private String sensorName(String operationName, String entityName) {
+ private String sensorName(final String operationName, final String entityName) {
if (entityName == null) {
return operationName;
} else {
@@ -97,12 +95,13 @@ public class StreamsMetricsImpl implements StreamsMetrics {
}
}
- public Map<String, String> tagMap(String... tags) {
+ public Map<String, String> tagMap(final String... tags) {
// extract the additional tags if there are any
- Map<String, String> tagMap = new HashMap<>(this.tags);
+ final Map<String, String> tagMap = new HashMap<>(this.tags);
if (tags != null) {
- if ((tags.length % 2) != 0)
+ if ((tags.length % 2) != 0) {
throw new IllegalArgumentException("Tags needs to be specified in key-value pairs");
+ }
for (int i = 0; i < tags.length; i += 2)
tagMap.put(tags[i], tags[i + 1]);
@@ -112,27 +111,30 @@ public class StreamsMetricsImpl implements StreamsMetrics {
private Map<String, String> constructTags(final String scopeName, final String entityName, final String... tags) {
- List<String> updatedTagList = new ArrayList<>(Arrays.asList(tags));
- updatedTagList.add(scopeName + "-id");
- updatedTagList.add(entityName);
- return tagMap(updatedTagList.toArray(new String[updatedTagList.size()]));
+ final String[] updatedTags = Arrays.copyOf(tags, tags.length + 2);
+ updatedTags[tags.length] = scopeName + "-id";
+ updatedTags[tags.length + 1] = entityName;
+ return tagMap(updatedTags);
}
/**
* @throws IllegalArgumentException if tags is not constructed in key-value pairs
*/
@Override
- public Sensor addLatencyAndThroughputSensor(String scopeName, String entityName, String operationName,
- Sensor.RecordingLevel recordingLevel, String... tags) {
+ public Sensor addLatencyAndThroughputSensor(final String scopeName,
+ final String entityName,
+ final String operationName,
+ final Sensor.RecordingLevel recordingLevel,
+ final String... tags) {
final Map<String, String> tagMap = constructTags(scopeName, entityName, tags);
final Map<String, String> allTagMap = constructTags(scopeName, "all", tags);
// first add the global operation metrics if not yet, with the global tags only
- Sensor parent = metrics.sensor(sensorName(operationName, null), recordingLevel);
+ final Sensor parent = metrics.sensor(sensorName(operationName, null), recordingLevel);
addLatencyAndThroughputMetrics(scopeName, parent, operationName, allTagMap);
// add the operation metrics with additional tags
- Sensor sensor = metrics.sensor(sensorName(operationName, entityName), recordingLevel, parent);
+ final Sensor sensor = metrics.sensor(sensorName(operationName, entityName), recordingLevel, parent);
addLatencyAndThroughputMetrics(scopeName, sensor, operationName, tagMap);
parentSensors.put(sensor, parent);
@@ -144,16 +146,20 @@ public class StreamsMetricsImpl implements StreamsMetrics {
* @throws IllegalArgumentException if tags is not constructed in key-value pairs
*/
@Override
- public Sensor addThroughputSensor(String scopeName, String entityName, String operationName, Sensor.RecordingLevel recordingLevel, String... tags) {
+ public Sensor addThroughputSensor(final String scopeName,
+ final String entityName,
+ final String operationName,
+ final Sensor.RecordingLevel recordingLevel,
+ final String... tags) {
final Map<String, String> tagMap = constructTags(scopeName, entityName, tags);
final Map<String, String> allTagMap = constructTags(scopeName, "all", tags);
// first add the global operation metrics if not yet, with the global tags only
- Sensor parent = metrics.sensor(sensorName(operationName, null), recordingLevel);
+ final Sensor parent = metrics.sensor(sensorName(operationName, null), recordingLevel);
addThroughputMetrics(scopeName, parent, operationName, allTagMap);
// add the operation metrics with additional tags
- Sensor sensor = metrics.sensor(sensorName(operationName, entityName), recordingLevel, parent);
+ final Sensor sensor = metrics.sensor(sensorName(operationName, entityName), recordingLevel, parent);
addThroughputMetrics(scopeName, sensor, operationName, tagMap);
parentSensors.put(sensor, parent);
@@ -161,8 +167,10 @@ public class StreamsMetricsImpl implements StreamsMetrics {
return sensor;
}
- private void addLatencyAndThroughputMetrics(String scopeName, Sensor sensor, String opName, Map<String, String> tags) {
-
+ private void addLatencyAndThroughputMetrics(final String scopeName,
+ final Sensor sensor,
+ final String opName,
+ final Map<String, String> tags) {
maybeAddMetric(sensor, metrics.metricName(opName + "-latency-avg", groupNameFromScope(scopeName),
"The average latency of " + opName + " operation.", tags), new Avg());
maybeAddMetric(sensor, metrics.metricName(opName + "-latency-max", groupNameFromScope(scopeName),
@@ -170,11 +178,14 @@ public class StreamsMetricsImpl implements StreamsMetrics {
addThroughputMetrics(scopeName, sensor, opName, tags);
}
- private void addThroughputMetrics(String scopeName, Sensor sensor, String opName, Map<String, String> tags) {
- MetricName rateMetricName = metrics.metricName(opName + "-rate", groupNameFromScope(scopeName),
+ private void addThroughputMetrics(final String scopeName,
+ final Sensor sensor,
+ final String opName,
+ final Map<String, String> tags) {
+ final MetricName rateMetricName = metrics.metricName(opName + "-rate", groupNameFromScope(scopeName),
"The average number of occurrence of " + opName + " operation per second.", tags);
- MetricName totalMetricName = metrics.metricName(opName + "-total", groupNameFromScope(scopeName),
- "The total number of occurrence of " + opName + " operations.", tags);
+ final MetricName totalMetricName = metrics.metricName(opName + "-total", groupNameFromScope(scopeName),
+ "The total number of occurrence of " + opName + " operations.", tags);
if (!metrics.metrics().containsKey(rateMetricName) && !metrics.metrics().containsKey(totalMetricName)) {
sensor.add(new Meter(new Count(), rateMetricName, totalMetricName));
} else {
@@ -182,12 +193,16 @@ public class StreamsMetricsImpl implements StreamsMetrics {
}
}
- public void maybeAddMetric(Sensor sensor, MetricName name, MeasurableStat stat) {
- if (!metrics.metrics().containsKey(name)) {
- sensor.add(name, stat);
- } else {
- log.trace("Trying to add metric twice: {}", name);
- }
+ /**
+ * Register a metric on the sensor if it isn't already there.
+ *
+ * @param sensor The sensor on which to register the metric
+ * @param name The name of the metric
+ * @param stat The metric to track
+ * @throws IllegalArgumentException if the same metric name is already in use elsewhere in the metrics
+ */
+ public void maybeAddMetric(final Sensor sensor, final MetricName name, final MeasurableStat stat) {
+ sensor.add(name, stat);
}
/**
@@ -196,11 +211,12 @@ public class StreamsMetricsImpl implements StreamsMetrics {
* action.run()
* endTs = time.nanoseconds()
* sensor.record(endTs - startTs)
- * @param time Time object.
- * @param action Action to run.
- * @param sensor Sensor to record value.
+ *
+ * @param time Time object.
+ * @param action Action to run.
+ * @param sensor Sensor to record value.
*/
- public void measureLatencyNs(final Time time, final Runnable action, final Sensor sensor) {
+ void measureLatencyNs(final Time time, final Runnable action, final Sensor sensor) {
long startNs = -1;
if (sensor.shouldRecord()) {
startNs = time.nanoseconds();
@@ -215,7 +231,7 @@ public class StreamsMetricsImpl implements StreamsMetrics {
* Deletes a sensor and its parents, if any
*/
@Override
- public void removeSensor(Sensor sensor) {
+ public void removeSensor(final Sensor sensor) {
Objects.requireNonNull(sensor, "Sensor is null");
metrics.removeSensor(sensor.name());
@@ -225,5 +241,4 @@ public class StreamsMetricsImpl implements StreamsMetrics {
}
}
-
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
index f838c55..01ee6b1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
@@ -32,7 +32,6 @@ import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
-import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
@@ -46,7 +45,7 @@ class NamedCache {
private LRUNode tail;
private LRUNode head;
private long currentSizeBytes;
- private NamedCacheMetrics namedCacheMetrics;
+ private final NamedCacheMetrics namedCacheMetrics;
// internal stats
private long numReadHits = 0;
@@ -56,7 +55,7 @@ class NamedCache {
NamedCache(final String name, final StreamsMetrics metrics) {
this.name = name;
- this.namedCacheMetrics = new NamedCacheMetrics(metrics);
+ this.namedCacheMetrics = new NamedCacheMetrics(metrics, name);
}
synchronized final String name() {
@@ -105,7 +104,7 @@ class NamedCache {
if (log.isTraceEnabled()) {
log.trace("Named cache {} stats on flush: #hits={}, #misses={}, #overwrites={}, #flushes={}",
- name, hits(), misses(), overwrites(), flushes());
+ name, hits(), misses(), overwrites(), flushes());
}
if (listener == null) {
@@ -116,7 +115,7 @@ class NamedCache {
return;
}
- final List<ThreadCache.DirtyEntry> entries = new ArrayList<>();
+ final List<ThreadCache.DirtyEntry> entries = new ArrayList<>();
final List<Bytes> deleted = new ArrayList<>();
// evicted already been removed from the cache so add it to the list of
@@ -126,7 +125,7 @@ class NamedCache {
dirtyKeys.remove(evicted.key);
}
- for (Bytes key : dirtyKeys) {
+ for (final Bytes key : dirtyKeys) {
final LRUNode node = getInternal(key);
if (node == null) {
throw new IllegalStateException("Key = " + key + " found in dirty key set, but entry is null");
@@ -140,17 +139,19 @@ class NamedCache {
// clear dirtyKeys before the listener is applied as it may be re-entrant.
dirtyKeys.clear();
listener.apply(entries);
- for (Bytes key : deleted) {
+ for (final Bytes key : deleted) {
delete(key);
}
}
synchronized void put(final Bytes key, final LRUCacheEntry value) {
if (!value.isDirty() && dirtyKeys.contains(key)) {
- throw new IllegalStateException(String.format("Attempting to put a clean entry for key [%s] " +
- "into NamedCache [%s] when it already contains " +
- "a dirty entry for the same key",
- key, name));
+ throw new IllegalStateException(
+ String.format(
+ "Attempting to put a clean entry for key [%s] into NamedCache [%s] when it already contains a dirty entry for the same key",
+ key, name
+ )
+ );
}
LRUNode node = cache.get(key);
if (node != null) {
@@ -190,13 +191,13 @@ class NamedCache {
return node;
}
- private void updateLRU(LRUNode node) {
+ private void updateLRU(final LRUNode node) {
remove(node);
putHead(node);
}
- private void remove(LRUNode node) {
+ private void remove(final LRUNode node) {
if (node.previous != null) {
node.previous.next = node.next;
} else {
@@ -209,7 +210,7 @@ class NamedCache {
}
}
- private void putHead(LRUNode node) {
+ private void putHead(final LRUNode node) {
node.next = head;
node.previous = null;
if (head != null) {
@@ -243,7 +244,7 @@ class NamedCache {
}
synchronized void putAll(final List<KeyValue<byte[], LRUCacheEntry>> entries) {
- for (KeyValue<byte[], LRUCacheEntry> entry : entries) {
+ for (final KeyValue<byte[], LRUCacheEntry> entry : entries) {
put(Bytes.wrap(entry.key), entry.value);
}
}
@@ -271,18 +272,12 @@ class NamedCache {
}
private Iterator<Bytes> keySetIterator(final Set<Bytes> keySet) {
- final TreeSet<Bytes> copy = new TreeSet<>();
- copy.addAll(keySet);
- return copy.iterator();
+ return new TreeSet<>(keySet).iterator();
}
synchronized Iterator<Bytes> allKeys() {
return keySetIterator(cache.navigableKeySet());
}
-
- synchronized NavigableSet<Bytes> keySet() {
- return cache.navigableKeySet();
- }
synchronized LRUCacheEntry first() {
if (head == null) {
@@ -338,11 +333,11 @@ class NamedCache {
}
long size() {
- return key.get().length +
- 8 + // entry
- 8 + // previous
- 8 + // next
- entry.size();
+ return key.get().length +
+ 8 + // entry
+ 8 + // previous
+ 8 + // next
+ entry.size();
}
LRUNode next() {
@@ -353,19 +348,19 @@ class NamedCache {
return previous;
}
- private void update(LRUCacheEntry entry) {
+ private void update(final LRUCacheEntry entry) {
this.entry = entry;
}
}
- class NamedCacheMetrics {
- final StreamsMetricsImpl metrics;
- final String groupName;
- final Map<String, String> metricTags;
- final Map<String, String> allMetricTags;
- final Sensor hitRatioSensor;
+ private static class NamedCacheMetrics {
+ private final StreamsMetricsImpl metrics;
+ private final String groupName;
+ private final Map<String, String> metricTags;
+ private final Map<String, String> allMetricTags;
+ private final Sensor hitRatioSensor;
- public NamedCacheMetrics(StreamsMetrics metrics) {
+ private NamedCacheMetrics(final StreamsMetrics metrics, final String name) {
final String scope = "record-cache";
final String opName = "hitRatio";
final String tagKey = scope + "-id";
@@ -373,18 +368,18 @@ class NamedCache {
this.groupName = "stream-" + scope + "-metrics";
this.metrics = (StreamsMetricsImpl) metrics;
this.allMetricTags = ((StreamsMetricsImpl) metrics).tagMap(tagKey, "all",
- "task-id", ThreadCache.taskIDfromCacheName(name));
+ "task-id", ThreadCache.taskIDfromCacheName(name));
this.metricTags = ((StreamsMetricsImpl) metrics).tagMap(tagKey, tagValue,
- "task-id", ThreadCache.taskIDfromCacheName(name));
+ "task-id", ThreadCache.taskIDfromCacheName(name));
// add parent
- Sensor parent = this.metrics.registry().sensor(opName, Sensor.RecordingLevel.DEBUG);
+ final Sensor parent = this.metrics.registry().sensor(opName, Sensor.RecordingLevel.DEBUG);
((StreamsMetricsImpl) metrics).maybeAddMetric(parent, this.metrics.registry().metricName(opName + "-avg", groupName,
- "The average cache hit ratio.", allMetricTags), new Avg());
+ "The average cache hit ratio.", allMetricTags), new Avg());
((StreamsMetricsImpl) metrics).maybeAddMetric(parent, this.metrics.registry().metricName(opName + "-min", groupName,
- "The minimum cache hit ratio.", allMetricTags), new Min());
+ "The minimum cache hit ratio.", allMetricTags), new Min());
((StreamsMetricsImpl) metrics).maybeAddMetric(parent, this.metrics.registry().metricName(opName + "-max", groupName,
- "The maximum cache hit ratio.", allMetricTags), new Max());
+ "The maximum cache hit ratio.", allMetricTags), new Max());
// add child
hitRatioSensor = this.metrics.registry().sensor(opName, Sensor.RecordingLevel.DEBUG, parent);
@@ -397,7 +392,7 @@ class NamedCache {
}
- public void removeAllSensors() {
+ private void removeAllSensors() {
metrics.removeSensor(hitRatioSensor);
}
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java
index 0f8109d..eebb7d2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java
@@ -73,40 +73,42 @@ public class GlobalKTableJoinsTest {
@Test
public void shouldLeftJoinWithStream() {
- stream.leftJoin(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER)
- .foreach(action);
+ stream
+ .leftJoin(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER)
+ .foreach(action);
final Map<String, String> expected = new HashMap<>();
expected.put("1", "a+A");
expected.put("2", "b+B");
expected.put("3", "c+null");
- verifyJoin(expected, streamTopic);
+ verifyJoin(expected);
}
@Test
public void shouldInnerJoinWithStream() {
- stream.join(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER)
- .foreach(action);
+ stream
+ .join(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER)
+ .foreach(action);
final Map<String, String> expected = new HashMap<>();
expected.put("1", "a+A");
expected.put("2", "b+B");
- verifyJoin(expected, streamTopic);
+ verifyJoin(expected);
}
- private void verifyJoin(final Map<String, String> expected, final String joinInput) {
+ private void verifyJoin(final Map<String, String> expected) {
driver.setUp(builder, stateDir);
driver.setTime(0L);
// write some data to the global table
driver.process(globalTopic, "a", "A");
driver.process(globalTopic, "b", "B");
//write some data to the stream
- driver.process(joinInput, "1", "a");
- driver.process(joinInput, "2", "b");
- driver.process(joinInput, "3", "c");
+ driver.process(streamTopic, "1", "a");
+ driver.process(streamTopic, "2", "b");
+ driver.process(streamTopic, "3", "c");
driver.flushState();
assertEquals(expected, results);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
index 9d8b479..2b87b30 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
@@ -220,7 +220,7 @@ public class KGroupedStreamImplTest {
@Test
public void shouldAggregateSessionWindows() {
final Map<Windowed<String>, Integer> results = new HashMap<>();
- KTable<Windowed<String>, Integer> table = groupedStream.aggregate(new Initializer<Integer>() {
+ final KTable<Windowed<String>, Integer> table = groupedStream.aggregate(new Initializer<Integer>() {
@Override
public Integer apply() {
return 0;
@@ -251,7 +251,7 @@ public class KGroupedStreamImplTest {
@Test
public void shouldAggregateSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, Integer> results = new HashMap<>();
- KTable<Windowed<String>, Integer> table = groupedStream.aggregate(new Initializer<Integer>() {
+ final KTable<Windowed<String>, Integer> table = groupedStream.aggregate(new Initializer<Integer>() {
@Override
public Integer apply() {
return 0;
@@ -301,7 +301,7 @@ public class KGroupedStreamImplTest {
@Test
public void shouldCountSessionWindows() {
final Map<Windowed<String>, Long> results = new HashMap<>();
- KTable<Windowed<String>, Long> table = groupedStream.count(SessionWindows.with(30), "session-store");
+ final KTable<Windowed<String>, Long> table = groupedStream.count(SessionWindows.with(30), "session-store");
table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() {
@Override
public void apply(final Windowed<String> key, final Long value) {
@@ -316,7 +316,7 @@ public class KGroupedStreamImplTest {
@Test
public void shouldCountSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, Long> results = new HashMap<>();
- KTable<Windowed<String>, Long> table = groupedStream.count(SessionWindows.with(30));
+ final KTable<Windowed<String>, Long> table = groupedStream.count(SessionWindows.with(30));
table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() {
@Override
public void apply(final Windowed<String> key, final Long value) {
@@ -351,14 +351,16 @@ public class KGroupedStreamImplTest {
@Test
public void shouldReduceSessionWindows() {
final Map<Windowed<String>, String> results = new HashMap<>();
- KTable<Windowed<String>, String> table = groupedStream.reduce(
- new Reducer<String>() {
- @Override
- public String apply(final String value1, final String value2) {
- return value1 + ":" + value2;
- }
- }, SessionWindows.with(30),
- "session-store");
+ final KTable<Windowed<String>, String> table = groupedStream.reduce(
+ new Reducer<String>() {
+ @Override
+ public String apply(final String value1, final String value2) {
+ return value1 + ":" + value2;
+ }
+ },
+ SessionWindows.with(30),
+ "session-store"
+ );
table.toStream().foreach(new ForeachAction<Windowed<String>, String>() {
@Override
public void apply(final Windowed<String> key, final String value) {
@@ -373,13 +375,15 @@ public class KGroupedStreamImplTest {
@Test
public void shouldReduceSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, String> results = new HashMap<>();
- KTable<Windowed<String>, String> table = groupedStream.reduce(
- new Reducer<String>() {
- @Override
- public String apply(final String value1, final String value2) {
- return value1 + ":" + value2;
- }
- }, SessionWindows.with(30));
+ final KTable<Windowed<String>, String> table = groupedStream.reduce(
+ new Reducer<String>() {
+ @Override
+ public String apply(final String value1, final String value2) {
+ return value1 + ":" + value2;
+ }
+ },
+ SessionWindows.with(30)
+ );
table.toStream().foreach(new ForeachAction<Windowed<String>, String>() {
@Override
public void apply(final Windowed<String> key, final String value) {
@@ -445,12 +449,13 @@ public class KGroupedStreamImplTest {
@SuppressWarnings("deprecation")
@Test(expected = NullPointerException.class)
public void shouldNotAcceptNullSessionMergerWhenAggregatingSessionWindows() {
- groupedStream.aggregate(MockInitializer.STRING_INIT,
- MockAggregator.TOSTRING_ADDER,
- null,
- SessionWindows.with(10),
- Serdes.String(),
- "storeName");
+ groupedStream.aggregate(
+ MockInitializer.STRING_INIT,
+ MockAggregator.TOSTRING_ADDER,
+ null,
+ SessionWindows.with(10),
+ Serdes.String(),
+ "storeName");
}
@SuppressWarnings("deprecation")
@@ -539,15 +544,14 @@ public class KGroupedStreamImplTest {
groupedStream.count((Materialized) null);
}
- @SuppressWarnings("unchecked")
@Test
public void shouldCountAndMaterializeResults() {
- groupedStream.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("count")
- .withKeySerde(Serdes.String()));
+ groupedStream.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("count").withKeySerde(Serdes.String()));
processData();
- final KeyValueStore<String, Long> count = (KeyValueStore<String, Long>) driver.allStateStores().get("count");
+ @SuppressWarnings("unchecked") final KeyValueStore<String, Long> count =
+ (KeyValueStore<String, Long>) driver.allStateStores().get("count");
assertThat(count.get("1"), equalTo(3L));
assertThat(count.get("2"), equalTo(1L));
@@ -555,14 +559,14 @@ public class KGroupedStreamImplTest {
}
-
@SuppressWarnings("unchecked")
@Test
public void shouldReduceAndMaterializeResults() {
- groupedStream.reduce(MockReducer.STRING_ADDER,
- Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("reduce")
- .withKeySerde(Serdes.String())
- .withValueSerde(Serdes.String()));
+ groupedStream.reduce(
+ MockReducer.STRING_ADDER,
+ Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("reduce")
+ .withKeySerde(Serdes.String())
+ .withValueSerde(Serdes.String()));
processData();
@@ -576,11 +580,12 @@ public class KGroupedStreamImplTest {
@SuppressWarnings("unchecked")
@Test
public void shouldAggregateAndMaterializeResults() {
- groupedStream.aggregate(MockInitializer.STRING_INIT,
- MockAggregator.TOSTRING_ADDER,
- Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("aggregate")
- .withKeySerde(Serdes.String())
- .withValueSerde(Serdes.String()));
+ groupedStream.aggregate(
+ MockInitializer.STRING_INIT,
+ MockAggregator.TOSTRING_ADDER,
+ Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("aggregate")
+ .withKeySerde(Serdes.String())
+ .withValueSerde(Serdes.String()));
processData();
@@ -595,15 +600,16 @@ public class KGroupedStreamImplTest {
@Test
public void shouldAggregateWithDefaultSerdes() {
final Map<String, String> results = new HashMap<>();
- groupedStream.aggregate(MockInitializer.STRING_INIT,
- MockAggregator.TOSTRING_ADDER)
- .toStream()
- .foreach(new ForeachAction<String, String>() {
- @Override
- public void apply(final String key, final String value) {
- results.put(key, value);
- }
- });
+ groupedStream.aggregate(
+ MockInitializer.STRING_INIT,
+ MockAggregator.TOSTRING_ADDER)
+ .toStream()
+ .foreach(new ForeachAction<String, String>() {
+ @Override
+ public void apply(final String key, final String value) {
+ results.put(key, value);
+ }
+ });
processData();
@@ -637,13 +643,13 @@ public class KGroupedStreamImplTest {
driver.process(TOPIC, "2", "B");
driver.process(TOPIC, "2", "B");
assertThat(results, equalTo(Arrays.asList(
- KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 1L),
- KeyValue.pair(new Windowed<>("2", new TimeWindow(0, 500)), 1L),
- KeyValue.pair(new Windowed<>("3", new TimeWindow(0, 500)), 1L),
- KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 1L),
- KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 2L),
- KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 1L),
- KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 2L)
+ KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 1L),
+ KeyValue.pair(new Windowed<>("2", new TimeWindow(0, 500)), 1L),
+ KeyValue.pair(new Windowed<>("3", new TimeWindow(0, 500)), 1L),
+ KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 1L),
+ KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 2L),
+ KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 1L),
+ KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 2L)
)));
}
@@ -652,15 +658,15 @@ public class KGroupedStreamImplTest {
public void shouldCountWindowed() {
final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
groupedStream.count(
- TimeWindows.of(500L),
- "aggregate-by-key-windowed")
- .toStream()
- .foreach(new ForeachAction<Windowed<String>, Long>() {
- @Override
- public void apply(final Windowed<String> key, final Long value) {
- results.add(KeyValue.pair(key, value));
- }
- });
+ TimeWindows.of(500L),
+ "aggregate-by-key-windowed")
+ .toStream()
+ .foreach(new ForeachAction<Windowed<String>, Long>() {
+ @Override
+ public void apply(final Windowed<String> key, final Long value) {
+ results.add(KeyValue.pair(key, value));
+ }
+ });
doCountWindowed(results);
}
@@ -670,14 +676,14 @@ public class KGroupedStreamImplTest {
public void shouldCountWindowedWithInternalStoreName() {
final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
groupedStream.count(
- TimeWindows.of(500L))
- .toStream()
- .foreach(new ForeachAction<Windowed<String>, Long>() {
- @Override
- public void apply(final Windowed<String> key, final Long value) {
- results.add(KeyValue.pair(key, value));
- }
- });
+ TimeWindows.of(500L))
+ .toStream()
+ .foreach(new ForeachAction<Windowed<String>, Long>() {
+ @Override
+ public void apply(final Windowed<String> key, final Long value) {
+ results.add(KeyValue.pair(key, value));
+ }
+ });
doCountWindowed(results);
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
index 5600023..83fee9b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
@@ -25,8 +25,8 @@ import org.apache.kafka.streams.kstream.JoinWindows;
import org.apache.kafka.streams.kstream.Joined;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
-import org.apache.kafka.test.KStreamTestDriver;
import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.KStreamTestDriver;
import org.apache.kafka.test.MockProcessorSupplier;
import org.apache.kafka.test.MockValueJoiner;
import org.apache.kafka.test.TestUtils;
@@ -35,7 +35,6 @@ import org.junit.Rule;
import org.junit.Test;
import java.io.File;
-import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
@@ -57,31 +56,32 @@ public class KStreamKStreamJoinTest {
private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde);
@Before
- public void setUp() throws IOException {
+ public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test");
}
@Test
public void testJoin() {
- StreamsBuilder builder = new StreamsBuilder();
+ final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KStream<Integer, String> stream1;
- KStream<Integer, String> stream2;
- KStream<Integer, String> joined;
- MockProcessorSupplier<Integer, String> processor;
+ final KStream<Integer, String> stream1;
+ final KStream<Integer, String> stream2;
+ final KStream<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed);
- joined = stream1.join(stream2,
- MockValueJoiner.TOSTRING_JOINER,
- JoinWindows.of(100),
- Joined.with(intSerde, stringSerde, stringSerde));
+ joined = stream1.join(
+ stream2,
+ MockValueJoiner.TOSTRING_JOINER,
+ JoinWindows.of(100),
+ Joined.with(intSerde, stringSerde, stringSerde));
joined.process(processor);
- Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
+ final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
@@ -119,7 +119,7 @@ public class KStreamKStreamJoinTest {
// --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
// w2 = { 0:Y0, 1:Y1 }
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
@@ -131,7 +131,7 @@ public class KStreamKStreamJoinTest {
// --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
// w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
@@ -143,7 +143,7 @@ public class KStreamKStreamJoinTest {
// --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 }
// w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
@@ -164,25 +164,26 @@ public class KStreamKStreamJoinTest {
@Test
public void testOuterJoin() {
- StreamsBuilder builder = new StreamsBuilder();
+ final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KStream<Integer, String> stream1;
- KStream<Integer, String> stream2;
- KStream<Integer, String> joined;
- MockProcessorSupplier<Integer, String> processor;
+ final KStream<Integer, String> stream1;
+ final KStream<Integer, String> stream2;
+ final KStream<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed);
- joined = stream1.outerJoin(stream2,
- MockValueJoiner.TOSTRING_JOINER,
- JoinWindows.of(100),
- Joined.with(intSerde, stringSerde, stringSerde));
+ joined = stream1.outerJoin(
+ stream2,
+ MockValueJoiner.TOSTRING_JOINER,
+ JoinWindows.of(100),
+ Joined.with(intSerde, stringSerde, stringSerde));
joined.process(processor);
- Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
+ final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
@@ -220,7 +221,7 @@ public class KStreamKStreamJoinTest {
// --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
// w2 = { 0:Y0, 1:Y1 }
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
@@ -232,7 +233,7 @@ public class KStreamKStreamJoinTest {
// --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
// w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
@@ -244,7 +245,7 @@ public class KStreamKStreamJoinTest {
// --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 }
// w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
@@ -267,26 +268,27 @@ public class KStreamKStreamJoinTest {
public void testWindowing() {
long time = 0L;
- StreamsBuilder builder = new StreamsBuilder();
+ final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KStream<Integer, String> stream1;
- KStream<Integer, String> stream2;
- KStream<Integer, String> joined;
- MockProcessorSupplier<Integer, String> processor;
+ final KStream<Integer, String> stream1;
+ final KStream<Integer, String> stream2;
+ final KStream<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed);
- joined = stream1.join(stream2,
- MockValueJoiner.TOSTRING_JOINER,
- JoinWindows.of(100),
- Joined.with(intSerde, stringSerde, stringSerde));
+ joined = stream1.join(
+ stream2,
+ MockValueJoiner.TOSTRING_JOINER,
+ JoinWindows.of(100),
+ Joined.with(intSerde, stringSerde, stringSerde));
joined.process(processor);
- Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
+ final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
@@ -334,35 +336,35 @@ public class KStreamKStreamJoinTest {
time = 1000 + 100L;
setRecordContext(time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
@@ -372,35 +374,35 @@ public class KStreamKStreamJoinTest {
time = 1000L - 100L - 1L;
setRecordContext(time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult();
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
@@ -420,35 +422,35 @@ public class KStreamKStreamJoinTest {
time = 2000L + 100L;
setRecordContext(time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
setRecordContext(++time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
setRecordContext(++time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3");
setRecordContext(++time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
processor.checkAndClearProcessResult("3:XX3+Y3");
setRecordContext(++time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
@@ -458,35 +460,35 @@ public class KStreamKStreamJoinTest {
time = 2000L - 100L - 1L;
setRecordContext(time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
processor.checkAndClearProcessResult();
setRecordContext(++time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
processor.checkAndClearProcessResult("0:XX0+Y0");
setRecordContext(++time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1");
setRecordContext(++time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2");
setRecordContext(++time, topic1);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
@@ -494,31 +496,32 @@ public class KStreamKStreamJoinTest {
}
@Test
- public void testAsymetricWindowingAfter() {
+ public void testAsymmetricWindowingAfter() {
long time = 1000L;
- StreamsBuilder builder = new StreamsBuilder();
+ final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KStream<Integer, String> stream1;
- KStream<Integer, String> stream2;
- KStream<Integer, String> joined;
- MockProcessorSupplier<Integer, String> processor;
+ final KStream<Integer, String> stream1;
+ final KStream<Integer, String> stream2;
+ final KStream<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed);
- joined = stream1.join(stream2,
- MockValueJoiner.TOSTRING_JOINER,
- JoinWindows.of(0).after(100),
- Joined.with(intSerde,
- stringSerde,
- stringSerde));
+ joined = stream1.join(
+ stream2,
+ MockValueJoiner.TOSTRING_JOINER,
+ JoinWindows.of(0).after(100),
+ Joined.with(intSerde,
+ stringSerde,
+ stringSerde));
joined.process(processor);
- Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
+ final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
@@ -535,35 +538,35 @@ public class KStreamKStreamJoinTest {
time = 1000L - 1L;
setRecordContext(time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult();
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
@@ -572,35 +575,35 @@ public class KStreamKStreamJoinTest {
time = 1000 + 100L;
setRecordContext(time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
@@ -608,29 +611,30 @@ public class KStreamKStreamJoinTest {
}
@Test
- public void testAsymetricWindowingBefore() {
+ public void testAsymmetricWindowingBefore() {
long time = 1000L;
- StreamsBuilder builder = new StreamsBuilder();
+ final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KStream<Integer, String> stream1;
- KStream<Integer, String> stream2;
- KStream<Integer, String> joined;
- MockProcessorSupplier<Integer, String> processor;
+ final KStream<Integer, String> stream1;
+ final KStream<Integer, String> stream2;
+ final KStream<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed);
- joined = stream1.join(stream2,
- MockValueJoiner.TOSTRING_JOINER,
- JoinWindows.of(0).before(100),
- Joined.with(intSerde, stringSerde, stringSerde));
+ joined = stream1.join(
+ stream2,
+ MockValueJoiner.TOSTRING_JOINER,
+ JoinWindows.of(0).before(100),
+ Joined.with(intSerde, stringSerde, stringSerde));
joined.process(processor);
- Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
+ final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
@@ -647,35 +651,35 @@ public class KStreamKStreamJoinTest {
time = 1000L - 100L - 1L;
setRecordContext(time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult();
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
@@ -683,35 +687,35 @@ public class KStreamKStreamJoinTest {
time = 1000L;
setRecordContext(time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
processor.checkAndClearProcessResult("3:X3+YY3");
setRecordContext(++time, topic2);
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java
index a23e787..851808c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java
@@ -32,7 +32,6 @@ import org.junit.Rule;
import org.junit.Test;
import java.io.File;
-import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
@@ -49,14 +48,13 @@ public class KStreamKTableJoinTest {
final private Serde<String> stringSerde = Serdes.String();
@Rule
public final KStreamTestDriver driver = new KStreamTestDriver();
- private File stateDir = null;
private MockProcessorSupplier<Integer, String> processor;
private final int[] expectedKeys = {0, 1, 2, 3};
private StreamsBuilder builder;
@Before
- public void setUp() throws IOException {
- stateDir = TestUtils.tempDirectory("kafka-test");
+ public void setUp() {
+ final File stateDir = TestUtils.tempDirectory("kafka-test");
builder = new StreamsBuilder();
@@ -86,8 +84,8 @@ public class KStreamKTableJoinTest {
}
}
- private void pushNullValueToTable(final int messageCount) {
- for (int i = 0; i < messageCount; i++) {
+ private void pushNullValueToTable() {
+ for (int i = 0; i < 2; i++) {
driver.process(tableTopic, expectedKeys[i], null);
}
}
@@ -174,7 +172,7 @@ public class KStreamKTableJoinTest {
// push two items with null to the table as deletes. this should not produce any item.
- pushNullValueToTable(2);
+ pushNullValueToTable();
processor.checkAndClearProcessResult();
// push all four items to the primary stream. this should produce two items.
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
index 212c48d..2acf859 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
@@ -28,8 +28,8 @@ import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
import org.apache.kafka.streams.state.KeyValueIterator;
-import org.apache.kafka.streams.state.internals.RocksDBSessionStoreSupplier;
import org.apache.kafka.streams.state.SessionStore;
+import org.apache.kafka.streams.state.internals.RocksDBSessionStoreSupplier;
import org.apache.kafka.streams.state.internals.ThreadCache;
import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.NoOpRecordCollector;
@@ -72,14 +72,15 @@ public class KStreamSessionWindowAggregateProcessorTest {
}
};
private final KStreamSessionWindowAggregate<String, String, Long> sessionAggregator =
- new KStreamSessionWindowAggregate<>(SessionWindows.with(GAP_MS).until(3 * GAP_MS),
- STORE_NAME,
- initializer,
- aggregator,
- sessionMerger);
+ new KStreamSessionWindowAggregate<>(
+ SessionWindows.with(GAP_MS).until(3 * GAP_MS),
+ STORE_NAME,
+ initializer,
+ aggregator,
+ sessionMerger);
private final List<KeyValue> results = new ArrayList<>();
- private Processor<String, String> processor = sessionAggregator.get();
+ private final Processor<String, String> processor = sessionAggregator.get();
private SessionStore<String, Long> sessionStore;
private InternalMockProcessorContext context;
@@ -101,14 +102,15 @@ public class KStreamSessionWindowAggregateProcessorTest {
private void initStore(final boolean enableCaching) {
final RocksDBSessionStoreSupplier<String, Long> supplier =
- new RocksDBSessionStoreSupplier<>(STORE_NAME,
- GAP_MS * 3,
- Serdes.String(),
- Serdes.Long(),
- false,
- Collections.<String, String>emptyMap(),
- enableCaching);
- sessionStore = (SessionStore<String, Long>) supplier.get();
+ new RocksDBSessionStoreSupplier<>(
+ STORE_NAME,
+ GAP_MS * 3,
+ Serdes.String(),
+ Serdes.Long(),
+ false,
+ Collections.<String, String>emptyMap(),
+ enableCaching);
+ sessionStore = supplier.get();
sessionStore.init(context, sessionStore);
}
@@ -180,12 +182,14 @@ public class KStreamSessionWindowAggregateProcessorTest {
processor.process(sessionId, "third");
sessionStore.flush();
- assertEquals(Arrays.asList(
+ assertEquals(
+ Arrays.asList(
KeyValue.pair(new Windowed<>(sessionId, new SessionWindow(0, 0)), new Change<>(1L, null)),
KeyValue.pair(new Windowed<>(sessionId, new SessionWindow(GAP_MS + 1, GAP_MS + 1)), new Change<>(2L, null)),
KeyValue.pair(new Windowed<>(sessionId, new SessionWindow(time, time)), new Change<>(3L, null))
-
- ), results);
+ ),
+ results
+ );
}
@@ -226,15 +230,18 @@ public class KStreamSessionWindowAggregateProcessorTest {
sessionStore.flush();
- assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)),
- KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), new Change<>(1L, null)),
- KeyValue.pair(new Windowed<>("c", new SessionWindow(0, 0)), new Change<>(1L, null)),
- KeyValue.pair(new Windowed<>("d", new SessionWindow(0, GAP_MS / 2)), new Change<>(2L, null)),
- KeyValue.pair(new Windowed<>("b", new SessionWindow(GAP_MS + 1, GAP_MS + 1)), new Change<>(1L, null)),
- KeyValue.pair(new Windowed<>("a", new SessionWindow(GAP_MS + 1, GAP_MS + 1 + GAP_MS / 2)), new Change<>(2L, null)),
- KeyValue.pair(new Windowed<>("c", new SessionWindow(GAP_MS + 1 + GAP_MS / 2, GAP_MS + 1 + GAP_MS / 2)), new Change<>(1L, null))
- ),
- results);
+ assertEquals(
+ Arrays.asList(
+ KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)),
+ KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), new Change<>(1L, null)),
+ KeyValue.pair(new Windowed<>("c", new SessionWindow(0, 0)), new Change<>(1L, null)),
+ KeyValue.pair(new Windowed<>("d", new SessionWindow(0, GAP_MS / 2)), new Change<>(2L, null)),
+ KeyValue.pair(new Windowed<>("b", new SessionWindow(GAP_MS + 1, GAP_MS + 1)), new Change<>(1L, null)),
+ KeyValue.pair(new Windowed<>("a", new SessionWindow(GAP_MS + 1, GAP_MS + 1 + GAP_MS / 2)), new Change<>(2L, null)),
+ KeyValue.pair(new Windowed<>("c", new SessionWindow(GAP_MS + 1 + GAP_MS / 2, GAP_MS + 1 + GAP_MS / 2)), new Change<>(1L, null))
+ ),
+ results
+ );
}
@@ -263,9 +270,14 @@ public class KStreamSessionWindowAggregateProcessorTest {
processor.process("b", "1");
processor.process("c", "1");
- assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)),
- KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), new Change<>(1L, null)),
- KeyValue.pair(new Windowed<>("c", new SessionWindow(0, 0)), new Change<>(1L, null))), results);
+ assertEquals(
+ Arrays.asList(
+ KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)),
+ KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), new Change<>(1L, null)),
+ KeyValue.pair(new Windowed<>("c", new SessionWindow(0, 0)), new Change<>(1L, null))
+ ),
+ results
+ );
}
@Test
@@ -277,9 +289,14 @@ public class KStreamSessionWindowAggregateProcessorTest {
processor.process("a", "1");
context.setTime(5);
processor.process("a", "1");
- assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)),
- KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(null, null)),
- KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 5)), new Change<>(2L, null))), results);
+ assertEquals(
+ Arrays.asList(
+ KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)),
+ KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(null, null)),
+ KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 5)), new Change<>(2L, null))
+ ),
+ results
+ );
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
index d3a74e3..3d107c2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
@@ -21,17 +21,16 @@ import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.Consumed;
import org.apache.kafka.streams.StreamsBuilder;
-import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Serialized;
import org.apache.kafka.streams.kstream.TimeWindows;
import org.apache.kafka.streams.kstream.ValueJoiner;
import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.KStreamTestDriver;
import org.apache.kafka.test.MockAggregator;
import org.apache.kafka.test.MockInitializer;
-import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.MockProcessorSupplier;
import org.apache.kafka.test.TestUtils;
import org.junit.Before;
@@ -39,7 +38,6 @@ import org.junit.Rule;
import org.junit.Test;
import java.io.File;
-import java.io.IOException;
import static org.junit.Assert.assertEquals;
@@ -51,24 +49,21 @@ public class KStreamWindowAggregateTest {
public final KStreamTestDriver driver = new KStreamTestDriver();
@Before
- public void setUp() throws IOException {
+ public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test");
}
@Test
public void testAggBasic() {
final StreamsBuilder builder = new StreamsBuilder();
- String topic1 = "topic1";
+ final String topic1 = "topic1";
- KStream<String, String> stream1 = builder.stream(topic1, Consumed.with(strSerde, strSerde));
- KTable<Windowed<String>, String> table2 =
- stream1.groupByKey(Serialized.with(strSerde, strSerde))
- .aggregate(MockInitializer.STRING_INIT,
- MockAggregator.TOSTRING_ADDER,
- TimeWindows.of(10).advanceBy(5),
- strSerde, "topic1-Canonized");
+ final KTable<Windowed<String>, String> table2 = builder
+ .stream(topic1, Consumed.with(strSerde, strSerde))
+ .groupByKey(Serialized.with(strSerde, strSerde))
+ .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10).advanceBy(5), strSerde, "topic1-Canonized");
- MockProcessorSupplier<Windowed<String>, String> proc2 = new MockProcessorSupplier<>();
+ final MockProcessorSupplier<Windowed<String>, String> proc2 = new MockProcessorSupplier<>();
table2.toStream().process(proc2);
driver.setUp(builder, stateDir);
@@ -122,7 +117,8 @@ public class KStreamWindowAggregateTest {
driver.flushState();
- assertEquals(Utils.mkList(
+ assertEquals(
+ Utils.mkList(
"[A@0/10]:0+1",
"[B@0/10]:0+2",
"[C@0/10]:0+3",
@@ -130,55 +126,51 @@ public class KStreamWindowAggregateTest {
"[A@0/10]:0+1+1",
"[A@0/10]:0+1+1+1", "[A@5/15]:0+1",
- "[B@0/10]:0+2+2", "[B@5/15]:0+2",
- "[D@0/10]:0+4+4", "[D@5/15]:0+4",
+ "[B@0/10]:0+2+2", "[B@5/15]:0+2",
+ "[D@0/10]:0+4+4", "[D@5/15]:0+4",
"[B@0/10]:0+2+2+2", "[B@5/15]:0+2+2",
- "[C@0/10]:0+3+3", "[C@5/15]:0+3",
+ "[C@0/10]:0+3+3", "[C@5/15]:0+3",
- "[A@5/15]:0+1+1", "[A@10/20]:0+1",
- "[B@5/15]:0+2+2+2", "[B@10/20]:0+2",
- "[D@5/15]:0+4+4", "[D@10/20]:0+4",
+ "[A@5/15]:0+1+1", "[A@10/20]:0+1",
+ "[B@5/15]:0+2+2+2", "[B@10/20]:0+2",
+ "[D@5/15]:0+4+4", "[D@10/20]:0+4",
"[B@5/15]:0+2+2+2+2", "[B@10/20]:0+2+2",
- "[C@5/15]:0+3+3", "[C@10/20]:0+3"), proc2.processed);
+ "[C@5/15]:0+3+3", "[C@10/20]:0+3"
+ ),
+ proc2.processed
+ );
}
- private void setRecordContext(final long time, final String topic) {
+ private void setRecordContext(final long time, @SuppressWarnings("SameParameterValue") final String topic) {
((InternalMockProcessorContext) driver.context()).setRecordContext(new ProcessorRecordContext(time, 0, 0, topic));
}
@Test
public void testJoin() {
final StreamsBuilder builder = new StreamsBuilder();
- String topic1 = "topic1";
- String topic2 = "topic2";
-
- KStream<String, String> stream1 = builder.stream(topic1, Consumed.with(strSerde, strSerde));
- KTable<Windowed<String>, String> table1 =
- stream1.groupByKey(Serialized.with(strSerde, strSerde))
- .aggregate(MockInitializer.STRING_INIT,
- MockAggregator.TOSTRING_ADDER,
- TimeWindows.of(10).advanceBy(5),
- strSerde, "topic1-Canonized");
-
- MockProcessorSupplier<Windowed<String>, String> proc1 = new MockProcessorSupplier<>();
+ final String topic1 = "topic1";
+ final String topic2 = "topic2";
+
+ final KTable<Windowed<String>, String> table1 = builder
+ .stream(topic1, Consumed.with(strSerde, strSerde))
+ .groupByKey(Serialized.with(strSerde, strSerde))
+ .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10).advanceBy(5), strSerde, "topic1-Canonized");
+
+ final MockProcessorSupplier<Windowed<String>, String> proc1 = new MockProcessorSupplier<>();
table1.toStream().process(proc1);
- KStream<String, String> stream2 = builder.stream(topic2, Consumed.with(strSerde, strSerde));
- KTable<Windowed<String>, String> table2 =
- stream2.groupByKey(Serialized.with(strSerde, strSerde))
- .aggregate(MockInitializer.STRING_INIT,
- MockAggregator.TOSTRING_ADDER,
- TimeWindows.of(10).advanceBy(5),
- strSerde, "topic2-Canonized");
+ final KTable<Windowed<String>, String> table2 = builder
+ .stream(topic2, Consumed.with(strSerde, strSerde)).groupByKey(Serialized.with(strSerde, strSerde))
+ .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10).advanceBy(5), strSerde, "topic2-Canonized");
- MockProcessorSupplier<Windowed<String>, String> proc2 = new MockProcessorSupplier<>();
+ final MockProcessorSupplier<Windowed<String>, String> proc2 = new MockProcessorSupplier<>();
table2.toStream().process(proc2);
- MockProcessorSupplier<Windowed<String>, String> proc3 = new MockProcessorSupplier<>();
+ final MockProcessorSupplier<Windowed<String>, String> proc3 = new MockProcessorSupplier<>();
table1.join(table2, new ValueJoiner<String, String, String>() {
@Override
- public String apply(String p1, String p2) {
+ public String apply(final String p1, final String p2) {
return p1 + "%" + p2;
}
}).toStream().process(proc3);
@@ -202,11 +194,11 @@ public class KStreamWindowAggregateTest {
driver.flushState();
proc1.checkAndClearProcessResult(
- "[A@0/10]:0+1",
- "[B@0/10]:0+2",
- "[C@0/10]:0+3",
- "[D@0/10]:0+4",
- "[A@0/10]:0+1+1"
+ "[A@0/10]:0+1",
+ "[B@0/10]:0+2",
+ "[C@0/10]:0+3",
+ "[D@0/10]:0+4",
+ "[A@0/10]:0+1+1"
);
proc2.checkAndClearProcessResult();
proc3.checkAndClearProcessResult();
@@ -228,11 +220,11 @@ public class KStreamWindowAggregateTest {
driver.flushState();
proc1.checkAndClearProcessResult(
- "[A@0/10]:0+1+1+1", "[A@5/15]:0+1",
- "[B@0/10]:0+2+2", "[B@5/15]:0+2",
- "[D@0/10]:0+4+4", "[D@5/15]:0+4",
- "[B@0/10]:0+2+2+2", "[B@5/15]:0+2+2",
- "[C@0/10]:0+3+3", "[C@5/15]:0+3"
+ "[A@0/10]:0+1+1+1", "[A@5/15]:0+1",
+ "[B@0/10]:0+2+2", "[B@5/15]:0+2",
+ "[D@0/10]:0+4+4", "[D@5/15]:0+4",
+ "[B@0/10]:0+2+2+2", "[B@5/15]:0+2+2",
+ "[C@0/10]:0+3+3", "[C@5/15]:0+3"
);
proc2.checkAndClearProcessResult();
proc3.checkAndClearProcessResult();
@@ -255,18 +247,18 @@ public class KStreamWindowAggregateTest {
proc1.checkAndClearProcessResult();
proc2.checkAndClearProcessResult(
- "[A@0/10]:0+a",
- "[B@0/10]:0+b",
- "[C@0/10]:0+c",
- "[D@0/10]:0+d",
- "[A@0/10]:0+a+a"
+ "[A@0/10]:0+a",
+ "[B@0/10]:0+b",
+ "[C@0/10]:0+c",
+ "[D@0/10]:0+d",
+ "[A@0/10]:0+a+a"
);
proc3.checkAndClearProcessResult(
- "[A@0/10]:0+1+1+1%0+a",
- "[B@0/10]:0+2+2+2%0+b",
- "[C@0/10]:0+3+3%0+c",
- "[D@0/10]:0+4+4%0+d",
- "[A@0/10]:0+1+1+1%0+a+a");
+ "[A@0/10]:0+1+1+1%0+a",
+ "[B@0/10]:0+2+2+2%0+b",
+ "[C@0/10]:0+3+3%0+c",
+ "[D@0/10]:0+4+4%0+d",
+ "[A@0/10]:0+1+1+1%0+a+a");
setRecordContext(5, topic1);
driver.process(topic2, "A", "a");
@@ -285,18 +277,18 @@ public class KStreamWindowAggregateTest {
driver.flushState();
proc1.checkAndClearProcessResult();
proc2.checkAndClearProcessResult(
- "[A@0/10]:0+a+a+a", "[A@5/15]:0+a",
- "[B@0/10]:0+b+b", "[B@5/15]:0+b",
- "[D@0/10]:0+d+d", "[D@5/15]:0+d",
- "[B@0/10]:0+b+b+b", "[B@5/15]:0+b+b",
- "[C@0/10]:0+c+c", "[C@5/15]:0+c"
+ "[A@0/10]:0+a+a+a", "[A@5/15]:0+a",
+ "[B@0/10]:0+b+b", "[B@5/15]:0+b",
+ "[D@0/10]:0+d+d", "[D@5/15]:0+d",
+ "[B@0/10]:0+b+b+b", "[B@5/15]:0+b+b",
+ "[C@0/10]:0+c+c", "[C@5/15]:0+c"
);
proc3.checkAndClearProcessResult(
- "[A@0/10]:0+1+1+1%0+a+a+a", "[A@5/15]:0+1%0+a",
- "[B@0/10]:0+2+2+2%0+b+b", "[B@5/15]:0+2+2%0+b",
- "[D@0/10]:0+4+4%0+d+d", "[D@5/15]:0+4%0+d",
- "[B@0/10]:0+2+2+2%0+b+b+b", "[B@5/15]:0+2+2%0+b+b",
- "[C@0/10]:0+3+3%0+c+c", "[C@5/15]:0+3%0+c"
+ "[A@0/10]:0+1+1+1%0+a+a+a", "[A@5/15]:0+1%0+a",
+ "[B@0/10]:0+2+2+2%0+b+b", "[B@5/15]:0+2+2%0+b",
+ "[D@0/10]:0+4+4%0+d+d", "[D@5/15]:0+4%0+d",
+ "[B@0/10]:0+2+2+2%0+b+b+b", "[B@5/15]:0+2+2%0+b+b",
+ "[C@0/10]:0+3+3%0+c+c", "[C@5/15]:0+3%0+c"
);
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java
index b890e2f..98d3e52 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java
@@ -101,7 +101,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce two items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -110,7 +110,7 @@ public class KTableKTableInnerJoinTest {
checkJoinedValues(getter, kv(0, "XX0+Y0"), kv(1, "XX1+Y1"));
// push all items to the other stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
driver.flushState();
@@ -120,7 +120,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -140,7 +140,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce two items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -173,7 +173,6 @@ public class KTableKTableInnerJoinTest {
doTestJoin(builder, expectedKeys, processor, joined);
}
-
@Test
public void testQueryableJoin() {
final StreamsBuilder builder = new StreamsBuilder();
@@ -235,7 +234,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce two items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -243,7 +242,7 @@ public class KTableKTableInnerJoinTest {
proc.checkAndClearProcessResult("0:(XX0+Y0<-null)", "1:(XX1+Y1<-null)");
// push all items to the other stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
driver.flushState();
@@ -251,7 +250,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -267,7 +266,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce two items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -338,13 +337,14 @@ public class KTableKTableInnerJoinTest {
}
- private KeyValue<Integer, String> kv(Integer key, String value) {
+ private KeyValue<Integer, String> kv(final Integer key, final String value) {
return new KeyValue<>(key, value);
}
- private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) {
- for (KeyValue<Integer, String> kv : expected) {
- String value = getter.get(kv.key);
+ @SafeVarargs
+ private final void checkJoinedValues(final KTableValueGetter<Integer, String> getter, final KeyValue<Integer, String>... expected) {
+ for (final KeyValue<Integer, String> kv : expected) {
+ final String value = getter.get(kv.key);
if (kv.value == null) {
assertNull(value);
} else {
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
index ca0c81c..7261ae0 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
@@ -36,7 +36,6 @@ import org.junit.Rule;
import org.junit.Test;
import java.io.File;
-import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
@@ -53,8 +52,6 @@ public class KTableKTableLeftJoinTest {
final private String topic1 = "topic1";
final private String topic2 = "topic2";
- final private String storeName1 = "store-name-1";
- final private String storeName2 = "store-name-2";
final private Serde<Integer> intSerde = Serdes.Integer();
final private Serde<String> stringSerde = Serdes.String();
@@ -64,7 +61,7 @@ public class KTableKTableLeftJoinTest {
private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde);
@Before
- public void setUp() throws IOException {
+ public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test");
}
@@ -74,24 +71,24 @@ public class KTableKTableLeftJoinTest {
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KTable<Integer, String> table1 = builder.table(topic1, consumed);
- KTable<Integer, String> table2 = builder.table(topic2, consumed);
- KTable<Integer, String> joined = table1.leftJoin(table2, MockValueJoiner.TOSTRING_JOINER);
- MockProcessorSupplier<Integer, String> processor;
+ final KTable<Integer, String> table1 = builder.table(topic1, consumed);
+ final KTable<Integer, String> table2 = builder.table(topic2, consumed);
+ final KTable<Integer, String> joined = table1.leftJoin(table2, MockValueJoiner.TOSTRING_JOINER);
+ final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>();
joined.toStream().process(processor);
- Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
+ final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
- KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
+ final KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
driver.setUp(builder, stateDir);
driver.setTime(0L);
- KTableValueGetter<Integer, String> getter = getterSupplier.get();
+ final KTableValueGetter<Integer, String> getter = getterSupplier.get();
getter.init(driver.context());
// push two items to the primary stream. the other table is empty
@@ -119,7 +116,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -127,7 +124,7 @@ public class KTableKTableLeftJoinTest {
checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null"));
// push all items to the other stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
driver.flushState();
@@ -136,7 +133,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -154,7 +151,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -205,14 +202,14 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)");
// push all items to the other stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
driver.flushState();
@@ -220,7 +217,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -236,7 +233,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -249,10 +246,10 @@ public class KTableKTableLeftJoinTest {
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KTable<Integer, String> table1;
- KTable<Integer, String> table2;
- KTable<Integer, String> joined;
- MockProcessorSupplier<Integer, String> proc;
+ final KTable<Integer, String> table1;
+ final KTable<Integer, String> table2;
+ final KTable<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> proc;
table1 = builder.table(topic1, consumed);
table2 = builder.table(topic2, consumed);
@@ -288,14 +285,14 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)");
// push all items to the other stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
driver.flushState();
@@ -303,7 +300,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -319,7 +316,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -344,13 +341,18 @@ public class KTableKTableLeftJoinTest {
final StreamsBuilder builder = new StreamsBuilder();
final Consumed<Long, String> consumed = Consumed.with(Serdes.Long(), Serdes.String());
- final KTable<Long, String> aggTable = builder.table(agg, consumed)
- .groupBy(new KeyValueMapper<Long, String, KeyValue<Long, String>>() {
+ final KTable<Long, String> aggTable = builder
+ .table(agg, consumed)
+ .groupBy(
+ new KeyValueMapper<Long, String, KeyValue<Long, String>>() {
@Override
public KeyValue<Long, String> apply(final Long key, final String value) {
return new KeyValue<>(key, value);
}
- }, Serialized.with(Serdes.Long(), Serdes.String())).reduce(MockReducer.STRING_ADDER, MockReducer.STRING_ADDER, "agg-store");
+ },
+ Serialized.with(Serdes.Long(), Serdes.String())
+ )
+ .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_ADDER, "agg-store");
final KTable<Long, String> one = builder.table(tableOne, consumed);
final KTable<Long, String> two = builder.table(tableTwo, consumed);
@@ -369,23 +371,26 @@ public class KTableKTableLeftJoinTest {
final KTable<Long, String> eight = six.leftJoin(seven, MockValueJoiner.TOSTRING_JOINER);
- aggTable.leftJoin(one, MockValueJoiner.TOSTRING_JOINER)
- .leftJoin(two, MockValueJoiner.TOSTRING_JOINER)
- .leftJoin(three, MockValueJoiner.TOSTRING_JOINER)
- .leftJoin(four, MockValueJoiner.TOSTRING_JOINER)
- .leftJoin(five, MockValueJoiner.TOSTRING_JOINER)
- .leftJoin(eight, MockValueJoiner.TOSTRING_JOINER)
- .mapValues(mapper);
+ aggTable
+ .leftJoin(one, MockValueJoiner.TOSTRING_JOINER)
+ .leftJoin(two, MockValueJoiner.TOSTRING_JOINER)
+ .leftJoin(three, MockValueJoiner.TOSTRING_JOINER)
+ .leftJoin(four, MockValueJoiner.TOSTRING_JOINER)
+ .leftJoin(five, MockValueJoiner.TOSTRING_JOINER)
+ .leftJoin(eight, MockValueJoiner.TOSTRING_JOINER)
+ .mapValues(mapper);
driver.setUp(builder, stateDir, 250);
- final String[] values = {"a", "AA", "BBB", "CCCC", "DD", "EEEEEEEE", "F", "GGGGGGGGGGGGGGG", "HHH", "IIIIIIIIII",
- "J", "KK", "LLLL", "MMMMMMMMMMMMMMMMMMMMMM", "NNNNN", "O", "P", "QQQQQ", "R", "SSSS",
- "T", "UU", "VVVVVVVVVVVVVVVVVVV"};
+ final String[] values = {
+ "a", "AA", "BBB", "CCCC", "DD", "EEEEEEEE", "F", "GGGGGGGGGGGGGGG", "HHH", "IIIIIIIIII",
+ "J", "KK", "LLLL", "MMMMMMMMMMMMMMMMMMMMMM", "NNNNN", "O", "P", "QQQQQ", "R", "SSSS",
+ "T", "UU", "VVVVVVVVVVVVVVVVVVV"
+ };
final Random random = new Random();
for (int i = 0; i < 1000; i++) {
- for (String input : inputs) {
+ for (final String input : inputs) {
final Long key = (long) random.nextInt(1000);
final String value = values[random.nextInt(values.length)];
driver.process(input, key, value);
@@ -393,13 +398,14 @@ public class KTableKTableLeftJoinTest {
}
}
- private KeyValue<Integer, String> kv(Integer key, String value) {
+ private KeyValue<Integer, String> kv(final Integer key, final String value) {
return new KeyValue<>(key, value);
}
- private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) {
- for (KeyValue<Integer, String> kv : expected) {
- String value = getter.get(kv.key);
+ @SafeVarargs
+ private final void checkJoinedValues(final KTableValueGetter<Integer, String> getter, final KeyValue<Integer, String>... expected) {
+ for (final KeyValue<Integer, String> kv : expected) {
+ final String value = getter.get(kv.key);
if (kv.value == null) {
assertNull(value);
} else {
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
index d6ab6136..e094591 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
@@ -32,7 +32,6 @@ import org.junit.Rule;
import org.junit.Test;
import java.io.File;
-import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
@@ -47,8 +46,6 @@ public class KTableKTableOuterJoinTest {
final private String topic1 = "topic1";
final private String topic2 = "topic2";
- final private String storeName1 = "store-name-1";
- final private String storeName2 = "store-name-2";
final private Serde<Integer> intSerde = Serdes.Integer();
final private Serde<String> stringSerde = Serdes.String();
@@ -58,20 +55,20 @@ public class KTableKTableOuterJoinTest {
private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde);
@Before
- public void setUp() throws IOException {
+ public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test");
}
@Test
public void testJoin() {
- StreamsBuilder builder = new StreamsBuilder();
+ final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KTable<Integer, String> table1;
- KTable<Integer, String> table2;
- KTable<Integer, String> joined;
- MockProcessorSupplier<Integer, String> processor;
+ final KTable<Integer, String> table1;
+ final KTable<Integer, String> table2;
+ final KTable<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>();
table1 = builder.table(topic1, consumed);
@@ -79,16 +76,16 @@ public class KTableKTableOuterJoinTest {
joined = table1.outerJoin(table2, MockValueJoiner.TOSTRING_JOINER);
joined.toStream().process(processor);
- Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
+ final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
- KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
+ final KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
driver.setUp(builder, stateDir);
- KTableValueGetter<Integer, String> getter = getterSupplier.get();
+ final KTableValueGetter<Integer, String> getter = getterSupplier.get();
getter.init(driver.context());
// push two items to the primary stream. the other table is empty
@@ -115,7 +112,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -123,7 +120,7 @@ public class KTableKTableOuterJoinTest {
checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null"));
// push all items to the other stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
driver.flushState();
@@ -132,7 +129,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -150,7 +147,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -173,10 +170,10 @@ public class KTableKTableOuterJoinTest {
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KTable<Integer, String> table1;
- KTable<Integer, String> table2;
- KTable<Integer, String> joined;
- MockProcessorSupplier<Integer, String> proc;
+ final KTable<Integer, String> table1;
+ final KTable<Integer, String> table2;
+ final KTable<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> proc;
table1 = builder.table(topic1, consumed);
table2 = builder.table(topic2, consumed);
@@ -209,14 +206,14 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)");
// push all items to the other stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
driver.flushState();
@@ -224,7 +221,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -240,7 +237,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -261,10 +258,10 @@ public class KTableKTableOuterJoinTest {
final int[] expectedKeys = new int[]{0, 1, 2, 3};
- KTable<Integer, String> table1;
- KTable<Integer, String> table2;
- KTable<Integer, String> joined;
- MockProcessorSupplier<Integer, String> proc;
+ final KTable<Integer, String> table1;
+ final KTable<Integer, String> table2;
+ final KTable<Integer, String> joined;
+ final MockProcessorSupplier<Integer, String> proc;
table1 = builder.table(topic1, consumed);
table2 = builder.table(topic2, consumed);
@@ -299,14 +296,14 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)");
// push all items to the other stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey);
}
driver.flushState();
@@ -314,7 +311,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey);
}
driver.flushState();
@@ -330,7 +327,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items.
- for (int expectedKey : expectedKeys) {
+ for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey);
}
driver.flushState();
@@ -345,13 +342,14 @@ public class KTableKTableOuterJoinTest {
proc.checkAndClearProcessResult("1:(null<-XX1+null)", "2:(null+YY2<-XX2+YY2)");
}
- private KeyValue<Integer, String> kv(Integer key, String value) {
+ private KeyValue<Integer, String> kv(final Integer key, final String value) {
return new KeyValue<>(key, value);
}
- private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) {
- for (KeyValue<Integer, String> kv : expected) {
- String value = getter.get(kv.key);
+ @SafeVarargs
+ private final void checkJoinedValues(final KTableValueGetter<Integer, String> getter, final KeyValue<Integer, String>... expected) {
+ for (final KeyValue<Integer, String> kv : expected) {
+ final String value = getter.get(kv.key);
if (kv.value == null) {
assertNull(value);
} else {
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
index 35a3dbd..6ee1b1b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
@@ -53,11 +53,11 @@ public class KTableSourceTest {
public void testKTable() {
final StreamsBuilder builder = new StreamsBuilder();
- String topic1 = "topic1";
+ final String topic1 = "topic1";
- KTable<String, Integer> table1 = builder.table(topic1, Consumed.with(stringSerde, intSerde));
+ final KTable<String, Integer> table1 = builder.table(topic1, Consumed.with(stringSerde, intSerde));
- MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
+ final MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
table1.toStream().process(proc1);
driver.setUp(builder, stateDir);
@@ -77,14 +77,14 @@ public class KTableSourceTest {
public void testValueGetter() {
final StreamsBuilder builder = new StreamsBuilder();
- String topic1 = "topic1";
+ final String topic1 = "topic1";
- KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(topic1, stringConsumed);
+ final KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(topic1, stringConsumed);
- KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
+ final KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
driver.setUp(builder, stateDir);
- KTableValueGetter<String, String> getter1 = getterSupplier1.get();
+ final KTableValueGetter<String, String> getter1 = getterSupplier1.get();
getter1.init(driver.context());
driver.process(topic1, "A", "01");
@@ -121,11 +121,11 @@ public class KTableSourceTest {
public void testNotSendingOldValue() {
final StreamsBuilder builder = new StreamsBuilder();
- String topic1 = "topic1";
+ final String topic1 = "topic1";
- KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(topic1, stringConsumed);
+ final KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(topic1, stringConsumed);
- MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
+ final MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
builder.build().addProcessor("proc1", proc1, table1.name);
@@ -159,15 +159,15 @@ public class KTableSourceTest {
public void testSendingOldValue() {
final StreamsBuilder builder = new StreamsBuilder();
- String topic1 = "topic1";
+ final String topic1 = "topic1";
- KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(topic1, stringConsumed);
+ final KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(topic1, stringConsumed);
table1.enableSendingOldValues();
assertTrue(table1.sendingOldValueEnabled());
- MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
+ final MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
builder.build().addProcessor("proc1", proc1, table1.name);
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
index 38a017d..924bc52 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
@@ -56,12 +56,14 @@ public class GlobalStateTaskTest {
private final String topic2 = "t2";
private final TopicPartition t1 = new TopicPartition(topic1, 1);
private final TopicPartition t2 = new TopicPartition(topic2, 1);
- private final MockSourceNode sourceOne = new MockSourceNode<>(new String[]{topic1},
- new StringDeserializer(),
- new StringDeserializer());;
- private final MockSourceNode sourceTwo = new MockSourceNode<>(new String[]{topic2},
- new IntegerDeserializer(),
- new IntegerDeserializer());
+ private final MockSourceNode sourceOne = new MockSourceNode<>(
+ new String[]{topic1},
+ new StringDeserializer(),
+ new StringDeserializer());
+ private final MockSourceNode sourceTwo = new MockSourceNode<>(
+ new String[]{topic2},
+ new IntegerDeserializer(),
+ new IntegerDeserializer());
private final MockProcessorNode processorOne = new MockProcessorNode<>(-1);
private final MockProcessorNode processorTwo = new MockProcessorNode<>(-1);
@@ -81,10 +83,11 @@ public class GlobalStateTaskTest {
final Map<String, String> storeToTopic = new HashMap<>();
storeToTopic.put("t1-store", topic1);
storeToTopic.put("t2-store", topic2);
- topology = ProcessorTopology.with(Utils.mkList(sourceOne, sourceTwo, processorOne, processorTwo),
- sourceByTopics,
- Collections.<StateStore>emptyList(),
- storeToTopic);
+ topology = ProcessorTopology.with(
+ Utils.mkList(sourceOne, sourceTwo, processorOne, processorTwo),
+ sourceByTopics,
+ Collections.<StateStore>emptyList(),
+ storeToTopic);
offsets.put(t1, 50L);
offsets.put(t2, 100L);
@@ -134,17 +137,18 @@ public class GlobalStateTaskTest {
private void maybeDeserialize(final GlobalStateUpdateTask globalStateTask,
final byte[] key,
final byte[] recordValue,
- boolean failExpected) {
- final ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(topic2, 1, 1,
- 0L, TimestampType.CREATE_TIME, 0L, 0, 0,
- key, recordValue);
+ final boolean failExpected) {
+ final ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
+ topic2, 1, 1, 0L, TimestampType.CREATE_TIME,
+ 0L, 0, 0, key, recordValue
+ );
globalStateTask.initialize();
try {
globalStateTask.update(record);
if (failExpected) {
fail("Should have failed to deserialize.");
}
- } catch (StreamsException e) {
+ } catch (final StreamsException e) {
if (!failExpected) {
fail("Shouldn't have failed to deserialize.");
}
@@ -153,7 +157,7 @@ public class GlobalStateTaskTest {
@Test
- public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() throws Exception {
+ public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() {
final byte[] key = new LongSerializer().serialize(topic2, 1L);
final byte[] recordValue = new IntegerSerializer().serialize(topic2, 10);
maybeDeserialize(globalStateTask, key, recordValue, true);
@@ -161,14 +165,14 @@ public class GlobalStateTaskTest {
@Test
- public void shouldThrowStreamsExceptionWhenValueDeserializationFails() throws Exception {
+ public void shouldThrowStreamsExceptionWhenValueDeserializationFails() {
final byte[] key = new IntegerSerializer().serialize(topic2, 1);
final byte[] recordValue = new LongSerializer().serialize(topic2, 10L);
maybeDeserialize(globalStateTask, key, recordValue, true);
}
@Test
- public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() throws Exception {
+ public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() {
final GlobalStateUpdateTask globalStateTask2 = new GlobalStateUpdateTask(
topology,
context,
@@ -182,7 +186,7 @@ public class GlobalStateTaskTest {
}
@Test
- public void shouldNotThrowStreamsExceptionWhenValueDeserializationFails() throws Exception {
+ public void shouldNotThrowStreamsExceptionWhenValueDeserializationFails() {
final GlobalStateUpdateTask globalStateTask2 = new GlobalStateUpdateTask(
topology,
context,
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java
index 62661d2..d374136 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java
@@ -16,12 +16,13 @@
*/
package org.apache.kafka.streams.processor.internals;
-import java.util.Collections;
import org.apache.kafka.common.metrics.Metrics;
+import java.util.Collections;
+
public class MockStreamsMetrics extends StreamsMetricsImpl {
- public MockStreamsMetrics(Metrics metrics) {
+ public MockStreamsMetrics(final Metrics metrics) {
super(metrics, "mock-stream-metrics",
Collections.<String, String>emptyMap());
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java
index 00a2c35..5a182de 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java
@@ -30,9 +30,10 @@ import org.apache.kafka.test.MockTimestampExtractor;
import org.junit.Test;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.List;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.junit.Assert.assertEquals;
public class PartitionGroupTest {
@@ -61,12 +62,7 @@ public class PartitionGroupTest {
private final byte[] recordValue = intSerializer.serialize(null, 10);
private final byte[] recordKey = intSerializer.serialize(null, 1);
- private final PartitionGroup group = new PartitionGroup(new HashMap<TopicPartition, RecordQueue>() {
- {
- put(partition1, queue1);
- put(partition2, queue2);
- }
- });
+ private final PartitionGroup group = new PartitionGroup(mkMap(mkEntry(partition1, queue1), mkEntry(partition2, queue2)));
@Test
public void testTimeTracking() {
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
index 0dea193..146f3ff 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
@@ -28,22 +28,22 @@ import org.apache.kafka.test.InternalMockProcessorContext;
import org.junit.Test;
import java.util.Collections;
-import java.util.Map;
import java.util.LinkedHashMap;
+import java.util.Map;
import static org.junit.Assert.assertNotNull;
public class ProcessorNodeTest {
@SuppressWarnings("unchecked")
- @Test (expected = StreamsException.class)
+ @Test(expected = StreamsException.class)
public void shouldThrowStreamsExceptionIfExceptionCaughtDuringInit() {
final ProcessorNode node = new ProcessorNode("name", new ExceptionalProcessor(), Collections.emptySet());
node.init(null);
}
@SuppressWarnings("unchecked")
- @Test (expected = StreamsException.class)
+ @Test(expected = StreamsException.class)
public void shouldThrowStreamsExceptionIfExceptionCaughtDuringClose() {
final ProcessorNode node = new ProcessorNode("name", new ExceptionalProcessor(), Collections.emptySet());
node.close();
@@ -71,7 +71,7 @@ public class ProcessorNodeTest {
}
}
- private static class NoOpProcessor implements Processor {
+ private static class NoOpProcessor implements Processor<Object, Object> {
@Override
public void init(final ProcessorContext context) {
@@ -93,15 +93,16 @@ public class ProcessorNodeTest {
}
}
- private void testSpecificMetrics(final Metrics metrics, final String groupName,
+ private void testSpecificMetrics(final Metrics metrics,
+ @SuppressWarnings("SameParameterValue") final String groupName,
final String opName,
final Map<String, String> metricTags) {
assertNotNull(metrics.metrics().get(metrics.metricName(opName + "-latency-avg", groupName,
- "The average latency of " + opName + " operation.", metricTags)));
+ "The average latency of " + opName + " operation.", metricTags)));
assertNotNull(metrics.metrics().get(metrics.metricName(opName + "-latency-max", groupName,
- "The max latency of " + opName + " operation.", metricTags)));
+ "The max latency of " + opName + " operation.", metricTags)));
assertNotNull(metrics.metrics().get(metrics.metricName(opName + "-rate", groupName,
- "The average number of occurrence of " + opName + " operation per second.", metricTags)));
+ "The average number of occurrence of " + opName + " operation per second.", metricTags)));
}
@@ -110,36 +111,40 @@ public class ProcessorNodeTest {
final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
final Metrics metrics = new Metrics();
- final InternalMockProcessorContext context = new InternalMockProcessorContext(anyStateSerde, new RecordCollectorImpl(null, null, new LogContext("processnode-test "), new DefaultProductionExceptionHandler()), metrics);
- final ProcessorNode node = new ProcessorNode("name", new NoOpProcessor(), Collections.emptySet());
+ final InternalMockProcessorContext context = new InternalMockProcessorContext(
+ anyStateSerde,
+ new RecordCollectorImpl(null, null, new LogContext("processnode-test "), new DefaultProductionExceptionHandler()),
+ metrics
+ );
+ final ProcessorNode<Object, Object> node = new ProcessorNode<>("name", new NoOpProcessor(), Collections.<String>emptySet());
node.init(context);
- String[] latencyOperations = {"process", "punctuate", "create", "destroy"};
- String throughputOperation = "forward";
- String groupName = "stream-processor-node-metrics";
+ final String[] latencyOperations = {"process", "punctuate", "create", "destroy"};
+ final String throughputOperation = "forward";
+ final String groupName = "stream-processor-node-metrics";
final Map<String, String> metricTags = new LinkedHashMap<>();
metricTags.put("processor-node-id", node.name());
metricTags.put("task-id", context.taskId().toString());
- for (String operation : latencyOperations) {
+ for (final String operation : latencyOperations) {
assertNotNull(metrics.getSensor(operation));
}
assertNotNull(metrics.getSensor(throughputOperation));
- for (String opName : latencyOperations) {
+ for (final String opName : latencyOperations) {
testSpecificMetrics(metrics, groupName, opName, metricTags);
}
assertNotNull(metrics.metrics().get(metrics.metricName(throughputOperation + "-rate", groupName,
- "The average number of occurrence of " + throughputOperation + " operation per second.", metricTags)));
+ "The average number of occurrence of " + throughputOperation + " operation per second.", metricTags)));
// test "all"
metricTags.put("processor-node-id", "all");
- for (String opName : latencyOperations) {
+ for (final String opName : latencyOperations) {
testSpecificMetrics(metrics, groupName, opName, metricTags);
}
assertNotNull(metrics.metrics().get(metrics.metricName(throughputOperation + "-rate", groupName,
- "The average number of occurrence of " + throughputOperation + " operation per second.", metricTags)));
+ "The average number of occurrence of " + throughputOperation + " operation per second.", metricTags)));
context.close();
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
index 39fe7ab..d877450 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
@@ -49,13 +49,13 @@ public class RecordCollectorTest {
private final LogContext logContext = new LogContext("test ");
private final List<PartitionInfo> infos = Arrays.asList(
- new PartitionInfo("topic1", 0, Node.noNode(), new Node[0], new Node[0]),
- new PartitionInfo("topic1", 1, Node.noNode(), new Node[0], new Node[0]),
- new PartitionInfo("topic1", 2, Node.noNode(), new Node[0], new Node[0])
+ new PartitionInfo("topic1", 0, Node.noNode(), new Node[0], new Node[0]),
+ new PartitionInfo("topic1", 1, Node.noNode(), new Node[0], new Node[0]),
+ new PartitionInfo("topic1", 2, Node.noNode(), new Node[0], new Node[0])
);
private final Cluster cluster = new Cluster("cluster", Collections.singletonList(Node.noNode()), infos,
- Collections.<String>emptySet(), Collections.<String>emptySet());
+ Collections.<String>emptySet(), Collections.<String>emptySet());
private final ByteArraySerializer byteArraySerializer = new ByteArraySerializer();
@@ -72,8 +72,10 @@ public class RecordCollectorTest {
public void testSpecificPartition() {
final RecordCollectorImpl collector = new RecordCollectorImpl(
- new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer),
- "RecordCollectorTest-TestSpecificPartition", new LogContext("RecordCollectorTest-TestSpecificPartition "), new DefaultProductionExceptionHandler());
+ new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer),
+ "RecordCollectorTest-TestSpecificPartition",
+ new LogContext("RecordCollectorTest-TestSpecificPartition "),
+ new DefaultProductionExceptionHandler());
collector.send("topic1", "999", "0", 0, null, stringSerializer, stringSerializer);
collector.send("topic1", "999", "0", 0, null, stringSerializer, stringSerializer);
@@ -104,8 +106,10 @@ public class RecordCollectorTest {
public void testStreamPartitioner() {
final RecordCollectorImpl collector = new RecordCollectorImpl(
- new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer),
- "RecordCollectorTest-TestStreamPartitioner", new LogContext("RecordCollectorTest-TestStreamPartitioner "), new DefaultProductionExceptionHandler());
+ new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer),
+ "RecordCollectorTest-TestStreamPartitioner",
+ new LogContext("RecordCollectorTest-TestStreamPartitioner "),
+ new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
collector.send("topic1", "9", "0", null, stringSerializer, stringSerializer, streamPartitioner);
@@ -130,15 +134,15 @@ public class RecordCollectorTest {
@Test(expected = StreamsException.class)
public void shouldThrowStreamsExceptionOnAnyExceptionButProducerFencedException() {
final RecordCollector collector = new RecordCollectorImpl(
- new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
- @Override
- public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
- throw new KafkaException();
- }
- },
- "test",
- logContext,
- new DefaultProductionExceptionHandler());
+ new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
+ @Override
+ public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
+ throw new KafkaException();
+ }
+ },
+ "test",
+ logContext,
+ new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
}
@@ -147,16 +151,16 @@ public class RecordCollectorTest {
@Test
public void shouldThrowStreamsExceptionOnSubsequentCallIfASendFailsWithDefaultExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl(
- new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
- @Override
- public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
- callback.onCompletion(null, new Exception());
- return null;
- }
- },
- "test",
- logContext,
- new DefaultProductionExceptionHandler());
+ new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
+ @Override
+ public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
+ callback.onCompletion(null, new Exception());
+ return null;
+ }
+ },
+ "test",
+ logContext,
+ new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
try {
@@ -169,16 +173,16 @@ public class RecordCollectorTest {
@Test
public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContinueExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl(
- new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
- @Override
- public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
- callback.onCompletion(null, new Exception());
- return null;
- }
- },
- "test",
- logContext,
- new AlwaysContinueProductionExceptionHandler());
+ new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
+ @Override
+ public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
+ callback.onCompletion(null, new Exception());
+ return null;
+ }
+ },
+ "test",
+ logContext,
+ new AlwaysContinueProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
@@ -188,16 +192,16 @@ public class RecordCollectorTest {
@Test
public void shouldThrowStreamsExceptionOnFlushIfASendFailedWithDefaultExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl(
- new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
- @Override
- public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
- callback.onCompletion(null, new Exception());
- return null;
- }
- },
- "test",
- logContext,
- new DefaultProductionExceptionHandler());
+ new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
+ @Override
+ public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
+ callback.onCompletion(null, new Exception());
+ return null;
+ }
+ },
+ "test",
+ logContext,
+ new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
try {
@@ -210,16 +214,16 @@ public class RecordCollectorTest {
@Test
public void shouldNotThrowStreamsExceptionOnFlushIfASendFailedWithContinueExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl(
- new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
- @Override
- public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
- callback.onCompletion(null, new Exception());
- return null;
- }
- },
- "test",
- logContext,
- new AlwaysContinueProductionExceptionHandler());
+ new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
+ @Override
+ public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
+ callback.onCompletion(null, new Exception());
+ return null;
+ }
+ },
+ "test",
+ logContext,
+ new AlwaysContinueProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
collector.flush();
@@ -229,16 +233,16 @@ public class RecordCollectorTest {
@Test
public void shouldThrowStreamsExceptionOnCloseIfASendFailedWithDefaultExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl(
- new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
- @Override
- public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
- callback.onCompletion(null, new Exception());
- return null;
- }
- },
- "test",
- logContext,
- new DefaultProductionExceptionHandler());
+ new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
+ @Override
+ public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
+ callback.onCompletion(null, new Exception());
+ return null;
+ }
+ },
+ "test",
+ logContext,
+ new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
try {
@@ -251,16 +255,16 @@ public class RecordCollectorTest {
@Test
public void shouldNotThrowStreamsExceptionOnCloseIfASendFailedWithContinueExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl(
- new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
- @Override
- public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
- callback.onCompletion(null, new Exception());
- return null;
- }
- },
- "test",
- logContext,
- new AlwaysContinueProductionExceptionHandler());
+ new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
+ @Override
+ public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
+ callback.onCompletion(null, new Exception());
+ return null;
+ }
+ },
+ "test",
+ logContext,
+ new AlwaysContinueProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
collector.close();
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java
index 3a4efba..7e90466 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java
@@ -56,22 +56,17 @@ public class RecordDeserializerTest {
assertEquals(TimestampType.CREATE_TIME, record.timestampType());
}
- static class TheSourceNode extends SourceNode {
+ static class TheSourceNode extends SourceNode<Object, Object> {
private final boolean keyThrowsException;
private final boolean valueThrowsException;
private final Object key;
private final Object value;
- TheSourceNode(final boolean keyThrowsException, final boolean valueThrowsException) {
- this(keyThrowsException, valueThrowsException, null, null);
- }
-
- @SuppressWarnings("unchecked")
TheSourceNode(final boolean keyThrowsException,
final boolean valueThrowsException,
final Object key,
final Object value) {
- super("", Collections.EMPTY_LIST, null, null);
+ super("", Collections.<String>emptyList(), null, null);
this.keyThrowsException = keyThrowsException;
this.valueThrowsException = valueThrowsException;
this.key = key;
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java
index faf72e9..48be292 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java
@@ -55,7 +55,7 @@ public class RecordQueueTest {
private final String[] topics = {"topic"};
final InternalMockProcessorContext context = new InternalMockProcessorContext(StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class),
- new RecordCollectorImpl(null, null, new LogContext("record-queue-test "), new DefaultProductionExceptionHandler()));
+ new RecordCollectorImpl(null, null, new LogContext("record-queue-test "), new DefaultProductionExceptionHandler()));
private final MockSourceNode mockSourceNodeWithMetrics = new MockSourceNode<>(topics, intDeserializer, intDeserializer);
private final RecordQueue queue = new RecordQueue(
new TopicPartition(topics[0], 1),
@@ -93,7 +93,7 @@ public class RecordQueueTest {
assertEquals(TimestampTracker.NOT_KNOWN, queue.timestamp());
// add three 3 out-of-order records with timestamp 2, 1, 3
- List<ConsumerRecord<byte[], byte[]>> list1 = Arrays.asList(
+ final List<ConsumerRecord<byte[], byte[]>> list1 = Arrays.asList(
new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
@@ -118,7 +118,7 @@ public class RecordQueueTest {
// add three 3 out-of-order records with timestamp 4, 1, 2
// now with 3, 4, 1, 2
- List<ConsumerRecord<byte[], byte[]>> list2 = Arrays.asList(
+ final List<ConsumerRecord<byte[], byte[]>> list2 = Arrays.asList(
new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
@@ -151,7 +151,7 @@ public class RecordQueueTest {
assertEquals(0, queue.timeTracker().size());
// add three more records with 4, 5, 6
- List<ConsumerRecord<byte[], byte[]>> list3 = Arrays.asList(
+ final List<ConsumerRecord<byte[], byte[]>> list3 = Arrays.asList(
new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
@@ -185,7 +185,7 @@ public class RecordQueueTest {
public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() {
final byte[] key = Serdes.Long().serializer().serialize("foo", 1L);
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
- new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, key, recordValue));
+ new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, key, recordValue));
queue.addRawRecords(records);
}
@@ -194,24 +194,23 @@ public class RecordQueueTest {
public void shouldThrowStreamsExceptionWhenValueDeserializationFails() {
final byte[] value = Serdes.Long().serializer().serialize("foo", 1L);
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
- new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, value));
+ new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, value));
queue.addRawRecords(records);
}
@Test
- public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() throws Exception {
+ public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() {
final byte[] key = Serdes.Long().serializer().serialize("foo", 1L);
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, key, recordValue));
- final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
queueThatSkipsDeserializeErrors.addRawRecords(records);
assertEquals(0, queueThatSkipsDeserializeErrors.size());
}
@Test
- public void shouldNotThrowStreamsExceptionWhenValueDeserializationFailsWithSkipHandler() throws Exception {
+ public void shouldNotThrowStreamsExceptionWhenValueDeserializationFailsWithSkipHandler() {
final byte[] value = Serdes.Long().serializer().serialize("foo", 1L);
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, value));
@@ -224,28 +223,30 @@ public class RecordQueueTest {
@Test(expected = StreamsException.class)
public void shouldThrowOnNegativeTimestamp() {
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
- new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
-
- final RecordQueue queue = new RecordQueue(new TopicPartition(topics[0], 1),
- new MockSourceNode<>(topics, intDeserializer, intDeserializer),
- new FailOnInvalidTimestamp(),
- new LogAndContinueExceptionHandler(),
- null,
- new LogContext());
+ new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
+
+ final RecordQueue queue = new RecordQueue(
+ new TopicPartition(topics[0], 1),
+ new MockSourceNode<>(topics, intDeserializer, intDeserializer),
+ new FailOnInvalidTimestamp(),
+ new LogAndContinueExceptionHandler(),
+ null,
+ new LogContext());
queue.addRawRecords(records);
}
@Test
public void shouldDropOnNegativeTimestamp() {
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
- new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
-
- final RecordQueue queue = new RecordQueue(new TopicPartition(topics[0], 1),
- new MockSourceNode<>(topics, intDeserializer, intDeserializer),
- new LogAndSkipOnInvalidTimestamp(),
- new LogAndContinueExceptionHandler(),
- null,
- new LogContext());
+ new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
+
+ final RecordQueue queue = new RecordQueue(
+ new TopicPartition(topics[0], 1),
+ new MockSourceNode<>(topics, intDeserializer, intDeserializer),
+ new LogAndSkipOnInvalidTimestamp(),
+ new LogAndContinueExceptionHandler(),
+ null,
+ new LogContext());
queue.addRawRecords(records);
assertEquals(0, queue.size());
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java
index 4b48a17..d005bbd 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java
@@ -35,10 +35,17 @@ import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.fail;
public class SinkNodeTest {
- private final Serializer anySerializer = Serdes.Bytes().serializer();
- private final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
- private final InternalMockProcessorContext context = new InternalMockProcessorContext(anyStateSerde,
- new RecordCollectorImpl(new MockProducer<byte[], byte[]>(true, anySerializer, anySerializer), null, new LogContext("sinknode-test "), new DefaultProductionExceptionHandler()));
+ private final Serializer<byte[]> anySerializer = Serdes.ByteArray().serializer();
+ private final StateSerdes<Bytes, Bytes> anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
+ private final InternalMockProcessorContext context = new InternalMockProcessorContext(
+ anyStateSerde,
+ new RecordCollectorImpl(
+ new MockProducer<>(true, anySerializer, anySerializer),
+ null,
+ new LogContext("sinknode-test "),
+ new DefaultProductionExceptionHandler()
+ )
+ );
private final SinkNode sink = new SinkNode<>("anyNodeName", "any-output-topic", anySerializer, anySerializer, null);
@Before
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
index d6a5276..8d81b1f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
@@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.serialization.ByteArraySerializer;
@@ -57,13 +58,14 @@ import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
import java.util.Map;
-import java.util.Properties;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkProperties;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.MatcherAssert.assertThat;
@@ -88,7 +90,7 @@ public class StreamTaskTest {
private final MockSourceNode<Integer, Integer> source2 = new MockSourceNode<>(new String[]{topic2}, intDeserializer, intDeserializer);
private final MockSourceNode<Integer, Integer> source3 = new MockSourceNode<Integer, Integer>(new String[]{topic2}, intDeserializer, intDeserializer) {
@Override
- public void process(Integer key, Integer value) {
+ public void process(final Integer key, final Integer value) {
throw new RuntimeException("KABOOM!");
}
@@ -106,13 +108,8 @@ public class StreamTaskTest {
private final Long offset = 543L;
private final ProcessorTopology topology = ProcessorTopology.withSources(
- Utils.<ProcessorNode>mkList(source1, source2, processorStreamTime, processorSystemTime),
- new HashMap<String, SourceNode>() {
- {
- put(topic1, source1);
- put(topic2, source2);
- }
- }
+ Utils.<ProcessorNode>mkList(source1, source2, processorStreamTime, processorSystemTime),
+ mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source2))
);
private final MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
@@ -131,41 +128,39 @@ public class StreamTaskTest {
private final StreamsMetrics streamsMetrics = new MockStreamsMetrics(metrics);
private final TaskId taskId00 = new TaskId(0, 0);
private final MockTime time = new MockTime();
- private File baseDir = TestUtils.tempDirectory();
+ private final File baseDir = TestUtils.tempDirectory();
private StateDirectory stateDirectory;
- private StreamsConfig config;
- private StreamsConfig eosConfig;
private StreamTask task;
private long punctuatedAt;
- private Punctuator punctuator = new Punctuator() {
+ private final Punctuator punctuator = new Punctuator() {
@Override
- public void punctuate(long timestamp) {
+ public void punctuate(final long timestamp) {
punctuatedAt = timestamp;
}
};
- private StreamsConfig createConfig(final boolean enableEoS) throws IOException {
- return new StreamsConfig(new Properties() {
- {
- setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "stream-task-test");
- setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171");
- setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3");
- setProperty(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath());
- setProperty(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName());
- if (enableEoS) {
- setProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE);
- }
- }
- });
+ private StreamsConfig createConfig(final boolean enableEoS) {
+ final String canonicalPath;
+ try {
+ canonicalPath = baseDir.getCanonicalPath();
+ } catch (final IOException e) {
+ throw new RuntimeException(e);
+ }
+ return new StreamsConfig(mkProperties(mkMap(
+ mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "stream-task-test"),
+ mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"),
+ mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"),
+ mkEntry(StreamsConfig.STATE_DIR_CONFIG, canonicalPath),
+ mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()),
+ mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, enableEoS ? StreamsConfig.EXACTLY_ONCE : StreamsConfig.AT_LEAST_ONCE)
+ )));
}
@Before
- public void setup() throws IOException {
+ public void setup() {
consumer.assign(Arrays.asList(partition1, partition2));
- config = createConfig(false);
- eosConfig = createConfig(true);
- stateDirectory = new StateDirectory(config, new MockTime());
+ stateDirectory = new StateDirectory(createConfig(false), new MockTime());
}
@After
@@ -174,7 +169,7 @@ public class StreamTaskTest {
if (task != null) {
try {
task.close(true, false);
- } catch (Exception e) {
+ } catch (final Exception e) {
// swallow
}
}
@@ -186,18 +181,18 @@ public class StreamTaskTest {
@SuppressWarnings("unchecked")
@Test
public void testProcessOrder() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
- task.addRecords(partition1, records(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition1, Arrays.asList(
+ getConsumerRecord(partition1, 10),
+ getConsumerRecord(partition1, 20),
+ getConsumerRecord(partition1, 30)
));
- task.addRecords(partition2, records(
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition2, Arrays.asList(
+ getConsumerRecord(partition2, 25),
+ getConsumerRecord(partition2, 35),
+ getConsumerRecord(partition2, 45)
));
assertTrue(task.process());
@@ -232,48 +227,44 @@ public class StreamTaskTest {
}
- private void testSpecificMetrics(final String operation, final String groupName, final Map<String, String> tags) {
- assertNotNull(metrics.metrics().get(metrics.metricName(operation + "-latency-avg", groupName,
- "The average latency of " + operation + " operation.", tags)));
- assertNotNull(metrics.metrics().get(metrics.metricName(operation + "-latency-max", groupName,
- "The max latency of " + operation + " operation.", tags)));
- assertNotNull(metrics.metrics().get(metrics.metricName(operation + "-rate", groupName,
- "The average number of occurrence of " + operation + " operation per second.", tags)));
- }
-
-
@Test
public void testMetrics() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
- final String name = task.id().toString();
- final Map<String, String> metricTags = new LinkedHashMap<>();
- metricTags.put("task-id", name);
- final String operation = "commit";
+ assertNotNull(metrics.getSensor("commit"));
+ assertNotNull(getMetric("%s-latency-avg", "The average latency of %s operation.", task.id().toString()));
+ assertNotNull(getMetric("%s-latency-max", "The max latency of %s operation.", task.id().toString()));
+ assertNotNull(getMetric("%s-rate", "The average number of occurrence of %s operation per second.", task.id().toString()));
- final String groupName = "stream-task-metrics";
+ assertNotNull(getMetric("%s-latency-avg", "The average latency of %s operation.", "all"));
+ assertNotNull(getMetric("%s-latency-max", "The max latency of %s operation.", "all"));
+ assertNotNull(getMetric("%s-rate", "The average number of occurrence of %s operation per second.", "all"));
+ }
- assertNotNull(metrics.getSensor(operation));
- testSpecificMetrics(operation, groupName, metricTags);
- metricTags.put("task-id", "all");
- testSpecificMetrics(operation, groupName, metricTags);
+ private KafkaMetric getMetric(final String nameFormat, final String descriptionFormat, final String taskId) {
+ return metrics.metrics().get(metrics.metricName(
+ String.format(nameFormat, "commit"),
+ "stream-task-metrics",
+ String.format(descriptionFormat, "commit"),
+ mkMap(mkEntry("task-id", taskId))
+ ));
}
@SuppressWarnings("unchecked")
@Test
public void testPauseResume() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
- task.addRecords(partition1, records(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition1, Arrays.asList(
+ getConsumerRecord(partition1, 10),
+ getConsumerRecord(partition1, 20)
));
- task.addRecords(partition2, records(
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition2, Arrays.asList(
+ getConsumerRecord(partition2, 35),
+ getConsumerRecord(partition2, 45),
+ getConsumerRecord(partition2, 55),
+ getConsumerRecord(partition2, 65)
));
assertTrue(task.process());
@@ -283,10 +274,10 @@ public class StreamTaskTest {
assertEquals(1, consumer.paused().size());
assertTrue(consumer.paused().contains(partition2));
- task.addRecords(partition1, records(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition1, Arrays.asList(
+ getConsumerRecord(partition1, 30),
+ getConsumerRecord(partition1, 40),
+ getConsumerRecord(partition1, 50)
));
assertEquals(2, consumer.paused().size());
@@ -317,23 +308,23 @@ public class StreamTaskTest {
@SuppressWarnings("unchecked")
@Test
public void testMaybePunctuateStreamTime() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
- task.addRecords(partition1, records(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 32, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 60, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition1, Arrays.asList(
+ getConsumerRecord(partition1, 0),
+ getConsumerRecord(partition1, 20),
+ getConsumerRecord(partition1, 32),
+ getConsumerRecord(partition1, 40),
+ getConsumerRecord(partition1, 60)
));
- task.addRecords(partition2, records(
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 61, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition2, Arrays.asList(
+ getConsumerRecord(partition2, 25),
+ getConsumerRecord(partition2, 35),
+ getConsumerRecord(partition2, 45),
+ getConsumerRecord(partition2, 61)
));
assertTrue(task.maybePunctuateStreamTime());
@@ -408,22 +399,22 @@ public class StreamTaskTest {
@SuppressWarnings("unchecked")
@Test
public void shouldPunctuateOnceStreamTimeAfterGap() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
- task.addRecords(partition1, records(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 142, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 155, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 160, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition1, Arrays.asList(
+ getConsumerRecord(partition1, 20),
+ getConsumerRecord(partition1, 142),
+ getConsumerRecord(partition1, 155),
+ getConsumerRecord(partition1, 160)
));
- task.addRecords(partition2, records(
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 145, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 159, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 161, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition2, Arrays.asList(
+ getConsumerRecord(partition2, 25),
+ getConsumerRecord(partition2, 145),
+ getConsumerRecord(partition2, 159),
+ getConsumerRecord(partition2, 161)
));
assertTrue(task.maybePunctuateStreamTime()); // punctuate at 20
@@ -494,20 +485,20 @@ public class StreamTaskTest {
@SuppressWarnings("unchecked")
@Test
public void testCancelPunctuateStreamTime() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
- task.addRecords(partition1, records(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition1, Arrays.asList(
+ getConsumerRecord(partition1, 20),
+ getConsumerRecord(partition1, 30),
+ getConsumerRecord(partition1, 40)
));
- task.addRecords(partition2, records(
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)
+ task.addRecords(partition2, Arrays.asList(
+ getConsumerRecord(partition2, 25),
+ getConsumerRecord(partition2, 35),
+ getConsumerRecord(partition2, 45)
));
assertTrue(task.maybePunctuateStreamTime());
@@ -527,10 +518,10 @@ public class StreamTaskTest {
@Test
public void shouldPunctuateSystemTimeWhenIntervalElapsed() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
- long now = time.milliseconds();
+ final long now = time.milliseconds();
time.sleep(10);
assertTrue(task.maybePunctuateSystemTime());
time.sleep(10);
@@ -547,7 +538,7 @@ public class StreamTaskTest {
@Test
public void shouldNotPunctuateSystemTimeWhenIntervalNotElapsed() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
assertFalse(task.maybePunctuateSystemTime());
@@ -558,10 +549,10 @@ public class StreamTaskTest {
@Test
public void shouldPunctuateOnceSystemTimeAfterGap() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
- long now = time.milliseconds();
+ final long now = time.milliseconds();
time.sleep(100);
assertTrue(task.maybePunctuateSystemTime());
assertFalse(task.maybePunctuateSystemTime());
@@ -584,10 +575,10 @@ public class StreamTaskTest {
@Test
public void testCancelPunctuateSystemTime() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
- long now = time.milliseconds();
+ final long now = time.milliseconds();
time.sleep(10);
assertTrue(task.maybePunctuateSystemTime());
processorSystemTime.supplier.scheduleCancellable.cancel();
@@ -601,8 +592,7 @@ public class StreamTaskTest {
task = createTaskThatThrowsException();
task.initializeStateStores();
task.initializeTopology();
- task.addRecords(partition2, Collections.singletonList(
- new ConsumerRecord<>(partition2.topic(), partition2.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
+ task.addRecords(partition2, singletonList(getConsumerRecord(partition2, 0)));
try {
task.process();
@@ -614,14 +604,14 @@ public class StreamTaskTest {
@Test
public void shouldWrapKafkaExceptionsWithStreamsExceptionAndAddContextWhenPunctuatingStreamTime() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
try {
task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, new Punctuator() {
@Override
- public void punctuate(long timestamp) {
+ public void punctuate(final long timestamp) {
throw new KafkaException("KABOOM!");
}
});
@@ -635,14 +625,14 @@ public class StreamTaskTest {
@Test
public void shouldWrapKafkaExceptionsWithStreamsExceptionAndAddContextWhenPunctuatingWallClockTimeTime() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
try {
task.punctuate(processorSystemTime, 1, PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
@Override
- public void punctuate(long timestamp) {
+ public void punctuate(final long timestamp) {
throw new KafkaException("KABOOM!");
}
});
@@ -659,7 +649,7 @@ public class StreamTaskTest {
final AtomicBoolean flushed = new AtomicBoolean(false);
final StreamsMetrics streamsMetrics = new MockStreamsMetrics(new Metrics());
final StreamTask streamTask = new StreamTask(taskId00, partitions, topology, consumer,
- changelogReader, config, streamsMetrics, stateDirectory, null, time, producer) {
+ changelogReader, createConfig(false), streamsMetrics, stateDirectory, null, time, producer) {
@Override
RecordCollector createRecordCollector(final LogContext logContext,
@@ -678,31 +668,34 @@ public class StreamTaskTest {
@Test
public void shouldCheckpointOffsetsOnCommit() throws IOException {
- task = createStatefulTask(false, true);
+ task = createStatefulTask(createConfig(false), true);
task.initializeStateStores();
task.initializeTopology();
task.commit();
- final OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId00),
- ProcessorStateManager.CHECKPOINT_FILE_NAME));
+ final OffsetCheckpoint checkpoint = new OffsetCheckpoint(
+ new File(stateDirectory.directoryForTask(taskId00), ProcessorStateManager.CHECKPOINT_FILE_NAME)
+ );
assertThat(checkpoint.read(), equalTo(Collections.singletonMap(changelogPartition, offset)));
}
@Test
public void shouldNotCheckpointOffsetsOnCommitIfEosIsEnabled() {
- task = createStatefulTask(true, true);
+ task = createStatefulTask(createConfig(true), true);
task.initializeStateStores();
task.initializeTopology();
task.commit();
- final File checkpointFile = new File(stateDirectory.directoryForTask(taskId00),
- ProcessorStateManager.CHECKPOINT_FILE_NAME);
+ final File checkpointFile = new File(
+ stateDirectory.directoryForTask(taskId00),
+ ProcessorStateManager.CHECKPOINT_FILE_NAME
+ );
assertFalse(checkpointFile.exists());
}
@Test
public void shouldThrowIllegalStateExceptionIfCurrentNodeIsNotNullWhenPunctuateCalled() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
task.processorContext.setCurrentNode(processorStreamTime);
@@ -716,7 +709,7 @@ public class StreamTaskTest {
@Test
public void shouldCallPunctuateOnPassedInProcessorNode() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
task.punctuate(processorStreamTime, 5, PunctuationType.STREAM_TIME, punctuator);
@@ -726,8 +719,8 @@ public class StreamTaskTest {
}
@Test
- public void shouldSetProcessorNodeOnContextBackToNullAfterSuccesfullPunctuate() {
- task = createStatelessTask(false);
+ public void shouldSetProcessorNodeOnContextBackToNullAfterSuccessfulPunctuate() {
+ task = createStatelessTask(createConfig(false));
task.initializeStateStores();
task.initializeTopology();
task.punctuate(processorStreamTime, 5, PunctuationType.STREAM_TIME, punctuator);
@@ -736,10 +729,10 @@ public class StreamTaskTest {
@Test(expected = IllegalStateException.class)
public void shouldThrowIllegalStateExceptionOnScheduleIfCurrentNodeIsNull() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.schedule(1, PunctuationType.STREAM_TIME, new Punctuator() {
@Override
- public void punctuate(long timestamp) {
+ public void punctuate(final long timestamp) {
// no-op
}
});
@@ -747,11 +740,11 @@ public class StreamTaskTest {
@Test
public void shouldNotThrowExceptionOnScheduleIfCurrentNodeIsNotNull() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.processorContext.setCurrentNode(processorStreamTime);
task.schedule(1, PunctuationType.STREAM_TIME, new Punctuator() {
@Override
- public void punctuate(long timestamp) {
+ public void punctuate(final long timestamp) {
// no-op
}
});
@@ -775,7 +768,7 @@ public class StreamTaskTest {
@Test
public void shouldInitAndBeginTransactionOnCreateIfEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
task.initializeTopology();
assertTrue(producer.transactionInitialized());
@@ -784,7 +777,7 @@ public class StreamTaskTest {
@Test
public void shouldNotThrowOnCloseIfTaskWasNotInitializedWithEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
assertTrue(!producer.transactionInFlight());
task.close(false, false);
@@ -792,7 +785,7 @@ public class StreamTaskTest {
@Test
public void shouldNotInitOrBeginTransactionOnCreateIfEosDisabled() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
assertFalse(producer.transactionInitialized());
assertFalse(producer.transactionInFlight());
@@ -800,11 +793,10 @@ public class StreamTaskTest {
@Test
public void shouldSendOffsetsAndCommitTransactionButNotStartNewTransactionOnSuspendIfEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
task.initializeTopology();
- task.addRecords(partition1, Collections.singletonList(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
+ task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
task.process();
task.suspend();
@@ -815,7 +807,7 @@ public class StreamTaskTest {
@Test
public void shouldCommitTransactionOnSuspendEvenIfTransactionIsEmptyIfEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
task.initializeTopology();
task.suspend();
@@ -825,9 +817,8 @@ public class StreamTaskTest {
@Test
public void shouldNotSendOffsetsAndCommitTransactionNorStartNewTransactionOnSuspendIfEosDisabled() {
- task = createStatelessTask(false);
- task.addRecords(partition1, Collections.singletonList(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
+ task = createStatelessTask(createConfig(false));
+ task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
task.process();
task.suspend();
@@ -838,11 +829,10 @@ public class StreamTaskTest {
@Test
public void shouldStartNewTransactionOnResumeIfEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
task.initializeTopology();
- task.addRecords(partition1, Collections.singletonList(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
+ task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
task.process();
task.suspend();
@@ -853,10 +843,9 @@ public class StreamTaskTest {
@Test
public void shouldNotStartNewTransactionOnResumeIfEosDisabled() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
- task.addRecords(partition1, Collections.singletonList(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
+ task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
task.process();
task.suspend();
@@ -866,11 +855,10 @@ public class StreamTaskTest {
@Test
public void shouldStartNewTransactionOnCommitIfEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
task.initializeTopology();
- task.addRecords(partition1, Collections.singletonList(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
+ task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
task.process();
task.commit();
@@ -879,10 +867,9 @@ public class StreamTaskTest {
@Test
public void shouldNotStartNewTransactionOnCommitIfEosDisabled() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
- task.addRecords(partition1, Collections.singletonList(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
+ task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
task.process();
task.commit();
@@ -891,7 +878,7 @@ public class StreamTaskTest {
@Test
public void shouldAbortTransactionOnDirtyClosedIfEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
task.initializeTopology();
task.close(false, false);
task = null;
@@ -901,7 +888,7 @@ public class StreamTaskTest {
@Test
public void shouldNotAbortTransactionOnZombieClosedIfEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
task.close(false, true);
task = null;
@@ -910,7 +897,7 @@ public class StreamTaskTest {
@Test
public void shouldNotAbortTransactionOnDirtyClosedIfEosDisabled() {
- task = createStatelessTask(false);
+ task = createStatelessTask(createConfig(false));
task.close(false, false);
task = null;
@@ -919,7 +906,7 @@ public class StreamTaskTest {
@Test
public void shouldCloseProducerOnCloseWhenEosEnabled() {
- task = createStatelessTask(true);
+ task = createStatelessTask(createConfig(true));
task.close(true, false);
task = null;
@@ -935,7 +922,7 @@ public class StreamTaskTest {
try {
task.commit();
fail("should have thrown an exception");
- } catch (Exception e) {
+ } catch (final Exception e) {
// all good
}
}
@@ -949,21 +936,21 @@ public class StreamTaskTest {
try {
task.suspend();
fail("should have thrown an exception");
- } catch (Exception e) {
+ } catch (final Exception e) {
// all good
}
}
@Test
public void shouldCloseStateManagerIfFailureOnTaskClose() {
- task = createStatefulTaskThatThrowsExceptionOnClose(true, false);
+ task = createStatefulTaskThatThrowsExceptionOnClose();
task.initializeStateStores();
task.initializeTopology();
try {
task.close(true, false);
fail("should have thrown an exception");
- } catch (Exception e) {
+ } catch (final Exception e) {
// all good
}
@@ -976,21 +963,21 @@ public class StreamTaskTest {
final StreamTask task = createTaskThatThrowsException();
try {
task.close(false, false);
- } catch (Exception e) {
- fail("should have not closed unitialized topology");
+ } catch (final Exception e) {
+ fail("should have not closed non-initialized topology");
}
}
@Test
public void shouldBeInitializedIfChangelogPartitionsIsEmpty() {
- final StreamTask task = createStatefulTask(false, false);
+ final StreamTask task = createStatefulTask(createConfig(false), false);
assertTrue(task.initializeStateStores());
}
@Test
public void shouldNotBeInitializedIfChangelogPartitionsIsNonEmpty() {
- final StreamTask task = createStatefulTask(false, true);
+ final StreamTask task = createStatefulTask(createConfig(false), true);
assertFalse(task.initializeStateStores());
}
@@ -998,79 +985,91 @@ public class StreamTaskTest {
@Test
public void shouldReturnOffsetsForRepartitionTopicsForPurging() {
final TopicPartition repartition = new TopicPartition("repartition", 1);
+
final ProcessorTopology topology = ProcessorTopology.withRepartitionTopics(
- Utils.<ProcessorNode>mkList(source1, source2),
- new HashMap<String, SourceNode>() {
- {
- put(topic1, source1);
- put(repartition.topic(), source2);
- }
- },
- Collections.singleton(repartition.topic())
+ Utils.<ProcessorNode>mkList(source1, source2),
+ mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(repartition.topic(), (SourceNode) source2)),
+ Collections.singleton(repartition.topic())
);
consumer.assign(Arrays.asList(partition1, repartition));
- task = new StreamTask(taskId00, Utils.mkSet(partition1, repartition), topology, consumer, changelogReader, config,
- streamsMetrics, stateDirectory, null, time, producer);
+ task = new StreamTask(
+ taskId00,
+ Utils.mkSet(partition1, repartition),
+ topology,
+ consumer,
+ changelogReader,
+ createConfig(false),
+ streamsMetrics,
+ stateDirectory,
+ null,
+ time,
+ producer
+ );
task.initializeStateStores();
task.initializeTopology();
- task.addRecords(partition1, Collections.singletonList(
- new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5L, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
- task.addRecords(repartition, Collections.singletonList(
- new ConsumerRecord<>(repartition.topic(), repartition.partition(), 10L, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
+ task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 5L)));
+ task.addRecords(repartition, singletonList(getConsumerRecord(repartition, 10L)));
assertTrue(task.process());
assertTrue(task.process());
task.commit();
- Map<TopicPartition, Long> map = task.purgableOffsets();
+ final Map<TopicPartition, Long> map = task.purgableOffsets();
assertThat(map, equalTo(Collections.singletonMap(repartition, 11L)));
}
- private StreamTask createStatefulTask(final boolean eosEnabled, final boolean logged) {
+ private StreamTask createStatefulTask(final StreamsConfig config, final boolean logged) {
final ProcessorTopology topology = ProcessorTopology.with(
- Utils.<ProcessorNode>mkList(source1, source2),
- new HashMap<String, SourceNode>() {
- {
- put(topic1, source1);
- put(topic2, source2);
- }
- },
- Collections.singletonList(stateStore),
- logged ? Collections.singletonMap(storeName, storeName + "-changelog") : Collections.<String, String>emptyMap());
-
- return new StreamTask(taskId00, partitions, topology, consumer, changelogReader, eosEnabled ? eosConfig : config,
- streamsMetrics, stateDirectory, null, time, producer);
+ Utils.<ProcessorNode>mkList(source1, source2),
+ mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source2)),
+ singletonList(stateStore),
+ logged ? Collections.singletonMap(storeName, storeName + "-changelog") : Collections.<String, String>emptyMap());
+
+ return new StreamTask(
+ taskId00,
+ partitions,
+ topology,
+ consumer,
+ changelogReader,
+ config,
+ streamsMetrics,
+ stateDirectory,
+ null,
+ time,
+ producer
+ );
}
- private StreamTask createStatefulTaskThatThrowsExceptionOnClose(final boolean eosEnabled, final boolean logged) {
+ private StreamTask createStatefulTaskThatThrowsExceptionOnClose() {
final ProcessorTopology topology = ProcessorTopology.with(
- Utils.<ProcessorNode>mkList(source1, source3),
- new HashMap<String, SourceNode>() {
- {
- put(topic1, source1);
- put(topic2, source3);
- }
- },
- Collections.singletonList(stateStore),
- logged ? Collections.singletonMap(storeName, changelogPartition.topic()) : Collections.<String, String>emptyMap());
-
- return new StreamTask(taskId00, partitions, topology, consumer, changelogReader, eosEnabled ? eosConfig : config,
- streamsMetrics, stateDirectory, null, time, producer);
+ Utils.<ProcessorNode>mkList(source1, source3),
+ mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source3)),
+ singletonList(stateStore),
+ Collections.<String, String>emptyMap());
+
+ return new StreamTask(
+ taskId00,
+ partitions,
+ topology,
+ consumer,
+ changelogReader,
+ createConfig(true),
+ streamsMetrics,
+ stateDirectory,
+ null,
+ time,
+ producer
+ );
}
- private StreamTask createStatelessTask(final boolean eosEnabled) {
+ private StreamTask createStatelessTask(final StreamsConfig streamsConfig) {
final ProcessorTopology topology = ProcessorTopology.withSources(
- Utils.<ProcessorNode>mkList(source1, source2, processorStreamTime, processorSystemTime),
- new HashMap<String, SourceNode>() {
- {
- put(topic1, source1);
- put(topic2, source2);
- }
- }
+ Utils.<ProcessorNode>mkList(source1, source2, processorStreamTime, processorSystemTime),
+ mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source2))
);
source1.addChild(processorStreamTime);
@@ -1078,20 +1077,26 @@ public class StreamTaskTest {
source1.addChild(processorSystemTime);
source2.addChild(processorSystemTime);
- return new StreamTask(taskId00, partitions, topology, consumer, changelogReader, eosEnabled ? eosConfig : config,
- streamsMetrics, stateDirectory, null, time, producer);
+ return new StreamTask(
+ taskId00,
+ partitions,
+ topology,
+ consumer,
+ changelogReader,
+ streamsConfig,
+ streamsMetrics,
+ stateDirectory,
+ null,
+ time,
+ producer
+ );
}
// this task will throw exception when processing (on partition2), flushing, suspending and closing
private StreamTask createTaskThatThrowsException() {
final ProcessorTopology topology = ProcessorTopology.withSources(
- Utils.<ProcessorNode>mkList(source1, source3, processorStreamTime, processorSystemTime),
- new HashMap<String, SourceNode>() {
- {
- put(topic1, source1);
- put(topic2, source3);
- }
- }
+ Utils.<ProcessorNode>mkList(source1, source3, processorStreamTime, processorSystemTime),
+ mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source3))
);
source1.addChild(processorStreamTime);
@@ -1099,9 +1104,19 @@ public class StreamTaskTest {
source1.addChild(processorSystemTime);
source3.addChild(processorSystemTime);
- return new StreamTask(taskId00, partitions, topology, consumer, changelogReader, config,
- streamsMetrics, stateDirectory, null, time, producer) {
-
+ return new StreamTask(
+ taskId00,
+ partitions,
+ topology,
+ consumer,
+ changelogReader,
+ createConfig(false),
+ streamsMetrics,
+ stateDirectory,
+ null,
+ time,
+ producer
+ ) {
@Override
protected void flushState() {
throw new RuntimeException("KABOOM!");
@@ -1109,8 +1124,19 @@ public class StreamTaskTest {
};
}
- private Iterable<ConsumerRecord<byte[], byte[]>> records(final ConsumerRecord<byte[], byte[]>... recs) {
- return Arrays.asList(recs);
+ private ConsumerRecord<byte[], byte[]> getConsumerRecord(final TopicPartition topicPartition, final long offset) {
+ return new ConsumerRecord<>(
+ topicPartition.topic(),
+ topicPartition.partition(),
+ offset,
+ 0L,
+ TimestampType.CREATE_TIME,
+ 0L,
+ 0,
+ 0,
+ recordKey,
+ recordValue
+ );
}
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
index b22d98e..b95507d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
@@ -72,6 +72,10 @@ import java.util.Properties;
import java.util.Set;
import java.util.UUID;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkProperties;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -93,7 +97,7 @@ public class StreamThreadTest {
private InternalTopologyBuilder internalTopologyBuilder;
private final StreamsConfig config = new StreamsConfig(configProps(false));
private final String stateDir = TestUtils.tempDirectory().getPath();
- private final StateDirectory stateDirectory = new StateDirectory(config, mockTime);
+ private final StateDirectory stateDirectory = new StateDirectory(config, mockTime);
private StreamsMetadataState streamsMetadataState;
private final ConsumedInternal<Object, Object> consumed = new ConsumedInternal<>();
@@ -118,19 +122,15 @@ public class StreamThreadTest {
private final TaskId task2 = new TaskId(0, 2);
private final TaskId task3 = new TaskId(1, 1);
- private Properties configProps(final boolean enableEos) {
- return new Properties() {
- {
- setProperty(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
- setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171");
- setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3");
- setProperty(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName());
- setProperty(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath());
- if (enableEos) {
- setProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE);
- }
- }
- };
+ private Properties configProps(final boolean enableEoS) {
+ return mkProperties(mkMap(
+ mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId),
+ mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"),
+ mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"),
+ mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()),
+ mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()),
+ mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, enableEoS ? StreamsConfig.EXACTLY_ONCE : StreamsConfig.AT_LEAST_ONCE)
+ ));
}
@Test
@@ -146,8 +146,8 @@ public class StreamThreadTest {
final ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener;
thread.setState(StreamThread.State.RUNNING);
- List<TopicPartition> revokedPartitions;
- List<TopicPartition> assignedPartitions;
+ final List<TopicPartition> revokedPartitions;
+ final List<TopicPartition> assignedPartitions;
// revoke nothing
revokedPartitions = Collections.emptyList();
@@ -156,7 +156,7 @@ public class StreamThreadTest {
assertEquals(thread.state(), StreamThread.State.PARTITIONS_REVOKED);
// assign single partition
- assignedPartitions = Collections.singletonList(t1p1);
+ assignedPartitions = singletonList(t1p1);
thread.taskManager().setAssignmentMetadata(Collections.<TaskId, Set<TopicPartition>>emptyMap(), Collections.<TaskId, Set<TopicPartition>>emptyMap());
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
@@ -169,7 +169,7 @@ public class StreamThreadTest {
Assert.assertEquals(StreamThread.State.PARTITIONS_ASSIGNED, stateListener.oldState);
thread.shutdown();
- assertTrue(thread.state() == StreamThread.State.PENDING_SHUTDOWN);
+ assertSame(StreamThread.State.PENDING_SHUTDOWN, thread.state());
}
@Test
@@ -199,39 +199,44 @@ public class StreamThreadTest {
assertEquals(thread.state(), StreamThread.State.DEAD);
}
- private Cluster createCluster(final int numNodes) {
- HashMap<Integer, Node> nodes = new HashMap<>();
- for (int i = 0; i < numNodes; ++i) {
- nodes.put(i, new Node(i, "localhost", 8121 + i));
- }
- return new Cluster("mockClusterId", nodes.values(),
- Collections.<PartitionInfo>emptySet(), Collections.<String>emptySet(),
- Collections.<String>emptySet(), nodes.get(0));
+ private Cluster createCluster() {
+ final Node node = new Node(0, "localhost", 8121);
+ return new Cluster(
+ "mockClusterId",
+ singletonList(node),
+ Collections.<PartitionInfo>emptySet(),
+ Collections.<String>emptySet(),
+ Collections.<String>emptySet(),
+ node
+ );
}
- private StreamThread createStreamThread(final String clientId, final StreamsConfig config, final boolean eosEnabled) {
+ private StreamThread createStreamThread(@SuppressWarnings("SameParameterValue") final String clientId,
+ final StreamsConfig config,
+ final boolean eosEnabled) {
if (eosEnabled) {
clientSupplier.setApplicationIdForProducer(applicationId);
}
- clientSupplier.setClusterForAdminClient(createCluster(1));
-
- return StreamThread.create(internalTopologyBuilder,
- config,
- clientSupplier,
- clientSupplier.getAdminClient(config.getAdminConfigs(clientId)),
- processId,
- clientId,
- metrics,
- mockTime,
- streamsMetadataState,
- 0,
- stateDirectory,
- new MockStateRestoreListener());
+ clientSupplier.setClusterForAdminClient(createCluster());
+
+ return StreamThread.create(
+ internalTopologyBuilder,
+ config,
+ clientSupplier,
+ clientSupplier.getAdminClient(config.getAdminConfigs(clientId)),
+ processId,
+ clientId,
+ metrics,
+ mockTime,
+ streamsMetadataState,
+ 0,
+ stateDirectory,
+ new MockStateRestoreListener());
}
@Test
- public void testMetrics() {
+ public void testMetricsCreatedAtStartup() {
final StreamThread thread = createStreamThread(clientId, config, false);
final String defaultGroupName = "stream-metrics";
final String defaultPrefix = "thread." + thread.getName();
@@ -248,18 +253,25 @@ public class StreamThreadTest {
assertNotNull(metrics.metrics().get(metrics.metricName("commit-latency-avg", defaultGroupName, "The average commit time in ms", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("commit-latency-max", defaultGroupName, "The maximum commit time in ms", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("commit-rate", defaultGroupName, "The average per-second number of commit calls", defaultTags)));
+ assertNotNull(metrics.metrics().get(metrics.metricName("commit-total", defaultGroupName, "The total number of commit calls", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("poll-latency-avg", defaultGroupName, "The average poll time in ms", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("poll-latency-max", defaultGroupName, "The maximum poll time in ms", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("poll-rate", defaultGroupName, "The average per-second number of record-poll calls", defaultTags)));
+ assertNotNull(metrics.metrics().get(metrics.metricName("poll-total", defaultGroupName, "The total number of record-poll calls", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("process-latency-avg", defaultGroupName, "The average process time in ms", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("process-latency-max", defaultGroupName, "The maximum process time in ms", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("process-rate", defaultGroupName, "The average per-second number of process calls", defaultTags)));
+ assertNotNull(metrics.metrics().get(metrics.metricName("process-total", defaultGroupName, "The total number of process calls", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-latency-avg", defaultGroupName, "The average punctuate time in ms", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-latency-max", defaultGroupName, "The maximum punctuate time in ms", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-rate", defaultGroupName, "The average per-second number of punctuate calls", defaultTags)));
+ assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-total", defaultGroupName, "The total number of punctuate calls", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("task-created-rate", defaultGroupName, "The average per-second number of newly created tasks", defaultTags)));
+ assertNotNull(metrics.metrics().get(metrics.metricName("task-created-total", defaultGroupName, "The total number of newly created tasks", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("task-closed-rate", defaultGroupName, "The average per-second number of closed tasks", defaultTags)));
+ assertNotNull(metrics.metrics().get(metrics.metricName("task-closed-total", defaultGroupName, "The total number of closed tasks", defaultTags)));
assertNotNull(metrics.metrics().get(metrics.metricName("skipped-records-rate", defaultGroupName, "The average per-second number of skipped records.", defaultTags)));
+ assertNotNull(metrics.metrics().get(metrics.metricName("skipped-records-total", defaultGroupName, "The total number of skipped records.", defaultTags)));
}
@@ -275,17 +287,18 @@ public class StreamThreadTest {
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 1);
- StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
- final StreamThread thread = new StreamThread(mockTime,
- config,
- consumer,
- consumer,
- null,
- taskManager,
- streamsMetrics,
- internalTopologyBuilder,
- clientId,
- new LogContext("")
+ final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
+ final StreamThread thread = new StreamThread(
+ mockTime,
+ config,
+ consumer,
+ consumer,
+ null,
+ taskManager,
+ streamsMetrics,
+ internalTopologyBuilder,
+ clientId,
+ new LogContext("")
);
thread.maybeCommit(mockTime.milliseconds());
mockTime.sleep(commitInterval - 10L);
@@ -296,7 +309,7 @@ public class StreamThreadTest {
@SuppressWarnings({"unchecked", "ThrowableNotThrown"})
@Test
- public void shouldNotCauseExceptionIfNothingCommited() {
+ public void shouldNotCauseExceptionIfNothingCommitted() {
final long commitInterval = 1000L;
final Properties props = configProps(false);
props.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir);
@@ -306,17 +319,18 @@ public class StreamThreadTest {
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 0);
- StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
- final StreamThread thread = new StreamThread(mockTime,
- config,
- consumer,
- consumer,
- null,
- taskManager,
- streamsMetrics,
- internalTopologyBuilder,
- clientId,
- new LogContext(""));
+ final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
+ final StreamThread thread = new StreamThread(
+ mockTime,
+ config,
+ consumer,
+ consumer,
+ null,
+ taskManager,
+ streamsMetrics,
+ internalTopologyBuilder,
+ clientId,
+ new LogContext(""));
thread.maybeCommit(mockTime.milliseconds());
mockTime.sleep(commitInterval - 10L);
thread.maybeCommit(mockTime.milliseconds());
@@ -337,17 +351,18 @@ public class StreamThreadTest {
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
final TaskManager taskManager = mockTaskManagerCommit(consumer, 2, 1);
- StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
- final StreamThread thread = new StreamThread(mockTime,
- config,
- consumer,
- consumer,
- null,
- taskManager,
- streamsMetrics,
- internalTopologyBuilder,
- clientId,
- new LogContext(""));
+ final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
+ final StreamThread thread = new StreamThread(
+ mockTime,
+ config,
+ consumer,
+ consumer,
+ null,
+ taskManager,
+ streamsMetrics,
+ internalTopologyBuilder,
+ clientId,
+ new LogContext(""));
thread.maybeCommit(mockTime.milliseconds());
mockTime.sleep(commitInterval + 1);
thread.maybeCommit(mockTime.milliseconds());
@@ -385,7 +400,7 @@ public class StreamThreadTest {
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
mockConsumer.assign(assignedPartitions);
- Map<TopicPartition, Long> beginOffsets = new HashMap<>();
+ final Map<TopicPartition, Long> beginOffsets = new HashMap<>();
beginOffsets.put(t1p1, 0L);
beginOffsets.put(t1p2, 0L);
mockConsumer.updateBeginningOffsets(beginOffsets);
@@ -422,7 +437,7 @@ public class StreamThreadTest {
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
mockConsumer.assign(assignedPartitions);
- Map<TopicPartition, Long> beginOffsets = new HashMap<>();
+ final Map<TopicPartition, Long> beginOffsets = new HashMap<>();
beginOffsets.put(t1p1, 0L);
beginOffsets.put(t1p2, 0L);
mockConsumer.updateBeginningOffsets(beginOffsets);
@@ -456,7 +471,7 @@ public class StreamThreadTest {
thread.taskManager().setAssignmentMetadata(activeTasks, Collections.<TaskId, Set<TopicPartition>>emptyMap());
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
mockConsumer.assign(assignedPartitions);
- Map<TopicPartition, Long> beginOffsets = new HashMap<>();
+ final Map<TopicPartition, Long> beginOffsets = new HashMap<>();
beginOffsets.put(t1p1, 0L);
beginOffsets.put(t1p2, 0L);
mockConsumer.updateBeginningOffsets(beginOffsets);
@@ -483,30 +498,32 @@ public class StreamThreadTest {
EasyMock.replay(taskManager, consumer);
final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(
- metrics,
- "",
- "",
- Collections.<String, String>emptyMap());
+ metrics,
+ "",
+ "",
+ Collections.<String, String>emptyMap()
+ );
final StreamThread thread = new StreamThread(
- mockTime,
- config,
- consumer,
- consumer,
- null,
- taskManager,
- streamsMetrics,
- internalTopologyBuilder,
- clientId,
- new LogContext(""));
+ mockTime,
+ config,
+ consumer,
+ consumer,
+ null,
+ taskManager,
+ streamsMetrics,
+ internalTopologyBuilder,
+ clientId,
+ new LogContext("")
+ );
thread.setStateListener(
- new StreamThread.StateListener() {
- @Override
- public void onChange(final Thread t, final ThreadStateTransitionValidator newState, final ThreadStateTransitionValidator oldState) {
- if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.RUNNING) {
- thread.shutdown();
- }
+ new StreamThread.StateListener() {
+ @Override
+ public void onChange(final Thread t, final ThreadStateTransitionValidator newState, final ThreadStateTransitionValidator oldState) {
+ if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.RUNNING) {
+ thread.shutdown();
}
- });
+ }
+ });
thread.run();
EasyMock.verify(taskManager);
}
@@ -521,21 +538,23 @@ public class StreamThreadTest {
EasyMock.replay(taskManager, consumer);
final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(
- metrics,
- "",
- "",
- Collections.<String, String>emptyMap());
+ metrics,
+ "",
+ "",
+ Collections.<String, String>emptyMap()
+ );
final StreamThread thread = new StreamThread(
- mockTime,
- config,
- consumer,
- consumer,
- null,
- taskManager,
- streamsMetrics,
- internalTopologyBuilder,
- clientId,
- new LogContext(""));
+ mockTime,
+ config,
+ consumer,
+ consumer,
+ null,
+ taskManager,
+ streamsMetrics,
+ internalTopologyBuilder,
+ clientId,
+ new LogContext("")
+ );
thread.shutdown();
EasyMock.verify(taskManager);
}
@@ -601,7 +620,7 @@ public class StreamThreadTest {
final MockConsumer<byte[], byte[]> consumer = clientSupplier.consumer;
- consumer.updatePartitions(topic1, Collections.singletonList(new PartitionInfo(topic1, 1, null, null, null)));
+ consumer.updatePartitions(topic1, singletonList(new PartitionInfo(topic1, 1, null, null, null)));
thread.setState(StreamThread.State.RUNNING);
thread.rebalanceListener.onPartitionsRevoked(null);
@@ -749,7 +768,7 @@ public class StreamThreadTest {
thread.runOnce(-1);
- ThreadMetadata threadMetadata = thread.threadMetadata();
+ final ThreadMetadata threadMetadata = thread.threadMetadata();
assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState());
assertTrue(threadMetadata.activeTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1))));
assertTrue(threadMetadata.standbyTasks().isEmpty());
@@ -762,12 +781,15 @@ public class StreamThreadTest {
final StreamThread thread = createStreamThread(clientId, config, false);
final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer;
- restoreConsumer.updatePartitions("stream-thread-test-count-one-changelog",
- Collections.singletonList(new PartitionInfo("stream-thread-test-count-one-changelog",
- 0,
- null,
- new Node[0],
- new Node[0])));
+ restoreConsumer.updatePartitions(
+ "stream-thread-test-count-one-changelog",
+ singletonList(new PartitionInfo("stream-thread-test-count-one-changelog",
+ 0,
+ null,
+ new Node[0],
+ new Node[0])
+ )
+ );
final HashMap<TopicPartition, Long> offsets = new HashMap<>();
offsets.put(new TopicPartition("stream-thread-test-count-one-changelog", 1), 0L);
@@ -789,7 +811,7 @@ public class StreamThreadTest {
thread.runOnce(-1);
- ThreadMetadata threadMetadata = thread.threadMetadata();
+ final ThreadMetadata threadMetadata = thread.threadMetadata();
assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState());
assertTrue(threadMetadata.standbyTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1))));
assertTrue(threadMetadata.activeTasks().isEmpty());
@@ -804,17 +826,22 @@ public class StreamThreadTest {
final TopicPartition partition1 = new TopicPartition(changelogName, 1);
final TopicPartition partition2 = t2p1;
internalStreamsBuilder.stream(Collections.singleton(topic1), consumed)
- .groupByKey().count(Materialized.<Object, Long, KeyValueStore<Bytes, byte[]>>as(storeName1));
+ .groupByKey().count(Materialized.<Object, Long, KeyValueStore<Bytes, byte[]>>as(storeName1));
internalStreamsBuilder.table(topic2, new ConsumedInternal(), new MaterializedInternal(Materialized.as(storeName2), internalStreamsBuilder, ""));
final StreamThread thread = createStreamThread(clientId, config, false);
final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer;
restoreConsumer.updatePartitions(changelogName,
- Collections.singletonList(new PartitionInfo(changelogName,
- 1,
- null,
- new Node[0],
- new Node[0])));
+ singletonList(
+ new PartitionInfo(
+ changelogName,
+ 1,
+ null,
+ new Node[0],
+ new Node[0]
+ )
+ )
+ );
restoreConsumer.assign(Utils.mkSet(partition1, partition2));
restoreConsumer.updateEndOffsets(Collections.singletonMap(partition1, 10L));
@@ -867,30 +894,30 @@ public class StreamThreadTest {
public Processor<Object, Object> get() {
return new Processor<Object, Object>() {
@Override
- public void init(ProcessorContext context) {
+ public void init(final ProcessorContext context) {
context.schedule(100L, PunctuationType.STREAM_TIME, new Punctuator() {
@Override
- public void punctuate(long timestamp) {
+ public void punctuate(final long timestamp) {
punctuatedStreamTime.add(timestamp);
}
});
context.schedule(100L, PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
@Override
- public void punctuate(long timestamp) {
+ public void punctuate(final long timestamp) {
punctuatedWallClockTime.add(timestamp);
}
});
}
@Override
- public void process(Object key, Object value) { }
+ public void process(final Object key, final Object value) {}
@SuppressWarnings("deprecation")
@Override
- public void punctuate(long timestamp) { }
+ public void punctuate(final long timestamp) {}
@Override
- public void close() { }
+ public void close() {}
};
}
};
@@ -959,18 +986,18 @@ public class StreamThreadTest {
final StreamThread thread = createStreamThread(clientId, config, false);
final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer;
restoreConsumer.updatePartitions("stream-thread-test-count-one-changelog",
- Utils.mkList(
- new PartitionInfo("stream-thread-test-count-one-changelog",
- 0,
- null,
- new Node[0],
- new Node[0]),
- new PartitionInfo("stream-thread-test-count-one-changelog",
- 1,
- null,
- new Node[0],
- new Node[0])
- ));
+ Utils.mkList(
+ new PartitionInfo("stream-thread-test-count-one-changelog",
+ 0,
+ null,
+ new Node[0],
+ new Node[0]),
+ new PartitionInfo("stream-thread-test-count-one-changelog",
+ 1,
+ null,
+ new Node[0],
+ new Node[0])
+ ));
final HashMap<TopicPartition, Long> offsets = new HashMap<>();
offsets.put(new TopicPartition("stream-thread-test-count-one-changelog", 0), 0L);
offsets.put(new TopicPartition("stream-thread-test-count-one-changelog", 1), 0L);
@@ -1006,7 +1033,7 @@ public class StreamThreadTest {
internalStreamsBuilder.stream(Collections.singleton("topic"), consumed)
.groupByKey().count(Materialized.<Object, Long, KeyValueStore<Bytes, byte[]>>as("count"));
- final StreamThread thread = createStreamThread("cliendId", config, false);
+ final StreamThread thread = createStreamThread("clientId", config, false);
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
final MockConsumer<byte[], byte[]> mockRestoreConsumer = (MockConsumer<byte[], byte[]>) thread.restoreConsumer;
@@ -1017,26 +1044,33 @@ public class StreamThreadTest {
activeTasks.put(new TaskId(0, 0), topicPartitionSet);
thread.taskManager().setAssignmentMetadata(activeTasks, Collections.<TaskId, Set<TopicPartition>>emptyMap());
- mockConsumer.updatePartitions("topic", new ArrayList<PartitionInfo>() {
- {
- add(new PartitionInfo("topic",
+ mockConsumer.updatePartitions(
+ "topic",
+ singletonList(
+ new PartitionInfo(
+ "topic",
0,
null,
new Node[0],
- new Node[0]));
- }
- });
+ new Node[0]
+ )
+ )
+ );
mockConsumer.updateBeginningOffsets(Collections.singletonMap(topicPartition, 0L));
- mockRestoreConsumer.updatePartitions("stream-thread-test-count-changelog", new ArrayList<PartitionInfo>() {
- {
- add(new PartitionInfo("stream-thread-test-count-changelog",
+ mockRestoreConsumer.updatePartitions(
+ "stream-thread-test-count-changelog",
+ singletonList(
+ new PartitionInfo(
+ "stream-thread-test-count-changelog",
0,
null,
new Node[0],
- new Node[0]));
- }
- });
+ new Node[0]
+ )
+ )
+ );
+
final TopicPartition changelogPartition = new TopicPartition("stream-thread-test-count-changelog", 0);
final Set<TopicPartition> changelogPartitionSet = Collections.singleton(changelogPartition);
mockRestoreConsumer.updateBeginningOffsets(Collections.singletonMap(changelogPartition, 0L));
@@ -1138,7 +1172,8 @@ public class StreamThreadTest {
assertEquals(6.0, metrics.metric(skippedTotalMetric).metricValue());
}
- private void assertThreadMetadataHasEmptyTasksWithState(ThreadMetadata metadata, StreamThread.State state) {
+ private void assertThreadMetadataHasEmptyTasksWithState(final ThreadMetadata metadata,
+ final StreamThread.State state) {
assertEquals(state.name(), metadata.threadState());
assertTrue(metadata.activeTasks().isEmpty());
assertTrue(metadata.standbyTasks().isEmpty());
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java
index 7b16246..ee8abd0 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java
@@ -32,57 +32,57 @@ public class StreamsMetricsImplTest {
@Test(expected = NullPointerException.class)
public void testNullMetrics() {
- String groupName = "doesNotMatter";
- Map<String, String> tags = new HashMap<>();
- StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(null, groupName, tags);
+ final String groupName = "doesNotMatter";
+ final Map<String, String> tags = new HashMap<>();
+ new StreamsMetricsImpl(null, groupName, tags);
}
@Test(expected = NullPointerException.class)
public void testRemoveNullSensor() {
- String groupName = "doesNotMatter";
- Map<String, String> tags = new HashMap<>();
- StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
+ final String groupName = "doesNotMatter";
+ final Map<String, String> tags = new HashMap<>();
+ final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
streamsMetrics.removeSensor(null);
}
@Test
public void testRemoveSensor() {
- String groupName = "doesNotMatter";
- String sensorName = "sensor1";
- String scope = "scope";
- String entity = "entity";
- String operation = "put";
- Map<String, String> tags = new HashMap<>();
- StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
-
- Sensor sensor1 = streamsMetrics.addSensor(sensorName, Sensor.RecordingLevel.DEBUG);
+ final String groupName = "doesNotMatter";
+ final String sensorName = "sensor1";
+ final String scope = "scope";
+ final String entity = "entity";
+ final String operation = "put";
+ final Map<String, String> tags = new HashMap<>();
+ final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
+
+ final Sensor sensor1 = streamsMetrics.addSensor(sensorName, Sensor.RecordingLevel.DEBUG);
streamsMetrics.removeSensor(sensor1);
- Sensor sensor1a = streamsMetrics.addSensor(sensorName, Sensor.RecordingLevel.DEBUG, sensor1);
+ final Sensor sensor1a = streamsMetrics.addSensor(sensorName, Sensor.RecordingLevel.DEBUG, sensor1);
streamsMetrics.removeSensor(sensor1a);
- Sensor sensor2 = streamsMetrics.addLatencyAndThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG);
+ final Sensor sensor2 = streamsMetrics.addLatencyAndThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG);
streamsMetrics.removeSensor(sensor2);
- Sensor sensor3 = streamsMetrics.addThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG);
+ final Sensor sensor3 = streamsMetrics.addThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG);
streamsMetrics.removeSensor(sensor3);
}
@Test
public void testLatencyMetrics() {
- String groupName = "doesNotMatter";
- String scope = "scope";
- String entity = "entity";
- String operation = "put";
- Map<String, String> tags = new HashMap<>();
- StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
+ final String groupName = "doesNotMatter";
+ final String scope = "scope";
+ final String entity = "entity";
+ final String operation = "put";
+ final Map<String, String> tags = new HashMap<>();
+ final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
- Sensor sensor1 = streamsMetrics.addLatencyAndThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG);
+ final Sensor sensor1 = streamsMetrics.addLatencyAndThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG);
Map<MetricName, ? extends Metric> metrics = streamsMetrics.metrics();
// 2 meters and 4 non-meter metrics plus a common metric that keeps track of total registered metrics in Metrics() constructor
- int meterMetricsCount = 2; // Each Meter is a combination of a Rate and a Total
- int otherMetricsCount = 4;
+ final int meterMetricsCount = 2; // Each Meter is a combination of a Rate and a Total
+ final int otherMetricsCount = 4;
assertEquals(meterMetricsCount * 2 + otherMetricsCount + 1, metrics.size());
streamsMetrics.removeSensor(sensor1);
@@ -92,17 +92,17 @@ public class StreamsMetricsImplTest {
@Test
public void testThroughputMetrics() {
- String groupName = "doesNotMatter";
- String scope = "scope";
- String entity = "entity";
- String operation = "put";
- Map<String, String> tags = new HashMap<>();
- StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
+ final String groupName = "doesNotMatter";
+ final String scope = "scope";
+ final String entity = "entity";
+ final String operation = "put";
+ final Map<String, String> tags = new HashMap<>();
+ final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
- Sensor sensor1 = streamsMetrics.addThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG);
+ final Sensor sensor1 = streamsMetrics.addThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG);
Map<MetricName, ? extends Metric> metrics = streamsMetrics.metrics();
- int meterMetricsCount = 2; // Each Meter is a combination of a Rate and a Total
+ final int meterMetricsCount = 2; // Each Meter is a combination of a Rate and a Total
// 2 meter metrics plus a common metric that keeps track of total registered metrics in Metrics() constructor
assertEquals(meterMetricsCount * 2 + 1, metrics.size());
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
index fc810e3..ff1efc9 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
@@ -53,7 +53,7 @@ import java.util.Set;
* all entries written to the Kafka topic by the store during {@link KeyValueStore#flush()} are captured for testing purposes.
* This class simplifies testing of various {@link KeyValueStore} instances, especially those that use
* {@link org.apache.kafka.streams.state.internals.MeteredKeyValueStore} to monitor and write its entries to the Kafka topic.
- * <p>
+ *
* <h2>Basic usage</h2>
* This component can be used to help test a {@link KeyValueStore}'s ability to read and write entries.
*
@@ -94,7 +94,7 @@ import java.util.Set;
* assertEquals(true, driver.flushedEntryRemoved(5));
* </pre>
*
- * <p>
+ *
* <h2>Restoring a store</h2>
* This component can be used to test whether a {@link KeyValueStore} implementation properly
* {@link ProcessorContext#register(StateStore, boolean, StateRestoreCallback) registers itself} with the {@link ProcessorContext}, so that
@@ -141,10 +141,10 @@ public class KeyValueStoreTestDriver<K, V> {
* {@code Long.class}, and {@code byte[].class}). This can be used when store is created to rely upon the
* ProcessorContext's default key and value serializers and deserializers.
*
- * @param keyClass the class for the keys; must be one of {@code String.class}, {@code Integer.class},
- * {@code Long.class}, or {@code byte[].class}
+ * @param keyClass the class for the keys; must be one of {@code String.class}, {@code Integer.class},
+ * {@code Long.class}, or {@code byte[].class}
* @param valueClass the class for the values; must be one of {@code String.class}, {@code Integer.class},
- * {@code Long.class}, or {@code byte[].class}
+ * {@code Long.class}, or {@code byte[].class}
* @return the test driver; never null
*/
public static <K, V> KeyValueStoreTestDriver<K, V> create(final Class<K> keyClass, final Class<V> valueClass) {
@@ -158,9 +158,9 @@ public class KeyValueStoreTestDriver<K, V> {
* deserializers. This can be used when store is created to rely upon the ProcessorContext's default key and value serializers
* and deserializers.
*
- * @param keySerializer the key serializer for the {@link ProcessorContext}; may not be null
- * @param keyDeserializer the key deserializer for the {@link ProcessorContext}; may not be null
- * @param valueSerializer the value serializer for the {@link ProcessorContext}; may not be null
+ * @param keySerializer the key serializer for the {@link ProcessorContext}; may not be null
+ * @param keyDeserializer the key deserializer for the {@link ProcessorContext}; may not be null
+ * @param valueSerializer the value serializer for the {@link ProcessorContext}; may not be null
* @param valueDeserializer the value deserializer for the {@link ProcessorContext}; may not be null
* @return the test driver; never null
*/
@@ -195,7 +195,7 @@ public class KeyValueStoreTestDriver<K, V> {
final Long timestamp,
final Serializer<K1> keySerializer,
final Serializer<V1> valueSerializer) {
- // for byte arrays we need to wrap it for comparison
+ // for byte arrays we need to wrap it for comparison
final K keyTest = serdes.keyFrom(keySerializer.serialize(topic, key));
final V valueTest = serdes.valueFrom(valueSerializer.serialize(topic, value));
@@ -215,7 +215,8 @@ public class KeyValueStoreTestDriver<K, V> {
}
};
- File stateDir = TestUtils.tempDirectory();
+ final File stateDir = TestUtils.tempDirectory();
+ //noinspection ResultOfMethodCallIgnored
stateDir.mkdirs();
stateSerdes = serdes;
@@ -294,7 +295,7 @@ public class KeyValueStoreTestDriver<K, V> {
* assertEquals(3, driver.sizeOf(store));
* </pre>
*
- * @param key the key for the entry
+ * @param key the key for the entry
* @param value the value for the entry
* @see #checkForRestoredEntries(KeyValueStore)
*/
@@ -346,7 +347,7 @@ public class KeyValueStoreTestDriver<K, V> {
*/
public int sizeOf(final KeyValueStore<K, V> store) {
int size = 0;
- try (KeyValueIterator<K, V> iterator = store.all()) {
+ try (final KeyValueIterator<K, V> iterator = store.all()) {
while (iterator.hasNext()) {
iterator.next();
++size;
@@ -360,7 +361,7 @@ public class KeyValueStoreTestDriver<K, V> {
*
* @param key the key
* @return the value that was flushed with the key, or {@code null} if no such key was flushed or if the entry with this
- * key was removed upon flush
+ * key was removed upon flush
*/
public V flushedEntryStored(final K key) {
return flushedEntries.get(key);
@@ -371,7 +372,7 @@ public class KeyValueStoreTestDriver<K, V> {
*
* @param key the key
* @return {@code true} if the entry with the given key was removed when flushed, or {@code false} if the entry was not
- * removed when last flushed
+ * removed when last flushed
*/
public boolean flushedEntryRemoved(final K key) {
return flushedRemovals.contains(key);
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
index b3a60a9..554169e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
@@ -53,6 +53,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
+import static java.util.Objects.requireNonNull;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertEquals;
@@ -61,6 +62,7 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
+@SuppressWarnings("PointlessArithmeticExpression")
public class RocksDBWindowStoreTest {
private static final long DEFAULT_CACHE_SIZE_BYTES = 1024 * 1024L;
@@ -80,15 +82,15 @@ public class RocksDBWindowStoreTest {
private final RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTestTask", new LogContext("RocksDBWindowStoreTestTask "), new DefaultProductionExceptionHandler()) {
@Override
public <K1, V1> void send(final String topic,
- K1 key,
- V1 value,
- Integer partition,
- Long timestamp,
- Serializer<K1> keySerializer,
- Serializer<V1> valueSerializer) {
+ final K1 key,
+ final V1 value,
+ final Integer partition,
+ final Long timestamp,
+ final Serializer<K1> keySerializer,
+ final Serializer<V1> valueSerializer) {
changeLog.add(new KeyValue<>(
- keySerializer.serialize(topic, key),
- valueSerializer.serialize(topic, value))
+ keySerializer.serialize(topic, key),
+ valueSerializer.serialize(topic, value))
);
}
};
@@ -99,13 +101,14 @@ public class RocksDBWindowStoreTest {
private WindowStore<Integer, String> createWindowStore(final ProcessorContext context, final boolean retainDuplicates) {
final WindowStore<Integer, String> store = Stores.windowStoreBuilder(
- Stores.persistentWindowStore(windowName,
- retentionPeriod,
- numSegments,
- windowSize,
- retainDuplicates),
- Serdes.Integer(),
- Serdes.String()).build();
+ Stores.persistentWindowStore(
+ windowName,
+ retentionPeriod,
+ numSegments,
+ windowSize,
+ retainDuplicates),
+ Serdes.Integer(),
+ Serdes.String()).build();
store.init(context, store);
return store;
@@ -158,7 +161,7 @@ public class RocksDBWindowStoreTest {
@Test
public void testRangeAndSinglePointFetch() {
windowStore = createWindowStore(context);
- long startTime = segmentSize - 4L;
+ final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context);
@@ -203,7 +206,7 @@ public class RocksDBWindowStoreTest {
// Flush the store and verify all current entries were properly flushed ...
windowStore.flush();
- Map<Integer, Set<String>> entriesByKey = entriesByKey(changeLog, startTime);
+ final Map<Integer, Set<String>> entriesByKey = entriesByKey(changeLog, startTime);
assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
@@ -217,7 +220,7 @@ public class RocksDBWindowStoreTest {
@Test
public void shouldGetAll() {
windowStore = createWindowStore(context);
- long startTime = segmentSize - 4L;
+ final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context);
@@ -228,15 +231,15 @@ public class RocksDBWindowStoreTest {
final KeyValue<Windowed<Integer>, String> five = windowedPair(5, "five", startTime + 5);
assertEquals(
- Utils.mkList(zero, one, two, four, five),
- StreamsTestUtils.toList(windowStore.all())
+ Utils.mkList(zero, one, two, four, five),
+ StreamsTestUtils.toList(windowStore.all())
);
}
@Test
public void shouldFetchAllInTimeRange() {
windowStore = createWindowStore(context);
- long startTime = segmentSize - 4L;
+ final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context);
@@ -247,25 +250,25 @@ public class RocksDBWindowStoreTest {
final KeyValue<Windowed<Integer>, String> five = windowedPair(5, "five", startTime + 5);
assertEquals(
- Utils.mkList(one, two, four),
- StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 4))
+ Utils.mkList(one, two, four),
+ StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 4))
);
assertEquals(
- Utils.mkList(zero, one, two),
- StreamsTestUtils.toList(windowStore.fetchAll(startTime + 0, startTime + 3))
+ Utils.mkList(zero, one, two),
+ StreamsTestUtils.toList(windowStore.fetchAll(startTime + 0, startTime + 3))
);
assertEquals(
- Utils.mkList(one, two, four, five),
- StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 5))
+ Utils.mkList(one, two, four, five),
+ StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 5))
);
}
@Test
public void testFetchRange() {
windowStore = createWindowStore(context);
- long startTime = segmentSize - 4L;
+ final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context);
@@ -293,7 +296,7 @@ public class RocksDBWindowStoreTest {
);
assertEquals(
Utils.mkList(zero, one, two,
- four, five),
+ four, five),
StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 0L - windowSize, startTime + 0L + windowSize + 5L))
);
assertEquals(
@@ -313,7 +316,7 @@ public class RocksDBWindowStoreTest {
@Test
public void testPutAndFetchBefore() {
windowStore = createWindowStore(context);
- long startTime = segmentSize - 4L;
+ final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context);
@@ -345,7 +348,7 @@ public class RocksDBWindowStoreTest {
// Flush the store and verify all current entries were properly flushed ...
windowStore.flush();
- Map<Integer, Set<String>> entriesByKey = entriesByKey(changeLog, startTime);
+ final Map<Integer, Set<String>> entriesByKey = entriesByKey(changeLog, startTime);
assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
@@ -359,7 +362,7 @@ public class RocksDBWindowStoreTest {
@Test
public void testPutAndFetchAfter() {
windowStore = createWindowStore(context);
- long startTime = segmentSize - 4L;
+ final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context);
@@ -391,7 +394,7 @@ public class RocksDBWindowStoreTest {
// Flush the store and verify all current entries were properly flushed ...
windowStore.flush();
- Map<Integer, Set<String>> entriesByKey = entriesByKey(changeLog, startTime);
+ final Map<Integer, Set<String>> entriesByKey = entriesByKey(changeLog, startTime);
assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
@@ -405,7 +408,7 @@ public class RocksDBWindowStoreTest {
@Test
public void testPutSameKeyTimestamp() {
windowStore = createWindowStore(context, true);
- long startTime = segmentSize - 4L;
+ final long startTime = segmentSize - 4L;
context.setRecordContext(createRecordContext(startTime));
windowStore.put(0, "zero");
@@ -425,7 +428,7 @@ public class RocksDBWindowStoreTest {
// Flush the store and verify all current entries were properly flushed ...
windowStore.flush();
- Map<Integer, Set<String>> entriesByKey = entriesByKey(changeLog, startTime);
+ final Map<Integer, Set<String>> entriesByKey = entriesByKey(changeLog, startTime);
assertEquals(Utils.mkSet("zero@0", "zero@0", "zero+@0", "zero++@0"), entriesByKey.get(0));
}
@@ -436,121 +439,156 @@ public class RocksDBWindowStoreTest {
// to validate segments
final Segments segments = new Segments(windowName, retentionPeriod, numSegments);
- long startTime = segmentSize * 2;
- long incr = segmentSize / 2;
+ final long startTime = segmentSize * 2;
+ final long increment = segmentSize / 2;
context.setRecordContext(createRecordContext(startTime));
windowStore.put(0, "zero");
assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir));
- context.setRecordContext(createRecordContext(startTime + incr));
+ context.setRecordContext(createRecordContext(startTime + increment));
windowStore.put(1, "one");
assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir));
- context.setRecordContext(createRecordContext(startTime + incr * 2));
+ context.setRecordContext(createRecordContext(startTime + increment * 2));
windowStore.put(2, "two");
- assertEquals(Utils.mkSet(segments.segmentName(2),
- segments.segmentName(3)), segmentDirs(baseDir));
+ assertEquals(
+ Utils.mkSet(
+ segments.segmentName(2),
+ segments.segmentName(3)
+ ),
+ segmentDirs(baseDir)
+ );
- context.setRecordContext(createRecordContext(startTime + incr * 4));
+ context.setRecordContext(createRecordContext(startTime + increment * 4));
windowStore.put(4, "four");
- assertEquals(Utils.mkSet(segments.segmentName(2),
- segments.segmentName(3),
- segments.segmentName(4)), segmentDirs(baseDir));
+ assertEquals(
+ Utils.mkSet(
+ segments.segmentName(2),
+ segments.segmentName(3),
+ segments.segmentName(4)
+ ),
+ segmentDirs(baseDir)
+ );
- context.setRecordContext(createRecordContext(startTime + incr * 5));
+ context.setRecordContext(createRecordContext(startTime + increment * 5));
windowStore.put(5, "five");
- assertEquals(Utils.mkSet(segments.segmentName(2),
- segments.segmentName(3),
- segments.segmentName(4)), segmentDirs(baseDir));
+ assertEquals(
+ Utils.mkSet(
+ segments.segmentName(2),
+ segments.segmentName(3),
+ segments.segmentName(4)
+ ),
+ segmentDirs(baseDir)
+ );
assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
- assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
- assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
- assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
- assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
+ assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
+ assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
+ assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
+ assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
- context.setRecordContext(createRecordContext(startTime + incr * 6));
+ context.setRecordContext(createRecordContext(startTime + increment * 6));
windowStore.put(6, "six");
- assertEquals(Utils.mkSet(segments.segmentName(3),
- segments.segmentName(4),
- segments.segmentName(5)), segmentDirs(baseDir));
+ assertEquals(
+ Utils.mkSet(
+ segments.segmentName(3),
+ segments.segmentName(4),
+ segments.segmentName(5)
+ ),
+ segmentDirs(baseDir)
+ );
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
- assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
- assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
- assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
- assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
+ assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
+ assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
+ assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
+ assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
- context.setRecordContext(createRecordContext(startTime + incr * 7));
+ context.setRecordContext(createRecordContext(startTime + increment * 7));
windowStore.put(7, "seven");
- assertEquals(Utils.mkSet(segments.segmentName(3),
- segments.segmentName(4),
- segments.segmentName(5)), segmentDirs(baseDir));
+ assertEquals(
+ Utils.mkSet(
+ segments.segmentName(3),
+ segments.segmentName(4),
+ segments.segmentName(5)
+ ),
+ segmentDirs(baseDir)
+ );
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
- assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
- assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
- assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
- assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
- assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
-
- context.setRecordContext(createRecordContext(startTime + incr * 8));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
+ assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
+ assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
+ assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
+ assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
+ assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
+
+ context.setRecordContext(createRecordContext(startTime + increment * 8));
windowStore.put(8, "eight");
- assertEquals(Utils.mkSet(segments.segmentName(4),
- segments.segmentName(5),
- segments.segmentName(6)), segmentDirs(baseDir));
+ assertEquals(
+ Utils.mkSet(
+ segments.segmentName(4),
+ segments.segmentName(5),
+ segments.segmentName(6)
+ ),
+ segmentDirs(baseDir)
+ );
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
- assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
- assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
- assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
- assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
- assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
+ assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
+ assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
+ assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
+ assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
+ assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
// check segment directories
windowStore.flush();
- assertEquals(Utils.mkSet(segments.segmentName(4),
- segments.segmentName(5),
- segments.segmentName(6)), segmentDirs(baseDir));
+ assertEquals(
+ Utils.mkSet(
+ segments.segmentName(4),
+ segments.segmentName(5),
+ segments.segmentName(6)
+ ),
+ segmentDirs(baseDir)
+ );
}
@Test
public void testRestore() throws IOException {
- long startTime = segmentSize * 2;
- long incr = segmentSize / 2;
+ final long startTime = segmentSize * 2;
+ final long increment = segmentSize / 2;
windowStore = createWindowStore(context);
context.setRecordContext(createRecordContext(startTime));
windowStore.put(0, "zero");
- context.setRecordContext(createRecordContext(startTime + incr));
+ context.setRecordContext(createRecordContext(startTime + increment));
windowStore.put(1, "one");
- context.setRecordContext(createRecordContext(startTime + incr * 2));
+ context.setRecordContext(createRecordContext(startTime + increment * 2));
windowStore.put(2, "two");
- context.setRecordContext(createRecordContext(startTime + incr * 3));
+ context.setRecordContext(createRecordContext(startTime + increment * 3));
windowStore.put(3, "three");
- context.setRecordContext(createRecordContext(startTime + incr * 4));
+ context.setRecordContext(createRecordContext(startTime + increment * 4));
windowStore.put(4, "four");
- context.setRecordContext(createRecordContext(startTime + incr * 5));
+ context.setRecordContext(createRecordContext(startTime + increment * 5));
windowStore.put(5, "five");
- context.setRecordContext(createRecordContext(startTime + incr * 6));
+ context.setRecordContext(createRecordContext(startTime + increment * 6));
windowStore.put(6, "six");
- context.setRecordContext(createRecordContext(startTime + incr * 7));
+ context.setRecordContext(createRecordContext(startTime + increment * 7));
windowStore.put(7, "seven");
- context.setRecordContext(createRecordContext(startTime + incr * 8));
+ context.setRecordContext(createRecordContext(startTime + increment * 8));
windowStore.put(8, "eight");
windowStore.flush();
@@ -561,32 +599,32 @@ public class RocksDBWindowStoreTest {
windowStore = createWindowStore(context);
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
context.restore(windowName, changeLog);
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
- assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
- assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
- assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
- assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
- assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
- assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
+ assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
+ assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
+ assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
+ assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
+ assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
+ assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
// check segment directories
windowStore.flush();
assertEquals(
- Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
+ segmentDirs(baseDir)
);
}
@@ -597,23 +635,23 @@ public class RocksDBWindowStoreTest {
context.setRecordContext(createRecordContext(0));
windowStore.put(0, "v");
assertEquals(
- Utils.mkSet(segments.segmentName(0L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(0L)),
+ segmentDirs(baseDir)
);
context.setRecordContext(createRecordContext(59999));
windowStore.put(0, "v");
windowStore.put(0, "v");
assertEquals(
- Utils.mkSet(segments.segmentName(0L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(0L)),
+ segmentDirs(baseDir)
);
context.setRecordContext(createRecordContext(60000));
windowStore.put(0, "v");
assertEquals(
- Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
+ segmentDirs(baseDir)
);
WindowStoreIterator iter;
@@ -628,8 +666,8 @@ public class RocksDBWindowStoreTest {
assertEquals(4, fetchedCount);
assertEquals(
- Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
+ segmentDirs(baseDir)
);
context.setRecordContext(createRecordContext(180000));
@@ -644,8 +682,8 @@ public class RocksDBWindowStoreTest {
assertEquals(2, fetchedCount);
assertEquals(
- Utils.mkSet(segments.segmentName(1L), segments.segmentName(3L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(1L), segments.segmentName(3L)),
+ segmentDirs(baseDir)
);
context.setRecordContext(createRecordContext(300000));
@@ -660,15 +698,16 @@ public class RocksDBWindowStoreTest {
assertEquals(1, fetchedCount);
assertEquals(
- Utils.mkSet(segments.segmentName(3L), segments.segmentName(5L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(3L), segments.segmentName(5L)),
+ segmentDirs(baseDir)
);
}
+ @SuppressWarnings("ResultOfMethodCallIgnored")
@Test
public void testInitialLoading() {
- File storeDir = new File(baseDir, windowName);
+ final File storeDir = new File(baseDir, windowName);
windowStore = createWindowStore(context);
@@ -684,19 +723,19 @@ public class RocksDBWindowStoreTest {
windowStore = createWindowStore(context);
assertEquals(
- Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
+ segmentDirs(baseDir)
);
- try (WindowStoreIterator iter = windowStore.fetch(0, 0L, 1000000L)) {
+ try (final WindowStoreIterator iter = windowStore.fetch(0, 0L, 1000000L)) {
while (iter.hasNext()) {
iter.next();
}
}
assertEquals(
- Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
- segmentDirs(baseDir)
+ Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
+ segmentDirs(baseDir)
);
}
@@ -712,16 +751,17 @@ public class RocksDBWindowStoreTest {
assertTrue(iterator.hasNext());
windowStore.close();
try {
+ //noinspection ResultOfMethodCallIgnored
iterator.hasNext();
fail("should have thrown InvalidStateStoreException on closed store");
- } catch (InvalidStateStoreException e) {
+ } catch (final InvalidStateStoreException e) {
// ok
}
try {
iterator.next();
fail("should have thrown InvalidStateStoreException on closed store");
- } catch (InvalidStateStoreException e) {
+ } catch (final InvalidStateStoreException e) {
// ok
}
}
@@ -731,13 +771,14 @@ public class RocksDBWindowStoreTest {
final long windowSize = 0x7a00000000000000L;
final long retentionPeriod = 0x7a00000000000000L;
final WindowStore<String, String> windowStore = Stores.windowStoreBuilder(
- Stores.persistentWindowStore(windowName,
- retentionPeriod,
- 2,
- windowSize,
- true),
- Serdes.String(),
- Serdes.String()).build();
+ Stores.persistentWindowStore(
+ windowName,
+ retentionPeriod,
+ 2,
+ windowSize,
+ true),
+ Serdes.String(),
+ Serdes.String()).build();
windowStore.init(context, windowStore);
@@ -796,13 +837,13 @@ public class RocksDBWindowStoreTest {
}
@Test
- public void shouldNoNullPointerWhenSerdeDoesntHandleNull() {
+ public void shouldNoNullPointerWhenSerdeDoesNotHandleNull() {
windowStore = new RocksDBWindowStore<>(
- new RocksDBSegmentedBytesStore(windowName, retentionPeriod, numSegments, new WindowKeySchema()),
- Serdes.Integer(),
- new SerdeThatDoesntHandleNull(),
- false,
- windowSize);
+ new RocksDBSegmentedBytesStore(windowName, retentionPeriod, numSegments, new WindowKeySchema()),
+ Serdes.Integer(),
+ new SerdeThatDoesntHandleNull(),
+ false,
+ windowSize);
windowStore.init(context, windowStore);
assertNull(windowStore.fetch(1, 0));
@@ -811,13 +852,14 @@ public class RocksDBWindowStoreTest {
@Test
public void shouldFetchAndIterateOverExactBinaryKeys() {
final WindowStore<Bytes, String> windowStore = Stores.windowStoreBuilder(
- Stores.persistentWindowStore(windowName,
- 60000,
- 2,
- 60000,
- true),
- Serdes.Bytes(),
- Serdes.String()).build();
+ Stores.persistentWindowStore(
+ windowName,
+ 60000,
+ 2,
+ 60000,
+ true),
+ Serdes.Bytes(),
+ Serdes.String()).build();
windowStore.init(context, windowStore);
@@ -842,7 +884,9 @@ public class RocksDBWindowStoreTest {
assertThat(toList(windowStore.fetch(key3, 0, Long.MAX_VALUE)), equalTo(expectedKey3));
}
- private void putFirstBatch(final WindowStore<Integer, String> store, final long startTime, final InternalMockProcessorContext context) {
+ private void putFirstBatch(final WindowStore<Integer, String> store,
+ @SuppressWarnings("SameParameterValue") final long startTime,
+ final InternalMockProcessorContext context) {
context.setRecordContext(createRecordContext(startTime));
store.put(0, "zero");
context.setRecordContext(createRecordContext(startTime + 1L));
@@ -855,7 +899,9 @@ public class RocksDBWindowStoreTest {
store.put(5, "five");
}
- private void putSecondBatch(final WindowStore<Integer, String> store, final long startTime, InternalMockProcessorContext context) {
+ private void putSecondBatch(final WindowStore<Integer, String> store,
+ @SuppressWarnings("SameParameterValue") final long startTime,
+ final InternalMockProcessorContext context) {
context.setRecordContext(createRecordContext(startTime + 3L));
store.put(2, "two+1");
context.setRecordContext(createRecordContext(startTime + 4L));
@@ -870,24 +916,25 @@ public class RocksDBWindowStoreTest {
store.put(2, "two+6");
}
- private <E> List<E> toList(WindowStoreIterator<E> iterator) {
- ArrayList<E> list = new ArrayList<>();
+ private <E> List<E> toList(final WindowStoreIterator<E> iterator) {
+ final ArrayList<E> list = new ArrayList<>();
while (iterator.hasNext()) {
list.add(iterator.next().value);
}
return list;
}
- private Set<String> segmentDirs(File baseDir) {
- File windowDir = new File(baseDir, windowName);
+ private Set<String> segmentDirs(final File baseDir) {
+ final File windowDir = new File(baseDir, windowName);
- return new HashSet<>(Arrays.asList(windowDir.list()));
+ return new HashSet<>(Arrays.asList(requireNonNull(windowDir.list())));
}
- private Map<Integer, Set<String>> entriesByKey(List<KeyValue<byte[], byte[]>> changeLog, long startTime) {
- HashMap<Integer, Set<String>> entriesByKey = new HashMap<>();
+ private Map<Integer, Set<String>> entriesByKey(final List<KeyValue<byte[], byte[]>> changeLog,
+ @SuppressWarnings("SameParameterValue") final long startTime) {
+ final HashMap<Integer, Set<String>> entriesByKey = new HashMap<>();
- for (KeyValue<byte[], byte[]> entry : changeLog) {
+ for (final KeyValue<byte[], byte[]> entry : changeLog) {
final long timestamp = WindowKeySchema.extractStoreTimestamp(entry.key);
final Integer key = WindowKeySchema.extractStoreKey(entry.key, serdes);
@@ -904,11 +951,11 @@ public class RocksDBWindowStoreTest {
return entriesByKey;
}
- private <K, V> KeyValue<Windowed<K>, V> windowedPair(K key, V value, long timestamp) {
+ private <K, V> KeyValue<Windowed<K>, V> windowedPair(final K key, final V value, final long timestamp) {
return windowedPair(key, value, timestamp, windowSize);
}
- private static <K, V> KeyValue<Windowed<K>, V> windowedPair(K key, V value, long timestamp, long windowSize) {
+ private static <K, V> KeyValue<Windowed<K>, V> windowedPair(final K key, final V value, final long timestamp, final long windowSize) {
return KeyValue.pair(new Windowed<>(key, WindowKeySchema.timeWindowForSize(timestamp, windowSize)), value);
}
}
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
index c62b09b..dcc2305 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
@@ -40,29 +40,29 @@ public class StoreChangeLoggerTest {
private final Map<Integer, String> logged = new HashMap<>();
private final InternalMockProcessorContext context = new InternalMockProcessorContext(StateSerdes.withBuiltinTypes(topic, Integer.class, String.class),
- new RecordCollectorImpl(null, "StoreChangeLoggerTest", new LogContext("StoreChangeLoggerTest "), new DefaultProductionExceptionHandler()) {
- @Override
- public <K1, V1> void send(final String topic,
- final K1 key,
- final V1 value,
- final Integer partition,
- final Long timestamp,
- final Serializer<K1> keySerializer,
- final Serializer<V1> valueSerializer) {
- logged.put((Integer) key, (String) value);
- }
+ new RecordCollectorImpl(null, "StoreChangeLoggerTest", new LogContext("StoreChangeLoggerTest "), new DefaultProductionExceptionHandler()) {
+ @Override
+ public <K1, V1> void send(final String topic,
+ final K1 key,
+ final V1 value,
+ final Integer partition,
+ final Long timestamp,
+ final Serializer<K1> keySerializer,
+ final Serializer<V1> valueSerializer) {
+ logged.put((Integer) key, (String) value);
+ }
- @Override
- public <K1, V1> void send(final String topic,
- final K1 key,
- final V1 value,
- final Long timestamp,
- final Serializer<K1> keySerializer,
- final Serializer<V1> valueSerializer,
- final StreamPartitioner<? super K1, ? super V1> partitioner) {
- throw new UnsupportedOperationException();
- }
+ @Override
+ public <K1, V1> void send(final String topic,
+ final K1 key,
+ final V1 value,
+ final Long timestamp,
+ final Serializer<K1> keySerializer,
+ final Serializer<V1> valueSerializer,
+ final StreamPartitioner<? super K1, ? super V1> partitioner) {
+ throw new UnsupportedOperationException();
}
+ }
);
private final StoreChangeLogger<Integer, String> changeLogger = new StoreChangeLogger<>(topic, context, StateSerdes.withBuiltinTypes(topic, Integer.class, String.class));
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
index 85c282c..a9c47b5 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
@@ -61,7 +61,6 @@ import static org.junit.Assert.assertEquals;
public class StreamThreadStateStoreProviderTest {
private StreamTask taskOne;
- private StreamTask taskTwo;
private StreamThreadStateStoreProvider provider;
private StateDirectory stateDirectory;
private File stateDir;
@@ -71,19 +70,19 @@ public class StreamThreadStateStoreProviderTest {
@SuppressWarnings("deprecation")
@Before
- public void before() throws IOException {
+ public void before() {
final TopologyBuilder builder = new TopologyBuilder();
builder.addSource("the-source", topicName);
builder.addProcessor("the-processor", new MockProcessorSupplier(), "the-source");
builder.addStateStore(Stores.create("kv-store")
- .withStringKeys()
- .withStringValues().inMemory().build(), "the-processor");
+ .withStringKeys()
+ .withStringValues().inMemory().build(), "the-processor");
builder.addStateStore(Stores.create("window-store")
- .withStringKeys()
- .withStringValues()
- .persistent()
- .windowed(10, 10, 2, false).build(), "the-processor");
+ .withStringKeys()
+ .withStringValues()
+ .persistent()
+ .windowed(10, 10, 2, false).build(), "the-processor");
final Properties properties = new Properties();
final String applicationId = "applicationId";
@@ -99,18 +98,17 @@ public class StreamThreadStateStoreProviderTest {
builder.setApplicationId(applicationId);
final ProcessorTopology topology = builder.build(null);
+
tasks = new HashMap<>();
stateDirectory = new StateDirectory(streamsConfig, new MockTime());
- taskOne = createStreamsTask(applicationId, streamsConfig, clientSupplier, topology,
- new TaskId(0, 0));
+
+ taskOne = createStreamsTask(streamsConfig, clientSupplier, topology, new TaskId(0, 0));
taskOne.initializeStateStores();
- tasks.put(new TaskId(0, 0),
- taskOne);
- taskTwo = createStreamsTask(applicationId, streamsConfig, clientSupplier, topology,
- new TaskId(0, 1));
+ tasks.put(new TaskId(0, 0), taskOne);
+
+ final StreamTask taskTwo = createStreamsTask(streamsConfig, clientSupplier, topology, new TaskId(0, 1));
taskTwo.initializeStateStores();
- tasks.put(new TaskId(0, 1),
- taskTwo);
+ tasks.put(new TaskId(0, 1), taskTwo);
threadMock = EasyMock.createNiceMock(StreamThread.class);
provider = new StreamThreadStateStoreProvider(threadMock);
@@ -121,7 +119,7 @@ public class StreamThreadStateStoreProviderTest {
public void cleanUp() throws IOException {
Utils.delete(stateDir);
}
-
+
@Test
public void shouldFindKeyValueStores() {
mockThread(true);
@@ -164,8 +162,10 @@ public class StreamThreadStateStoreProviderTest {
@Test
public void shouldReturnEmptyListIfStoreExistsButIsNotOfTypeValueStore() {
mockThread(true);
- assertEquals(Collections.emptyList(), provider.stores("window-store",
- QueryableStoreTypes.keyValueStore()));
+ assertEquals(
+ Collections.emptyList(),
+ provider.stores("window-store", QueryableStoreTypes.keyValueStore())
+ );
}
@Test(expected = InvalidStateStoreException.class)
@@ -174,8 +174,7 @@ public class StreamThreadStateStoreProviderTest {
provider.stores("kv-store", QueryableStoreTypes.keyValueStore());
}
- private StreamTask createStreamsTask(final String applicationId,
- final StreamsConfig streamsConfig,
+ private StreamTask createStreamsTask(final StreamsConfig streamsConfig,
final MockClientSupplier clientSupplier,
final ProcessorTopology topology,
final TaskId taskId) {
@@ -190,8 +189,8 @@ public class StreamThreadStateStoreProviderTest {
stateDirectory,
null,
new MockTime(),
- clientSupplier.getProducer(new HashMap<String, Object>())) {
-
+ clientSupplier.getProducer(new HashMap<String, Object>())
+ ) {
@Override
protected void updateOffsetLimits() {}
};
@@ -205,28 +204,21 @@ public class StreamThreadStateStoreProviderTest {
private void configureRestoreConsumer(final MockClientSupplier clientSupplier,
final String topic) {
- clientSupplier.restoreConsumer
- .updatePartitions(topic,
- Arrays.asList(
- new PartitionInfo(topic, 0, null,
- null, null),
- new PartitionInfo(topic, 1, null,
- null, null)));
+ final List<PartitionInfo> partitions = Arrays.asList(
+ new PartitionInfo(topic, 0, null, null, null),
+ new PartitionInfo(topic, 1, null, null, null)
+ );
+ clientSupplier.restoreConsumer.updatePartitions(topic, partitions);
final TopicPartition tp1 = new TopicPartition(topic, 0);
final TopicPartition tp2 = new TopicPartition(topic, 1);
- clientSupplier.restoreConsumer
- .assign(Arrays.asList(
- tp1,
- tp2));
+ clientSupplier.restoreConsumer.assign(Arrays.asList(tp1, tp2));
final Map<TopicPartition, Long> offsets = new HashMap<>();
offsets.put(tp1, 0L);
offsets.put(tp2, 0L);
- clientSupplier.restoreConsumer
- .updateBeginningOffsets(offsets);
- clientSupplier.restoreConsumer
- .updateEndOffsets(offsets);
+ clientSupplier.restoreConsumer.updateBeginningOffsets(offsets);
+ clientSupplier.restoreConsumer.updateEndOffsets(offsets);
}
}
diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
index 39183d9..aebb849 100644
--- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
@@ -45,7 +45,7 @@ import java.util.Set;
public class KStreamTestDriver extends ExternalResource {
- private static final long DEFAULT_CACHE_SIZE_BYTES = 1 * 1024 * 1024L;
+ private static final long DEFAULT_CACHE_SIZE_BYTES = 1024 * 1024L;
private ProcessorTopology topology;
private InternalMockProcessorContext context;
@@ -231,7 +231,7 @@ public class KStreamTestDriver extends ExternalResource {
final List<ProcessorNode> nodes = topology.processors();
- for (final ProcessorNode node: nodes) {
+ for (final ProcessorNode node : nodes) {
names.add(node.name());
}
@@ -241,7 +241,7 @@ public class KStreamTestDriver extends ExternalResource {
public ProcessorNode processor(final String name) {
final List<ProcessorNode> nodes = topology.processors();
- for (final ProcessorNode node: nodes) {
+ for (final ProcessorNode node : nodes) {
if (node.name().equals(name)) {
return node;
}
diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
index 82c39ee..fecf7e4 100644
--- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
@@ -32,7 +32,6 @@ import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
-import org.apache.kafka.streams.InternalTopologyAccessor;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsMetrics;
import org.apache.kafka.streams.Topology;
@@ -158,25 +157,15 @@ public class ProcessorTopologyTestDriver {
private GlobalStateUpdateTask globalStateTask;
/**
- * Create a new test diver instance
- * @param config the stream configuration for the topology
- * @param topology the {@link Topology} whose {@link InternalTopologyBuilder} will
- * be use to create the topology instance.
- */
- public ProcessorTopologyTestDriver(final StreamsConfig config,
- final Topology topology) {
- this(config, InternalTopologyAccessor.getInternalTopologyBuilder(topology));
- }
-
- /**
* Create a new test driver instance.
- * @param config the stream configuration for the topology
+ *
+ * @param config the stream configuration for the topology
* @param builder the topology builder that will be used to create the topology instance
*/
public ProcessorTopologyTestDriver(final StreamsConfig config,
final InternalTopologyBuilder builder) {
topology = builder.setApplicationId(APPLICATION_ID).build(null);
- final ProcessorTopology globalTopology = builder.buildGlobalStateTopology();
+ final ProcessorTopology globalTopology = builder.buildGlobalStateTopology();
// Set up the consumer and producer ...
final Consumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
@@ -218,36 +207,42 @@ public class ProcessorTopologyTestDriver {
globalPartitionsByTopic.put(topicName, partition);
offsetsByTopicPartition.put(partition, new AtomicLong());
}
- final GlobalStateManagerImpl stateManager = new GlobalStateManagerImpl(new LogContext("mock "),
- globalTopology,
- globalConsumer,
- stateDirectory,
- stateRestoreListener,
- config);
+ final GlobalStateManagerImpl stateManager = new GlobalStateManagerImpl(
+ new LogContext("mock "),
+ globalTopology,
+ globalConsumer,
+ stateDirectory,
+ stateRestoreListener,
+ config);
final GlobalProcessorContextImpl globalProcessorContext = new GlobalProcessorContextImpl(config, stateManager, streamsMetrics, cache);
stateManager.setGlobalProcessorContext(globalProcessorContext);
- globalStateTask = new GlobalStateUpdateTask(globalTopology,
- globalProcessorContext,
- stateManager,
- new LogAndContinueExceptionHandler(),
- new LogContext());
+ globalStateTask = new GlobalStateUpdateTask(
+ globalTopology,
+ globalProcessorContext,
+ stateManager,
+ new LogAndContinueExceptionHandler(),
+ new LogContext());
globalStateTask.initialize();
}
if (!partitionsByTopic.isEmpty()) {
- task = new StreamTask(TASK_ID,
- partitionsByTopic.values(),
- topology,
- consumer,
- new StoreChangelogReader(
- createRestoreConsumer(topology.storeToChangelogTopic()),
- new MockStateRestoreListener(),
- new LogContext("topology-test-driver ")),
- config,
- streamsMetrics, stateDirectory,
- cache,
- new MockTime(),
- producer);
+ task = new StreamTask(
+ TASK_ID,
+ partitionsByTopic.values(),
+ topology,
+ consumer,
+ new StoreChangelogReader(
+ createRestoreConsumer(topology.storeToChangelogTopic()),
+ new MockStateRestoreListener(),
+ new LogContext("topology-test-driver ")
+ ),
+ config,
+ streamsMetrics,
+ stateDirectory,
+ cache,
+ new MockTime(),
+ producer
+ );
task.initializeStateStores();
task.initializeTopology();
}
@@ -257,8 +252,8 @@ public class ProcessorTopologyTestDriver {
* Send an input message with the given key, value and timestamp on the specified topic to the topology, and then commit the messages.
*
* @param topicName the name of the topic on which the message is to be sent
- * @param key the raw message key
- * @param value the raw message value
+ * @param key the raw message key
+ * @param value the raw message value
* @param timestamp the raw message timestamp
*/
public void process(final String topicName,
@@ -307,8 +302,8 @@ public class ProcessorTopologyTestDriver {
* Send an input message with the given key and value on the specified topic to the topology.
*
* @param topicName the name of the topic on which the message is to be sent
- * @param key the raw message key
- * @param value the raw message value
+ * @param key the raw message key
+ * @param value the raw message value
*/
public void process(final String topicName,
final byte[] key,
@@ -319,10 +314,10 @@ public class ProcessorTopologyTestDriver {
/**
* Send an input message with the given key and value on the specified topic to the topology.
*
- * @param topicName the name of the topic on which the message is to be sent
- * @param key the raw message key
- * @param value the raw message value
- * @param keySerializer the serializer for the key
+ * @param topicName the name of the topic on which the message is to be sent
+ * @param key the raw message key
+ * @param value the raw message value
+ * @param keySerializer the serializer for the key
* @param valueSerializer the serializer for the value
*/
public <K, V> void process(final String topicName,
@@ -336,12 +331,12 @@ public class ProcessorTopologyTestDriver {
/**
* Send an input message with the given key and value and timestamp on the specified topic to the topology.
*
- * @param topicName the name of the topic on which the message is to be sent
- * @param key the raw message key
- * @param value the raw message value
- * @param keySerializer the serializer for the key
+ * @param topicName the name of the topic on which the message is to be sent
+ * @param key the raw message key
+ * @param value the raw message value
+ * @param keySerializer the serializer for the key
* @param valueSerializer the serializer for the value
- * @param timestamp the raw message timestamp
+ * @param timestamp the raw message timestamp
*/
public <K, V> void process(final String topicName,
final K key,
@@ -371,8 +366,8 @@ public class ProcessorTopologyTestDriver {
* Read the next record from the given topic. These records were output by the topology during the previous calls to
* {@link #process(String, byte[], byte[])}.
*
- * @param topic the name of the topic
- * @param keyDeserializer the deserializer for the key type
+ * @param topic the name of the topic
+ * @param keyDeserializer the deserializer for the key type
* @param valueDeserializer the deserializer for the value type
* @return the next record on that topic, or null if there is no record available
*/
@@ -464,7 +459,7 @@ public class ProcessorTopologyTestDriver {
}
};
// For each store ...
- for (final Map.Entry<String, String> storeAndTopic: storeToChangelogTopic.entrySet()) {
+ for (final Map.Entry<String, String> storeAndTopic : storeToChangelogTopic.entrySet()) {
final String topicName = storeAndTopic.getValue();
// Set up the restore-state topic ...
// consumer.subscribe(new TopicPartition(topicName, 1));
diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java
index 076c596..67304e1 100644
--- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java
+++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java
@@ -39,7 +39,7 @@ public class StreamsTestUtils {
final String valueSerdeClassName,
final Properties additional) {
- Properties streamsConfiguration = new Properties();
+ final Properties streamsConfiguration = new Properties();
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
streamsConfiguration.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
index e814fec..7730bed 100644
--- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
+++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
@@ -192,19 +192,22 @@ public class TopologyTestDriver implements Closeable {
* Initialized the internally mocked wall-clock time with {@link System#currentTimeMillis() current system time}.
*
* @param topology the topology to be tested
- * @param config the configuration for the topology
+ * @param config the configuration for the topology
*/
+ @SuppressWarnings("WeakerAccess")
public TopologyTestDriver(final Topology topology,
final Properties config) {
this(topology, config, System.currentTimeMillis());
}
+
/**
* Create a new test diver instance.
*
- * @param topology the topology to be tested
- * @param config the configuration for the topology
+ * @param topology the topology to be tested
+ * @param config the configuration for the topology
* @param initialWallClockTimeMs the initial value of internally mocked wall-clock time
*/
+ @SuppressWarnings("WeakerAccess")
public TopologyTestDriver(final Topology topology,
final Properties config,
final long initialWallClockTimeMs) {
@@ -215,7 +218,7 @@ public class TopologyTestDriver implements Closeable {
internalTopologyBuilder.setApplicationId(streamsConfig.getString(StreamsConfig.APPLICATION_ID_CONFIG));
processorTopology = internalTopologyBuilder.build(null);
- final ProcessorTopology globalTopology = internalTopologyBuilder.buildGlobalStateTopology();
+ final ProcessorTopology globalTopology = internalTopologyBuilder.buildGlobalStateTopology();
final Serializer<byte[]> bytesSerializer = new ByteArraySerializer();
producer = new MockProducer<byte[], byte[]>(true, bytesSerializer, bytesSerializer) {
@@ -237,13 +240,13 @@ public class TopologyTestDriver implements Closeable {
streamsMetrics);
final StateRestoreListener stateRestoreListener = new StateRestoreListener() {
@Override
- public void onRestoreStart(TopicPartition topicPartition, String storeName, long startingOffset, long endingOffset) {}
+ public void onRestoreStart(final TopicPartition topicPartition, final String storeName, final long startingOffset, final long endingOffset) {}
@Override
- public void onBatchRestored(TopicPartition topicPartition, String storeName, long batchEndOffset, long numRestored) {}
+ public void onBatchRestored(final TopicPartition topicPartition, final String storeName, final long batchEndOffset, final long numRestored) {}
@Override
- public void onRestoreEnd(TopicPartition topicPartition, String storeName, long totalRestored) {}
+ public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) {}
};
for (final InternalTopologyBuilder.TopicsInfo topicsInfo : internalTopologyBuilder.topicGroups().values()) {
@@ -321,6 +324,7 @@ public class TopologyTestDriver implements Closeable {
*
* @param consumerRecord the record to be processed
*/
+ @SuppressWarnings("WeakerAccess")
public void pipeInput(final ConsumerRecord<byte[], byte[]> consumerRecord) {
final String topicName = consumerRecord.topic();
@@ -400,11 +404,13 @@ public class TopologyTestDriver implements Closeable {
}
}
}
+
/**
* Send input messages to the topology and then commit each message individually.
*
* @param records a list of records to be processed
*/
+ @SuppressWarnings("WeakerAccess")
public void pipeInput(final List<ConsumerRecord<byte[], byte[]>> records) {
for (final ConsumerRecord<byte[], byte[]> record : records) {
pipeInput(record);
@@ -418,6 +424,7 @@ public class TopologyTestDriver implements Closeable {
*
* @param advanceMs the amount of time to advance wall-clock time in milliseconds
*/
+ @SuppressWarnings("WeakerAccess")
public void advanceWallClockTime(final long advanceMs) {
mockTime.sleep(advanceMs);
if (task != null) {
@@ -434,6 +441,7 @@ public class TopologyTestDriver implements Closeable {
* @param topic the name of the topic
* @return the next record on that topic, or {@code null} if there is no record available
*/
+ @SuppressWarnings("WeakerAccess")
public ProducerRecord<byte[], byte[]> readOutput(final String topic) {
final Queue<ProducerRecord<byte[], byte[]>> outputRecords = outputRecordsByTopic.get(topic);
if (outputRecords == null) {
@@ -446,11 +454,12 @@ public class TopologyTestDriver implements Closeable {
* Read the next record from the given topic.
* These records were output by the topology during the previous calls to {@link #pipeInput(ConsumerRecord)}.
*
- * @param topic the name of the topic
- * @param keyDeserializer the deserializer for the key type
+ * @param topic the name of the topic
+ * @param keyDeserializer the deserializer for the key type
* @param valueDeserializer the deserializer for the value type
* @return the next record on that topic, or {@code null} if there is no record available
*/
+ @SuppressWarnings("WeakerAccess")
public <K, V> ProducerRecord<K, V> readOutput(final String topic,
final Deserializer<K> keyDeserializer,
final Deserializer<V> valueDeserializer) {
@@ -476,6 +485,7 @@ public class TopologyTestDriver implements Closeable {
* @see #getWindowStore(String)
* @see #getSessionStore(String)
*/
+ @SuppressWarnings("WeakerAccess")
public Map<String, StateStore> getAllStateStores() {
final Map<String, StateStore> allStores = new HashMap<>();
for (final String storeName : internalTopologyBuilder.allStateStoreName()) {
@@ -521,7 +531,7 @@ public class TopologyTestDriver implements Closeable {
* @see #getWindowStore(String)
* @see #getSessionStore(String)
*/
- @SuppressWarnings("unchecked")
+ @SuppressWarnings({"unchecked", "WeakerAccess"})
public <K, V> KeyValueStore<K, V> getKeyValueStore(final String name) {
final StateStore store = getStateStore(name);
return store instanceof KeyValueStore ? (KeyValueStore<K, V>) store : null;
@@ -541,7 +551,7 @@ public class TopologyTestDriver implements Closeable {
* @see #getKeyValueStore(String)
* @see #getSessionStore(String) (String)
*/
- @SuppressWarnings("unchecked")
+ @SuppressWarnings({"unchecked", "WeakerAccess", "unused"})
public <K, V> WindowStore<K, V> getWindowStore(final String name) {
final StateStore store = getStateStore(name);
return store instanceof WindowStore ? (WindowStore<K, V>) store : null;
@@ -561,7 +571,7 @@ public class TopologyTestDriver implements Closeable {
* @see #getKeyValueStore(String)
* @see #getWindowStore(String)
*/
- @SuppressWarnings("unchecked")
+ @SuppressWarnings({"unchecked", "WeakerAccess", "unused"})
public <K, V> SessionStore<K, V> getSessionStore(final String name) {
final StateStore store = getStateStore(name);
return store instanceof SessionStore ? (SessionStore<K, V>) store : null;
@@ -634,7 +644,7 @@ public class TopologyTestDriver implements Closeable {
};
// for each store
- for (final Map.Entry<String, String> storeAndTopic: storeToChangelogTopic.entrySet()) {
+ for (final Map.Entry<String, String> storeAndTopic : storeToChangelogTopic.entrySet()) {
final String topicName = storeAndTopic.getValue();
// Set up the restore-state topic ...
// consumer.subscribe(new TopicPartition(topicName, 0));
diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java
index 03f871a..88801f7 100644
--- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java
+++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java
@@ -31,6 +31,7 @@ import org.apache.kafka.streams.processor.internals.StreamsMetricsImpl;
import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore;
import java.io.File;
+import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
@@ -45,7 +46,7 @@ import java.util.Properties;
* tests that serve as example usage.
* <p>
* Note that this class does not take any automated actions (such as firing scheduled punctuators).
- * It simply captures any data it witnessess.
+ * It simply captures any data it witnesses.
* If you require more automated tests, we recommend wrapping your {@link Processor} in a minimal source-processor-sink
* {@link Topology} and using the {@link TopologyTestDriver}.
*/
@@ -96,8 +97,9 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
return punctuator;
}
+ @SuppressWarnings("WeakerAccess")
public void cancel() {
- this.cancelled = true;
+ cancelled = true;
}
public boolean cancelled() {
@@ -122,7 +124,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
/**
* The child this data was forwarded to.
*
- * @return The child name, or {@code null} if it was broadcasted.
+ * @return The child name, or {@code null} if it was broadcast.
*/
public String childName() {
return childName;
@@ -147,7 +149,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
}
}
- // contructors ================================================
+ // constructors ================================================
/**
* Create a {@link MockProcessorContext} with dummy {@code config} and {@code taskId} and {@code null} {@code stateDir}.
@@ -192,7 +194,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
this.taskId = taskId;
this.config = streamsConfig;
this.stateDir = stateDir;
- this.metrics = new StreamsMetricsImpl(new Metrics(), "mock-processor-context", new HashMap<String, String>());
+ this.metrics = new StreamsMetricsImpl(new Metrics(), "mock-processor-context", Collections.<String, String>emptyMap());
}
@Override
@@ -379,15 +381,15 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
return capturedPunctuators;
}
+ @SuppressWarnings("unchecked")
@Override
public <K, V> void forward(final K key, final V value) {
- //noinspection unchecked
capturedForwards.add(new CapturedForward(To.all(), new KeyValue(key, value)));
}
+ @SuppressWarnings("unchecked")
@Override
public <K, V> void forward(final K key, final V value, final To to) {
- //noinspection unchecked
capturedForwards.add(new CapturedForward(to, new KeyValue(key, value)));
}
--
To stop receiving notification emails like this one, please contact
guozhang@apache.org.