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 <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This commit is contained in:
John Roesler 2018-04-17 15:13:15 -05:00 committed by Guozhang Wang
parent 83503404e4
commit ac9c3ed0b4
57 changed files with 2126 additions and 1800 deletions

View File

@ -51,6 +51,7 @@ import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
@ -628,6 +629,65 @@ public final class Utils {
return Arrays.asList(elems); return Arrays.asList(elems);
} }
/**
* 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) * Recursively delete the given file/directory and any subfiles (if any exist)
* *

View File

@ -32,7 +32,7 @@ public class KStreamAggregate<K, V, T> implements KStreamAggProcessorSupplier<K,
private boolean sendOldValues = false; 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.storeName = storeName;
this.initializer = initializer; this.initializer = initializer;
this.aggregator = aggregator; this.aggregator = aggregator;
@ -55,7 +55,7 @@ public class KStreamAggregate<K, V, T> implements KStreamAggProcessorSupplier<K,
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
store = (KeyValueStore<K, T>) context.getStateStore(storeName); store = (KeyValueStore<K, T>) context.getStateStore(storeName);
tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues); 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 @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 the key or value is null we don't need to proceed
if (key == null || value == null) { if (key == null || value == null) {
return; return;
@ -108,12 +108,12 @@ public class KStreamAggregate<K, V, T> implements KStreamAggProcessorSupplier<K,
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
store = (KeyValueStore<K, T>) context.getStateStore(storeName); store = (KeyValueStore<K, T>) context.getStateStore(storeName);
} }
@Override @Override
public T get(K key) { public T get(final K key) {
return store.get(key); return store.get(key);
} }
} }

View File

@ -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 ValueJoiner<? super V1, ? super V2, ? extends R> joiner;
private final boolean outer; 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.otherWindowName = otherWindowName;
this.joinBeforeMs = joinBeforeMs; this.joinBeforeMs = joinBeforeMs;
this.joinAfterMs = joinAfterMs; this.joinAfterMs = joinAfterMs;
@ -53,7 +53,7 @@ class KStreamKStreamJoin<K, R, V1, V2> implements ProcessorSupplier<K, V1> {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
otherWindow = (WindowStore<K, V2>) context.getStateStore(otherWindowName); 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 timeFrom = Math.max(0L, context().timestamp() - joinBeforeMs);
final long timeTo = Math.max(0L, context().timestamp() + joinAfterMs); 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()) { while (iter.hasNext()) {
needOuterJoin = false; needOuterJoin = false;
context().forward(key, joiner.apply(value, iter.next().value)); context().forward(key, joiner.apply(value, iter.next().value));

View File

@ -29,7 +29,7 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
private boolean sendOldValues = false; private boolean sendOldValues = false;
public KStreamReduce(String storeName, Reducer<V> reducer) { KStreamReduce(final String storeName, final Reducer<V> reducer) {
this.storeName = storeName; this.storeName = storeName;
this.reducer = reducer; this.reducer = reducer;
} }
@ -51,7 +51,7 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
store = (KeyValueStore<K, V>) context.getStateStore(storeName); store = (KeyValueStore<K, V>) context.getStateStore(storeName);
@ -60,13 +60,13 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
@Override @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 the key or value is null we don't need to proceed
if (key == null || value == null) { if (key == null || value == null) {
return; return;
} }
V oldAgg = store.get(key); final V oldAgg = store.get(key);
V newAgg = oldAgg; V newAgg = oldAgg;
// try to add the new value // try to add the new value
@ -75,7 +75,7 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
} else { } else {
newAgg = reducer.apply(newAgg, value); newAgg = reducer.apply(newAgg, value);
} }
// update the store with the new value // update the store with the new value
store.put(key, newAgg); store.put(key, newAgg);
tupleForwarder.maybeForward(key, newAgg, oldAgg); tupleForwarder.maybeForward(key, newAgg, oldAgg);
@ -104,12 +104,12 @@ public class KStreamReduce<K, V> implements KStreamAggProcessorSupplier<K, K, V,
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
store = (KeyValueStore<K, V>) context.getStateStore(storeName); store = (KeyValueStore<K, V>) context.getStateStore(storeName);
} }
@Override @Override
public V get(K key) { public V get(final K key) {
return store.get(key); return store.get(key);
} }
} }

View File

@ -71,9 +71,9 @@ class KStreamSessionWindowAggregate<K, V, T> implements KStreamAggProcessorSuppl
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(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); 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; SessionWindow mergedWindow = newSessionWindow;
T agg = initializer.apply(); T agg = initializer.apply();
try (final KeyValueIterator<Windowed<K>, T> iterator = store.findSessions(key, timestamp - windows.inactivityGap(), try (
timestamp + windows.inactivityGap())) { final KeyValueIterator<Windowed<K>, T> iterator = store.findSessions(
key,
timestamp - windows.inactivityGap(),
timestamp + windows.inactivityGap()
)
) {
while (iterator.hasNext()) { while (iterator.hasNext()) {
final KeyValue<Windowed<K>, T> next = iterator.next(); final KeyValue<Windowed<K>, T> next = iterator.next();
merged.add(next); merged.add(next);
@ -132,7 +137,7 @@ class KStreamSessionWindowAggregate<K, V, T> implements KStreamAggProcessorSuppl
@Override @Override
public String[] storeNames() { public String[] storeNames() {
return new String[] {storeName}; return new String[]{storeName};
} }
}; };
} }
@ -148,7 +153,7 @@ class KStreamSessionWindowAggregate<K, V, T> implements KStreamAggProcessorSuppl
@Override @Override
public T get(final Windowed<K> key) { 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()) { if (!iter.hasNext()) {
return null; return null;
} }

View File

@ -18,9 +18,9 @@ package org.apache.kafka.streams.kstream.internals;
import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Aggregator;
import org.apache.kafka.streams.kstream.Initializer; 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.Window;
import org.apache.kafka.streams.kstream.Windowed; 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.AbstractProcessor;
import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.ProcessorContext; 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) { public void process(final K key, final V value) {
// if the key is null, we do not need proceed aggregating the record // if the key is null, we do not need proceed aggregating the record
// the record with the table // the record with the table
if (key == null) if (key == null) {
return; return;
}
// first get the matching windows // first get the matching windows
final long timestamp = context().timestamp(); final long timestamp = context().timestamp();
@ -128,8 +129,8 @@ public class KStreamWindowAggregate<K, V, T, W extends Window> implements KStrea
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public T get(final Windowed<K> windowedKey) { public T get(final Windowed<K> windowedKey) {
K key = windowedKey.key(); final K key = windowedKey.key();
W window = (W) windowedKey.window(); final W window = (W) windowedKey.window();
return windowStore.fetch(key, window.start()); return windowStore.fetch(key, window.start());
} }

View File

@ -70,8 +70,9 @@ public class KStreamWindowReduce<K, V, W extends Window> implements KStreamAggPr
public void process(final K key, final V value) { public void process(final K key, final V value) {
// if the key is null, we do not need proceed aggregating // if the key is null, we do not need proceed aggregating
// the record with the table // the record with the table
if (key == null) if (key == null) {
return; return;
}
// first get the matching windows // first get the matching windows
final long timestamp = context().timestamp(); 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()) { for (final Map.Entry<Long, W> entry : matchedWindows.entrySet()) {
final V oldAgg = windowStore.fetch(key, entry.getKey()); final V oldAgg = windowStore.fetch(key, entry.getKey());
V newAgg; final V newAgg;
if (oldAgg == null) { if (oldAgg == null) {
newAgg = value; newAgg = value;
} else { } else {
@ -124,8 +125,8 @@ public class KStreamWindowReduce<K, V, W extends Window> implements KStreamAggPr
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public V get(final Windowed<K> windowedKey) { public V get(final Windowed<K> windowedKey) {
K key = windowedKey.key(); final K key = windowedKey.key();
W window = (W) windowedKey.window(); final W window = (W) windowedKey.window();
return windowStore.fetch(key, window.start()); return windowStore.fetch(key, window.start());
} }

View File

@ -69,15 +69,17 @@ public class KTableAggregate<K, V, T> implements KTableProcessorSupplier<K, V, T
* @throws StreamsException if key is null * @throws StreamsException if key is null
*/ */
@Override @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 // 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."); throw new StreamsException("Record key for KTable aggregate operator with state " + storeName + " should not be null.");
}
T oldAgg = store.get(key); T oldAgg = store.get(key);
if (oldAgg == null) if (oldAgg == null) {
oldAgg = initializer.apply(); oldAgg = initializer.apply();
}
T newAgg = oldAgg; T newAgg = oldAgg;

View File

@ -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); 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> { 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); super(valueGetterSupplier1, valueGetterSupplier2);
} }
@ -60,12 +63,12 @@ class KTableKTableInnerJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V2> valueGetter; private final KTableValueGetter<K, V2> valueGetter;
KTableKTableJoinProcessor(KTableValueGetter<K, V2> valueGetter) { KTableKTableJoinProcessor(final KTableValueGetter<K, V2> valueGetter) {
this.valueGetter = valueGetter; this.valueGetter = valueGetter;
} }
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
valueGetter.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); final V1 value1 = valueGetter1.get(key);
if (value1 != null) { if (value1 != null) {
V2 value2 = valueGetter2.get(keyValueMapper.apply(key, value1)); final V2 value2 = valueGetter2.get(keyValueMapper.apply(key, value1));
if (value2 != null) { if (value2 != null) {
return joiner.apply(value1, value2); return joiner.apply(value1, value2);

View File

@ -23,7 +23,9 @@ import org.apache.kafka.streams.processor.ProcessorContext;
class KTableKTableLeftJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R, V1, V2> { 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); 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> { 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); super(valueGetterSupplier1, valueGetterSupplier2);
} }
@ -53,12 +56,12 @@ class KTableKTableLeftJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V2> valueGetter; private final KTableValueGetter<K, V2> valueGetter;
KTableKTableLeftJoinProcessor(KTableValueGetter<K, V2> valueGetter) { KTableKTableLeftJoinProcessor(final KTableValueGetter<K, V2> valueGetter) {
this.valueGetter = valueGetter; this.valueGetter = valueGetter;
} }
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
valueGetter.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); newValue = joiner.apply(change.newValue, value2);
} }
if (sendOldValues && change.oldValue != null) if (sendOldValues && change.oldValue != null) {
oldValue = joiner.apply(change.oldValue, value2); oldValue = joiner.apply(change.oldValue, value2);
}
context().forward(key, new Change<>(newValue, oldValue)); 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, V1> valueGetter1;
private final KTableValueGetter<K, V2> valueGetter2; 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.valueGetter1 = valueGetter1;
this.valueGetter2 = valueGetter2; this.valueGetter2 = valueGetter2;
} }
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
valueGetter1.init(context); valueGetter1.init(context);
valueGetter2.init(context); valueGetter2.init(context);
} }
@Override @Override
public R get(K key) { public R get(final K key) {
V1 value1 = valueGetter1.get(key); final V1 value1 = valueGetter1.get(key);
if (value1 != null) { if (value1 != null) {
V2 value2 = valueGetter2.get(key); final V2 value2 = valueGetter2.get(key);
return joiner.apply(value1, value2); return joiner.apply(value1, value2);
} else { } else {
return null; return null;

View File

@ -23,7 +23,9 @@ import org.apache.kafka.streams.processor.ProcessorContext;
class KTableKTableOuterJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R, V1, V2> { 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); 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> { 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); super(valueGetterSupplier1, valueGetterSupplier2);
} }
@ -52,12 +55,12 @@ class KTableKTableOuterJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V2> valueGetter; private final KTableValueGetter<K, V2> valueGetter;
KTableKTableOuterJoinProcessor(KTableValueGetter<K, V2> valueGetter) { KTableKTableOuterJoinProcessor(final KTableValueGetter<K, V2> valueGetter) {
this.valueGetter = valueGetter; this.valueGetter = valueGetter;
} }
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
valueGetter.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, V1> valueGetter1;
private final KTableValueGetter<K, V2> valueGetter2; 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.valueGetter1 = valueGetter1;
this.valueGetter2 = valueGetter2; this.valueGetter2 = valueGetter2;
} }
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
valueGetter1.init(context); valueGetter1.init(context);
valueGetter2.init(context); valueGetter2.init(context);
} }
@Override @Override
public R get(K key) { public R get(final K key) {
R newValue = null; R newValue = null;
V1 value1 = valueGetter1.get(key); final V1 value1 = valueGetter1.get(key);
V2 value2 = valueGetter2.get(key); final V2 value2 = valueGetter2.get(key);
if (value1 != null || value2 != null) if (value1 != null || value2 != null) {
newValue = joiner.apply(value1, value2); newValue = joiner.apply(value1, value2);
}
return newValue; return newValue;
} }

View File

@ -24,7 +24,9 @@ import org.apache.kafka.streams.processor.ProcessorContext;
class KTableKTableRightJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R, V1, V2> { 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); 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> { 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); super(valueGetterSupplier1, valueGetterSupplier2);
} }
@ -53,12 +56,12 @@ class KTableKTableRightJoin<K, R, V1, V2> extends KTableKTableAbstractJoin<K, R,
private final KTableValueGetter<K, V2> valueGetter; private final KTableValueGetter<K, V2> valueGetter;
KTableKTableRightJoinProcessor(KTableValueGetter<K, V2> valueGetter) { KTableKTableRightJoinProcessor(final KTableValueGetter<K, V2> valueGetter) {
this.valueGetter = valueGetter; this.valueGetter = valueGetter;
} }
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
valueGetter.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, V1> valueGetter1;
private final KTableValueGetter<K, V2> valueGetter2; 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.valueGetter1 = valueGetter1;
this.valueGetter2 = valueGetter2; this.valueGetter2 = valueGetter2;
} }
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
valueGetter1.init(context); valueGetter1.init(context);
valueGetter2.init(context); valueGetter2.init(context);
} }
@Override @Override
public R get(K key) { public R get(final K key) {
V2 value2 = valueGetter2.get(key); final V2 value2 = valueGetter2.get(key);
if (value2 != null) { if (value2 != null) {
V1 value1 = valueGetter1.get(key); final V1 value1 = valueGetter1.get(key);
return joiner.apply(value1, value2); return joiner.apply(value1, value2);
} else { } else {
return null; return null;

View File

@ -57,19 +57,20 @@ public class KTableReduce<K, V> implements KTableProcessorSupplier<K, V, V> {
public void init(final ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
store = (KeyValueStore<K, V>) context.getStateStore(storeName); 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 * @throws StreamsException if key is null
*/ */
@Override @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 // 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."); 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; V newAgg = oldAgg;
// first try to add the new value // first try to add the new value
@ -94,6 +95,6 @@ public class KTableReduce<K, V> implements KTableProcessorSupplier<K, V, V> {
@Override @Override
public KTableValueGetterSupplier<K, V> view() { public KTableValueGetterSupplier<K, V> view() {
return new KTableMaterializedValueGetterSupplier<K, V>(storeName); return new KTableMaterializedValueGetterSupplier<>(storeName);
} }
} }

View File

@ -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. * 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). * 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>> { 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 * @throws StreamsException if key is null
*/ */
@Override @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 // 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."); 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 // 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); final 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> oldPair = change.oldValue == null ? null : mapper.apply(key, change.oldValue);
// if the selected repartition key or value is null, skip // if the selected repartition key or value is null, skip
// forward oldPair first, to be consistent with reduce and aggregate // 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) { if (newPair != null && newPair.key != null && newPair.value != null) {
context().forward(newPair.key, new Change<>(newPair.value, null)); context().forward(newPair.key, new Change<>(newPair.value, null));
} }
} }
} }

View File

@ -28,7 +28,7 @@ public class KTableSource<K, V> implements ProcessorSupplier<K, V> {
private boolean sendOldValues = false; private boolean sendOldValues = false;
public KTableSource(String storeName) { public KTableSource(final String storeName) {
this.storeName = storeName; this.storeName = storeName;
} }
@ -48,18 +48,19 @@ public class KTableSource<K, V> implements ProcessorSupplier<K, V> {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
store = (KeyValueStore<K, V>) context.getStateStore(storeName); store = (KeyValueStore<K, V>) context.getStateStore(storeName);
tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues); tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues);
} }
@Override @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 the key is null, then ignore the record
if (key == null) if (key == null) {
return; return;
V oldValue = store.get(key); }
final V oldValue = store.get(key);
store.put(key, value); store.put(key, value);
tupleForwarder.maybeForward(key, value, oldValue); tupleForwarder.maybeForward(key, value, oldValue);
} }

View File

@ -76,7 +76,7 @@ public abstract class AbstractTask implements Task {
this.eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG)); this.eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
this.stateDirectory = stateDirectory; 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.logContext = new LogContext(logPrefix);
this.log = logContext.logger(getClass()); 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. * Produces a string representation containing useful information about a StreamTask.
* This is useful in debugging scenarios. * This is useful in debugging scenarios.
*
* @return A string representation of the StreamTask instance. * @return A string representation of the StreamTask instance.
*/ */
@Override @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. * Produces a string representation containing useful information about a StreamTask starting with the given indent.
* This is useful in debugging scenarios. * This is useful in debugging scenarios.
*
* @return A string representation of the StreamTask instance. * @return A string representation of the StreamTask instance.
*/ */
public String toString(final String indent) { 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 * Package-private for testing only
*
* @throws StreamsException If the store's change log does not contain the partition
*/ */
void registerStateStores() { void registerStateStores() {
if (topology.stateStores().isEmpty()) { if (topology.stateStores().isEmpty()) {
@ -208,12 +210,12 @@ public abstract class AbstractTask implements Task {
try { try {
if (!stateDirectory.lock(id)) { if (!stateDirectory.lock(id)) {
throw new LockException(String.format("%sFailed to lock the state directory for task %s", throw new LockException(String.format("%sFailed to lock the state directory for task %s", logPrefix, id));
logPrefix, id));
} }
} catch (IOException e) { } catch (final IOException e) {
throw new StreamsException(String.format("%sFatal error while trying to lock the state directory for task %s", throw new StreamsException(
logPrefix, id)); String.format("%sFatal error while trying to lock the state directory for task %s",
logPrefix, id));
} }
log.trace("Initializing state stores"); 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 * @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 // visible for testing
void closeStateManager(final boolean writeCheckpoint) throws ProcessorStateException { void closeStateManager(final boolean writeCheckpoint) throws ProcessorStateException {
@ -246,7 +248,7 @@ public abstract class AbstractTask implements Task {
} finally { } finally {
try { try {
stateDirectory.unlock(id); stateDirectory.unlock(id);
} catch (IOException e) { } catch (final IOException e) {
if (exception == null) { if (exception == null) {
exception = new ProcessorStateException(String.format("%sFailed to release state dir lock", logPrefix), e); exception = new ProcessorStateException(String.format("%sFailed to release state dir lock", logPrefix), e);
} }

View File

@ -54,7 +54,7 @@ public class GlobalStateUpdateTask implements GlobalStateMaintainer {
/** /**
* @throws IllegalStateException If store gets registered after initialized is already finished * @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 @Override
public Map<TopicPartition, Long> initialize() { public Map<TopicPartition, Long> initialize() {
@ -100,7 +100,7 @@ public class GlobalStateUpdateTask implements GlobalStateMaintainer {
} }
private void initTopology() { private void initTopology() {
for (ProcessorNode node : this.topology.processors()) { for (final ProcessorNode node : this.topology.processors()) {
processorContext.setCurrentNode(node); processorContext.setCurrentNode(node);
try { try {
node.init(this.processorContext); node.init(this.processorContext);

View File

@ -86,11 +86,10 @@ public class GlobalStreamThread extends Thread {
* </pre> * </pre>
* *
* Note the following: * Note the following:
* - Any state can go to PENDING_SHUTDOWN. * <ul>
* That is because streams can be closed at any time. * <li>Any state can go to PENDING_SHUTDOWN. That is because streams can be closed at any time.</li>
* - State PENDING_SHUTDOWN may want to transit itself. * <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>
* In this case we will forbid the transition but will not treat as an error. * </ul>
*
*/ */
public enum State implements ThreadStateTransitionValidator { public enum State implements ThreadStateTransitionValidator {
CREATED(1, 2), RUNNING(2), PENDING_SHUTDOWN(3), DEAD; CREATED(1, 2), RUNNING(2), PENDING_SHUTDOWN(3), DEAD;
@ -107,7 +106,7 @@ public class GlobalStreamThread extends Thread {
@Override @Override
public boolean isValidTransition(final ThreadStateTransitionValidator newState) { public boolean isValidTransition(final ThreadStateTransitionValidator newState) {
State tmpState = (State) newState; final State tmpState = (State) newState;
return validTransitions.contains(tmpState.ordinal()); return validTransitions.contains(tmpState.ordinal());
} }
} }
@ -136,20 +135,21 @@ public class GlobalStreamThread extends Thread {
/** /**
* Sets the state * Sets the state
*
* @param newState New state * @param newState New state
*/ */
private boolean setState(final State newState) { private void setState(final State newState) {
final State oldState = state; final State oldState = state;
synchronized (stateLock) { synchronized (stateLock) {
if (state == State.PENDING_SHUTDOWN && newState == State.PENDING_SHUTDOWN) { if (state == State.PENDING_SHUTDOWN && newState == State.PENDING_SHUTDOWN) {
// when the state is already in PENDING_SHUTDOWN, its transition to itself // when the state is already in PENDING_SHUTDOWN, its transition to itself
// will be refused but we do not throw exception here // will be refused but we do not throw exception here
return false; return;
} else if (state == State.DEAD) { } else if (state == State.DEAD) {
// when the state is already in NOT_RUNNING, all its transitions // when the state is already in NOT_RUNNING, all its transitions
// will be refused but we do not throw exception here // will be refused but we do not throw exception here
return false; return;
} else if (!state.isValidTransition(newState)) { } else if (!state.isValidTransition(newState)) {
log.error("Unexpected state transition from {} to {}", oldState, newState); log.error("Unexpected state transition from {} to {}", oldState, newState);
throw new StreamsException(logPrefix + "Unexpected state transition from " + oldState + " to " + newState); throw new StreamsException(logPrefix + "Unexpected state transition from " + oldState + " to " + newState);
@ -163,8 +163,6 @@ public class GlobalStreamThread extends Thread {
if (stateListener != null) { if (stateListener != null) {
stateListener.onChange(this, state, oldState); stateListener.onChange(this, state, oldState);
} }
return true;
} }
public boolean stillRunning() { public boolean stillRunning() {
@ -194,7 +192,6 @@ public class GlobalStreamThread extends Thread {
this.log = logContext.logger(getClass()); this.log = logContext.logger(getClass());
this.cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics); this.cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics);
this.stateRestoreListener = stateRestoreListener; this.stateRestoreListener = stateRestoreListener;
} }
static class StateConsumer { static class StateConsumer {
@ -223,12 +220,12 @@ public class GlobalStreamThread extends Thread {
/** /**
* @throws IllegalStateException If store gets registered after initialized is already finished * @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() { void initialize() {
final Map<TopicPartition, Long> partitionOffsets = stateMaintainer.initialize(); final Map<TopicPartition, Long> partitionOffsets = stateMaintainer.initialize();
globalConsumer.assign(partitionOffsets.keySet()); 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()); globalConsumer.seek(entry.getKey(), entry.getValue());
} }
lastFlush = time.milliseconds(); lastFlush = time.milliseconds();
@ -237,7 +234,7 @@ public class GlobalStreamThread extends Thread {
void pollAndUpdate() { void pollAndUpdate() {
try { try {
final ConsumerRecords<byte[], byte[]> received = globalConsumer.poll(pollMs); final ConsumerRecords<byte[], byte[]> received = globalConsumer.poll(pollMs);
for (ConsumerRecord<byte[], byte[]> record : received) { for (final ConsumerRecord<byte[], byte[]> record : received) {
stateMaintainer.update(record); stateMaintainer.update(record);
} }
final long now = time.milliseconds(); final long now = time.milliseconds();
@ -309,12 +306,13 @@ public class GlobalStreamThread extends Thread {
private StateConsumer initialize() { private StateConsumer initialize() {
try { try {
final GlobalStateManager stateMgr = new GlobalStateManagerImpl(logContext, final GlobalStateManager stateMgr = new GlobalStateManagerImpl(
topology, logContext,
globalConsumer, topology,
stateDirectory, globalConsumer,
stateRestoreListener, stateDirectory,
config); stateRestoreListener,
config);
final GlobalProcessorContextImpl globalProcessorContext = new GlobalProcessorContextImpl( final GlobalProcessorContextImpl globalProcessorContext = new GlobalProcessorContextImpl(
config, config,
@ -323,17 +321,19 @@ public class GlobalStreamThread extends Thread {
cache); cache);
stateMgr.setGlobalProcessorContext(globalProcessorContext); stateMgr.setGlobalProcessorContext(globalProcessorContext);
final StateConsumer stateConsumer final StateConsumer stateConsumer = new StateConsumer(
= new StateConsumer(this.logContext, logContext,
globalConsumer, globalConsumer,
new GlobalStateUpdateTask(topology, new GlobalStateUpdateTask(
globalProcessorContext, topology,
stateMgr, globalProcessorContext,
config.defaultDeserializationExceptionHandler(), stateMgr,
logContext), config.defaultDeserializationExceptionHandler(),
time, logContext
config.getLong(StreamsConfig.POLL_MS_CONFIG), ),
config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)); time,
config.getLong(StreamsConfig.POLL_MS_CONFIG),
config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG));
stateConsumer.initialize(); stateConsumer.initialize();
return stateConsumer; return stateConsumer;

View File

@ -39,19 +39,19 @@ public class ProcessorNode<K, V> {
private final String name; private final String name;
private final Processor<K, V> processor; private final Processor<K, V> processor;
NodeMetrics nodeMetrics; private NodeMetrics nodeMetrics;
private Time time; private final Time time;
private K key; private K key;
private V value; private V value;
private Runnable processDelegate = new Runnable() { private final Runnable processDelegate = new Runnable() {
@Override @Override
public void run() { public void run() {
processor.process(key, value); processor.process(key, value);
} }
}; };
private ProcessorContext context; private ProcessorContext context;
private Runnable initDelegate = new Runnable() { private final Runnable initDelegate = new Runnable() {
@Override @Override
public void run() { public void run() {
if (processor != null) { if (processor != null) {
@ -59,7 +59,7 @@ public class ProcessorNode<K, V> {
} }
} }
}; };
private Runnable closeDelegate = new Runnable() { private final Runnable closeDelegate = new Runnable() {
@Override @Override
public void run() { public void run() {
if (processor != null) { if (processor != null) {
@ -70,12 +70,12 @@ public class ProcessorNode<K, V> {
public final Set<String> stateStores; public final Set<String> stateStores;
public ProcessorNode(String name) { public ProcessorNode(final String name) {
this(name, null, null); 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.name = name;
this.processor = processor; this.processor = processor;
this.children = new ArrayList<>(); this.children = new ArrayList<>();
@ -97,21 +97,21 @@ public class ProcessorNode<K, V> {
return children; return children;
} }
public final ProcessorNode getChild(final String childName) { final ProcessorNode getChild(final String childName) {
return childByName.get(childName); return childByName.get(childName);
} }
public void addChild(ProcessorNode<?, ?> child) { public void addChild(final ProcessorNode<?, ?> child) {
children.add(child); children.add(child);
childByName.put(child.name, child); childByName.put(child.name, child);
} }
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
this.context = context; this.context = context;
try { try {
nodeMetrics = new NodeMetrics(context.metrics(), name, context); nodeMetrics = new NodeMetrics(context.metrics(), name, context);
nodeMetrics.metrics.measureLatencyNs(time, initDelegate, nodeMetrics.nodeCreationSensor); 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); throw new StreamsException(String.format("failed to initialize processor %s", name), e);
} }
} }
@ -120,7 +120,7 @@ public class ProcessorNode<K, V> {
try { try {
nodeMetrics.metrics.measureLatencyNs(time, closeDelegate, nodeMetrics.nodeDestructionSensor); nodeMetrics.metrics.measureLatencyNs(time, closeDelegate, nodeMetrics.nodeDestructionSensor);
nodeMetrics.removeAllSensors(); nodeMetrics.removeAllSensors();
} catch (Exception e) { } catch (final Exception e) {
throw new StreamsException(String.format("failed to close processor %s", name), 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) { public void punctuate(final long timestamp, final Punctuator punctuator) {
Runnable punctuateDelegate = new Runnable() { final Runnable punctuateDelegate = new Runnable() {
@Override @Override
public void run() { public void run() {
punctuator.punctuate(timestamp); 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. * @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"); final StringBuilder sb = new StringBuilder(indent + name + ":\n");
if (stateStores != null && !stateStores.isEmpty()) { if (stateStores != null && !stateStores.isEmpty()) {
sb.append(indent).append("\tstates:\t\t["); sb.append(indent).append("\tstates:\t\t[");
for (String store : stateStores) { for (final String store : stateStores) {
sb.append(store); sb.append(store);
sb.append(", "); sb.append(", ");
} }
@ -168,39 +168,52 @@ public class ProcessorNode<K, V> {
return sb.toString(); return sb.toString();
} }
protected static final class NodeMetrics { Sensor sourceNodeForwardSensor() {
final StreamsMetricsImpl metrics; return nodeMetrics.sourceNodeForwardSensor;
}
final Sensor nodeProcessTimeSensor; Sensor sourceNodeSkippedDueToDeserializationErrorSensor() {
final Sensor nodePunctuateTimeSensor; return nodeMetrics.sourceNodeSkippedDueToDeserializationError;
final Sensor sourceNodeForwardSensor; }
final Sensor sourceNodeSkippedDueToDeserializationError;
final Sensor nodeCreationSensor;
final Sensor nodeDestructionSensor;
private static final class NodeMetrics {
private final StreamsMetricsImpl metrics;
public NodeMetrics(final StreamsMetrics metrics, final String name, final ProcessorContext context) { private final Sensor nodeProcessTimeSensor;
private final Sensor nodePunctuateTimeSensor;
private final Sensor sourceNodeForwardSensor;
private final Sensor sourceNodeSkippedDueToDeserializationError;
private final Sensor nodeCreationSensor;
private final Sensor nodeDestructionSensor;
private NodeMetrics(final StreamsMetrics metrics, final String name, final ProcessorContext context) {
final String scope = "processor-node"; final String scope = "processor-node";
final String tagKey = "task-id"; final String tagKey = "task-id";
final String tagValue = context.taskId().toString(); final String tagValue = context.taskId().toString();
this.metrics = (StreamsMetricsImpl) metrics; this.metrics = (StreamsMetricsImpl) metrics;
// these are all latency metrics // these are all latency metrics
this.nodeProcessTimeSensor = metrics.addLatencyAndThroughputSensor(scope, name, "process", this.nodeProcessTimeSensor = metrics.addLatencyAndThroughputSensor(
Sensor.RecordingLevel.DEBUG, tagKey, tagValue); scope, name, "process", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
this.nodePunctuateTimeSensor = metrics.addLatencyAndThroughputSensor(scope, name, "punctuate", );
Sensor.RecordingLevel.DEBUG, tagKey, tagValue); this.nodePunctuateTimeSensor = metrics.addLatencyAndThroughputSensor(
this.nodeCreationSensor = metrics.addLatencyAndThroughputSensor(scope, name, "create", scope, name, "punctuate", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
Sensor.RecordingLevel.DEBUG, tagKey, tagValue); );
this.nodeDestructionSensor = metrics.addLatencyAndThroughputSensor(scope, name, "destroy", this.nodeCreationSensor = metrics.addLatencyAndThroughputSensor(
Sensor.RecordingLevel.DEBUG, tagKey, tagValue); scope, name, "create", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
this.sourceNodeForwardSensor = metrics.addThroughputSensor(scope, name, "forward", );
Sensor.RecordingLevel.DEBUG, tagKey, tagValue); this.nodeDestructionSensor = metrics.addLatencyAndThroughputSensor(
this.sourceNodeSkippedDueToDeserializationError = metrics.addThroughputSensor(scope, name, "skippedDueToDeserializationError", scope, name, "destroy", Sensor.RecordingLevel.DEBUG, tagKey, tagValue
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(nodeProcessTimeSensor);
metrics.removeSensor(nodePunctuateTimeSensor); metrics.removeSensor(nodePunctuateTimeSensor);
metrics.removeSensor(sourceNodeForwardSensor); metrics.removeSensor(sourceNodeForwardSensor);

View File

@ -23,14 +23,14 @@ import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition; 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.AuthenticationException;
import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.OffsetMetadataTooLarge; import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.RetriableException; 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.SecurityDisabledException;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.Serializer;
@ -95,11 +95,11 @@ public class RecordCollectorImpl implements RecordCollector {
} }
private boolean productionExceptionIsFatal(final Exception exception) { private boolean productionExceptionIsFatal(final Exception exception) {
boolean securityException = exception instanceof AuthenticationException || final boolean securityException = exception instanceof AuthenticationException ||
exception instanceof AuthorizationException || exception instanceof AuthorizationException ||
exception instanceof SecurityDisabledException; exception instanceof SecurityDisabledException;
boolean communicationException = exception instanceof InvalidTopicException || final boolean communicationException = exception instanceof InvalidTopicException ||
exception instanceof UnknownServerException || exception instanceof UnknownServerException ||
exception instanceof SerializationException || exception instanceof SerializationException ||
exception instanceof OffsetMetadataTooLarge || exception instanceof OffsetMetadataTooLarge ||
@ -123,31 +123,32 @@ public class RecordCollectorImpl implements RecordCollector {
} }
log.error(errorLogMessage, key, value, timestamp, topic, exception.toString()); log.error(errorLogMessage, key, value, timestamp, topic, exception.toString());
sendException = new StreamsException( sendException = new StreamsException(
String.format(errorMessage, String.format(
logPrefix, errorMessage,
"an error caught", logPrefix,
key, "an error caught",
value, key,
timestamp, value,
topic, timestamp,
exception.toString()), topic,
exception.toString()
),
exception); exception);
} }
@Override @Override
public <K, V> void send(final String topic, public <K, V> void send(final String topic,
final K key, final K key,
final V value, final V value,
final Integer partition, final Integer partition,
final Long timestamp, final Long timestamp,
final Serializer<K> keySerializer, final Serializer<K> keySerializer,
final Serializer<V> valueSerializer) { final Serializer<V> valueSerializer) {
checkForException(); checkForException();
final byte[] keyBytes = keySerializer.serialize(topic, key); final byte[] keyBytes = keySerializer.serialize(topic, key);
final byte[] valBytes = valueSerializer.serialize(topic, value); final byte[] valBytes = valueSerializer.serialize(topic, value);
final ProducerRecord<byte[], byte[]> serializedRecord = final ProducerRecord<byte[], byte[]> serializedRecord = new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes);
new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes);
try { try {
producer.send(serializedRecord, new Callback() { producer.send(serializedRecord, new Callback() {
@ -165,14 +166,17 @@ public class RecordCollectorImpl implements RecordCollector {
if (exception instanceof ProducerFencedException) { if (exception instanceof ProducerFencedException) {
log.warn(LOG_MESSAGE, key, value, timestamp, topic, exception.toString()); log.warn(LOG_MESSAGE, key, value, timestamp, topic, exception.toString());
sendException = new ProducerFencedException( sendException = new ProducerFencedException(
String.format(EXCEPTION_MESSAGE, String.format(
logPrefix, EXCEPTION_MESSAGE,
"producer got fenced", logPrefix,
key, "producer got fenced",
value, key,
timestamp, value,
topic, timestamp,
exception.toString())); topic,
exception.toString()
)
);
} else { } else {
if (productionExceptionIsFatal(exception)) { if (productionExceptionIsFatal(exception)) {
recordSendError(key, value, timestamp, topic, 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)); throw new StreamsException(String.format("%sFailed to send record to topic %s due to timeout.", logPrefix, topic));
} catch (final Exception uncaughtException) { } catch (final Exception uncaughtException) {
throw new StreamsException( throw new StreamsException(
String.format(EXCEPTION_MESSAGE, String.format(
logPrefix, EXCEPTION_MESSAGE,
"an error caught", logPrefix,
key, "an error caught",
value, key,
timestamp, value,
topic, timestamp,
uncaughtException.toString()), topic,
uncaughtException.toString()
),
uncaughtException); uncaughtException);
} }
} }
private void checkForException() { private void checkForException() {
if (sendException != null) { if (sendException != null) {
throw sendException; throw sendException;
} }

View File

@ -41,7 +41,8 @@ class RecordDeserializer {
/** /**
* @throws StreamsException if a deserialization error occurs and the deserialization callback returns * @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") @SuppressWarnings("deprecation")
ConsumerRecord<Object, Object> deserialize(final ProcessorContext processorContext, ConsumerRecord<Object, Object> deserialize(final ProcessorContext processorContext,
@ -64,9 +65,10 @@ class RecordDeserializer {
try { try {
response = deserializationExceptionHandler.handle(processorContext, rawRecord, deserializationException); response = deserializationExceptionHandler.handle(processorContext, rawRecord, deserializationException);
} catch (final Exception fatalUserException) { } catch (final Exception fatalUserException) {
log.error("Deserialization error callback failed after deserialization error for record {}", log.error(
rawRecord, "Deserialization error callback failed after deserialization error for record {}",
deserializationException); rawRecord,
deserializationException);
throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException); 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.", DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.",
deserializationException); deserializationException);
} else { } else {
sourceNode.nodeMetrics.sourceNodeSkippedDueToDeserializationError.record(); sourceNode.sourceNodeSkippedDueToDeserializationErrorSensor().record();
} }
} }
return null; return null;

View File

@ -36,7 +36,11 @@ public class SourceNode<K, V> extends ProcessorNode<K, V> {
private ExtendedDeserializer<V> valDeserializer; private ExtendedDeserializer<V> valDeserializer;
private final TimestampExtractor timestampExtractor; 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); super(name);
this.topics = topics; this.topics = topics;
this.timestampExtractor = timestampExtractor; this.timestampExtractor = timestampExtractor;
@ -44,21 +48,24 @@ public class SourceNode<K, V> extends ProcessorNode<K, V> {
this.valDeserializer = ensureExtended(valDeserializer); 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); 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); 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); return valDeserializer.deserialize(topic, headers, data);
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
this.context = context; this.context = context;
@ -78,7 +85,7 @@ public class SourceNode<K, V> extends ProcessorNode<K, V> {
@Override @Override
public void process(final K key, final V value) { public void process(final K key, final V value) {
context.forward(key, 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. * @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)); final StringBuilder sb = new StringBuilder(super.toString(indent));
sb.append(indent).append("\ttopics:\t\t["); sb.append(indent).append("\ttopics:\t\t[");
for (String topic : topics) { for (final String topic : topics) {
sb.append(topic); sb.append(topic);
sb.append(", "); sb.append(", ");
} }

View File

@ -166,7 +166,7 @@ public class StandbyTask extends AbstractTask {
return stateMgr.updateStandbyStates(partition, records); return stateMgr.updateStandbyStates(partition, records);
} }
public Map<TopicPartition, Long> checkpointedOffsets() { Map<TopicPartition, Long> checkpointedOffsets() {
return checkpointedOffsets; return checkpointedOffsets;
} }

View File

@ -70,16 +70,15 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
private final Time time; private final Time time;
private final TaskMetrics metrics; private final TaskMetrics metrics;
protected class TaskMetrics { protected static class TaskMetrics {
final StreamsMetricsImpl metrics; final StreamsMetricsImpl metrics;
final Sensor taskCommitTimeSensor; final Sensor taskCommitTimeSensor;
TaskMetrics(final StreamsMetrics metrics) { TaskMetrics(final TaskId id, final StreamsMetrics metrics) {
final String name = id().toString(); final String name = id.toString();
this.metrics = (StreamsMetricsImpl) metrics; this.metrics = (StreamsMetricsImpl) metrics;
taskCommitTimeSensor = metrics.addLatencyAndThroughputSensor("task", name, "commit", taskCommitTimeSensor = metrics.addLatencyAndThroughputSensor("task", name, "commit", Sensor.RecordingLevel.DEBUG);
Sensor.RecordingLevel.DEBUG);
} }
void removeAllSensors() { void removeAllSensors() {
@ -89,17 +88,18 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
/** /**
* Create {@link StreamTask} with its assigned partitions * Create {@link StreamTask} with its assigned partitions
* @param id the ID of this task *
* @param partitions the collection of assigned {@link TopicPartition} * @param id the ID of this task
* @param topology the instance of {@link ProcessorTopology} * @param partitions the collection of assigned {@link TopicPartition}
* @param consumer the instance of {@link Consumer} * @param topology the instance of {@link ProcessorTopology}
* @param changelogReader the instance of {@link ChangelogReader} used for restoring state * @param consumer the instance of {@link Consumer}
* @param config the {@link StreamsConfig} specified by the user * @param changelogReader the instance of {@link ChangelogReader} used for restoring state
* @param metrics the {@link StreamsMetrics} created by the thread * @param config the {@link StreamsConfig} specified by the user
* @param stateDirectory the {@link StateDirectory} created by the thread * @param metrics the {@link StreamsMetrics} created by the thread
* @param cache the {@link ThreadCache} created by the thread * @param stateDirectory the {@link StateDirectory} created by the thread
* @param time the system {@link Time} of the thread * @param cache the {@link ThreadCache} created by the thread
* @param producer the instance of {@link Producer} used to produce records * @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, public StreamTask(final TaskId id,
final Collection<TopicPartition> partitions, final Collection<TopicPartition> partitions,
@ -116,7 +116,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
this.time = time; this.time = time;
this.producer = producer; this.producer = producer;
this.metrics = new TaskMetrics(metrics); this.metrics = new TaskMetrics(id, metrics);
final ProductionExceptionHandler productionExceptionHandler = config.defaultProductionExceptionHandler(); final ProductionExceptionHandler productionExceptionHandler = config.defaultProductionExceptionHandler();
@ -140,7 +140,14 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
for (final TopicPartition partition : partitions) { for (final TopicPartition partition : partitions) {
final SourceNode source = topology.source(partition.topic()); final SourceNode source = topology.source(partition.topic());
final TimestampExtractor sourceTimestampExtractor = source.getTimestampExtractor() != null ? source.getTimestampExtractor() : defaultTimestampExtractor; 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); partitionQueues.put(partition, queue);
} }
@ -167,6 +174,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* <pre> * <pre>
* - (re-)initialize the topology of the task * - (re-)initialize the topology of the task
* </pre> * </pre>
*
* @throws TaskMigratedException if the task producer got fenced (EOS only) * @throws TaskMigratedException if the task producer got fenced (EOS only)
*/ */
@Override @Override
@ -272,7 +280,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
} catch (final ProducerFencedException fatal) { } catch (final ProducerFencedException fatal) {
throw new TaskMigratedException(this, fatal); throw new TaskMigratedException(this, fatal);
} catch (final KafkaException e) { } 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 { } finally {
processorContext.setCurrentNode(null); processorContext.setCurrentNode(null);
} }
@ -289,6 +297,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* - if(!eos) write checkpoint * - if(!eos) write checkpoint
* - commit offsets and start new transaction * - commit offsets and start new transaction
* </pre> * </pre>
*
* @throws TaskMigratedException if committing offsets failed (non-EOS) * @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS) * or if the task producer got fenced (EOS)
*/ */
@ -377,9 +386,10 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
Map<TopicPartition, Long> purgableOffsets() { Map<TopicPartition, Long> purgableOffsets() {
final Map<TopicPartition, Long> purgableConsumedOffsets = new HashMap<>(); final Map<TopicPartition, Long> purgableConsumedOffsets = new HashMap<>();
for (final Map.Entry<TopicPartition, Long> entry : consumedOffsets.entrySet()) { for (final Map.Entry<TopicPartition, Long> entry : consumedOffsets.entrySet()) {
TopicPartition tp = entry.getKey(); final TopicPartition tp = entry.getKey();
if (topology.isRepartitionTopic(tp.topic())) if (topology.isRepartitionTopic(tp.topic())) {
purgableConsumedOffsets.put(tp, entry.getValue() + 1); purgableConsumedOffsets.put(tp, entry.getValue() + 1);
}
} }
return purgableConsumedOffsets; return purgableConsumedOffsets;
@ -406,6 +416,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* - if (!eos) write checkpoint * - if (!eos) write checkpoint
* - commit offsets * - commit offsets
* </pre> * </pre>
*
* @throws TaskMigratedException if committing offsets failed (non-EOS) * @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS) * or if the task producer got fenced (EOS)
*/ */
@ -423,6 +434,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* - if (!eos) write checkpoint * - if (!eos) write checkpoint
* - commit offsets * - commit offsets
* </pre> * </pre>
*
* @throws TaskMigratedException if committing offsets failed (non-EOS) * @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (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 (clean) write checkpoint
* - if (eos) close producer * - if (eos) close producer
* </pre> * </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} * @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) * @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (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 * and not added to the queue for processing
* *
* @param partition the partition * @param partition the partition
* @param records the records * @param records the records
* @return the number of added records * @return the number of added records
*/ */
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@ -578,7 +591,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* Schedules a punctuation for the processor * Schedules a punctuation for the processor
* *
* @param interval the interval in milliseconds * @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 * @throws IllegalStateException if the current node is not null
*/ */
public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator punctuator) { 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 * Schedules a punctuation for the processor
* *
* @param startTime time of the first punctuation * @param startTime time of the first punctuation
* @param interval the interval in milliseconds * @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 * @throws IllegalStateException if the current node is not null
*/ */
Cancellable schedule(final long startTime, final long interval, final PunctuationType type, final Punctuator punctuator) { 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 * Possibly trigger registered stream-time punctuation functions if
* current partition group timestamp has reached the defined stamp * current partition group timestamp has reached the defined stamp
* Note, this is only called in the presence of new records * Note, this is only called in the presence of new records
*
* @throws TaskMigratedException if the task producer got fenced (EOS only) * @throws TaskMigratedException if the task producer got fenced (EOS only)
*/ */
public boolean maybePunctuateStreamTime() { public boolean maybePunctuateStreamTime() {
@ -651,6 +665,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator
* Possibly trigger registered system-time punctuation functions if * Possibly trigger registered system-time punctuation functions if
* current system timestamp has reached the defined stamp * current system timestamp has reached the defined stamp
* Note, this is called irrespective of the presence of new records * Note, this is called irrespective of the presence of new records
*
* @throws TaskMigratedException if the task producer got fenced (EOS only) * @throws TaskMigratedException if the task producer got fenced (EOS only)
*/ */
public boolean maybePunctuateSystemTime() { public boolean maybePunctuateSystemTime() {

View File

@ -73,7 +73,7 @@ public class StreamThread extends Thread {
* Stream thread states are the possible states that a stream thread can be in. * 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 * A thread must only be in one state at a time
* The expected state transitions with the following defined states is: * The expected state transitions with the following defined states is:
* <p> *
* <pre> * <pre>
* +-------------+ * +-------------+
* +<--- | Created (0) | * +<--- | Created (0) |
@ -108,17 +108,20 @@ public class StreamThread extends Thread {
* +-------------+ * +-------------+
* </pre> * </pre>
* *
* <p>
* Note the following: * Note the following:
* - Any state can go to PENDING_SHUTDOWN. * <ul>
* That is because streams can be closed at any time. * <li>Any state can go to PENDING_SHUTDOWN. That is because streams can be closed at any time.</li>
* - State PENDING_SHUTDOWN may want to transit to some other states other than DEAD, in the corner case when * <li>
* the shutdown is triggered while the thread is still in the rebalance loop. * State PENDING_SHUTDOWN may want to transit to some other states other than DEAD,
* In this case we will forbid the transition but will not treat as an error. * in the corner case when the shutdown is triggered while the thread is still in the rebalance loop.
* - State PARTITIONS_REVOKED may want transit to itself indefinitely, in the corner case when * In this case we will forbid the transition but will not treat as an error.
* the coordinator repeatedly fails in-between revoking partitions and assigning new partitions. * </li>
* In this case we will forbid the transition but will not treat as an error. * <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 { public enum State implements ThreadStateTransitionValidator {
CREATED(1, 4), RUNNING(2, 4), PARTITIONS_REVOKED(3, 4), PARTITIONS_ASSIGNED(1, 2, 4), PENDING_SHUTDOWN(5), DEAD; 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 @Override
public boolean isValidTransition(final ThreadStateTransitionValidator newState) { public boolean isValidTransition(final ThreadStateTransitionValidator newState) {
State tmpState = (State) newState; final State tmpState = (State) newState;
return validTransitions.contains(tmpState.ordinal()); return validTransitions.contains(tmpState.ordinal());
} }
} }
@ -147,9 +150,10 @@ public class StreamThread extends Thread {
/** /**
* Called when state changes * Called when state changes
* @param thread thread changing state *
* @param newState current state * @param thread thread changing state
* @param oldState previous state * @param newState current state
* @param oldState previous state
*/ */
void onChange(final Thread thread, final ThreadStateTransitionValidator newState, final ThreadStateTransitionValidator oldState); void onChange(final Thread thread, final ThreadStateTransitionValidator newState, final ThreadStateTransitionValidator oldState);
} }
@ -172,6 +176,7 @@ public class StreamThread extends Thread {
/** /**
* Sets the state * Sets the state
*
* @param newState New state * @param newState New state
* @return The state prior to the call to setState, or null if the transition is invalid * @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 @Override
public void onPartitionsAssigned(final Collection<TopicPartition> assignment) { public void onPartitionsAssigned(final Collection<TopicPartition> assignment) {
log.debug("at state {}: partitions {} assigned at the end of consumer rebalance.\n" + log.debug("at state {}: partitions {} assigned at the end of consumer rebalance.\n" +
"\tcurrent suspended active tasks: {}\n" + "\tcurrent suspended active tasks: {}\n" +
"\tcurrent suspended standby tasks: {}\n", "\tcurrent suspended standby tasks: {}\n",
streamThread.state, streamThread.state,
assignment, assignment,
taskManager.suspendedActiveTaskIds(), taskManager.suspendedActiveTaskIds(),
taskManager.suspendedStandbyTaskIds()); taskManager.suspendedStandbyTaskIds());
final long start = time.milliseconds(); final long start = time.milliseconds();
try { try {
@ -259,18 +264,21 @@ public class StreamThread extends Thread {
} }
taskManager.createTasks(assignment); taskManager.createTasks(assignment);
} catch (final Throwable t) { } catch (final Throwable t) {
log.error("Error caught during partition assignment, " + log.error(
"will abort the current process and re-throw at the end of rebalance: {}", t); "Error caught during partition assignment, " +
"will abort the current process and re-throw at the end of rebalance: {}",
t
);
streamThread.setRebalanceException(t); streamThread.setRebalanceException(t);
} finally { } finally {
log.info("partition assignment took {} ms.\n" + log.info("partition assignment took {} ms.\n" +
"\tcurrent active tasks: {}\n" + "\tcurrent active tasks: {}\n" +
"\tcurrent standby tasks: {}\n" + "\tcurrent standby tasks: {}\n" +
"\tprevious active tasks: {}\n", "\tprevious active tasks: {}\n",
time.milliseconds() - start, time.milliseconds() - start,
taskManager.activeTaskIds(), taskManager.activeTaskIds(),
taskManager.standbyTaskIds(), taskManager.standbyTaskIds(),
taskManager.prevActiveTaskIds()); taskManager.prevActiveTaskIds());
} }
} }
@ -290,18 +298,21 @@ public class StreamThread extends Thread {
// suspend active tasks // suspend active tasks
taskManager.suspendTasksAndState(); taskManager.suspendTasksAndState();
} catch (final Throwable t) { } catch (final Throwable t) {
log.error("Error caught during partition revocation, " + log.error(
"will abort the current process and re-throw at the end of rebalance: {}", t); "Error caught during partition revocation, " +
"will abort the current process and re-throw at the end of rebalance: {}",
t
);
streamThread.setRebalanceException(t); streamThread.setRebalanceException(t);
} finally { } finally {
streamThread.clearStandbyRecords(); streamThread.clearStandbyRecords();
log.info("partition revocation took {} ms.\n" + log.info("partition revocation took {} ms.\n" +
"\tsuspended active tasks: {}\n" + "\tsuspended active tasks: {}\n" +
"\tsuspended standby tasks: {}", "\tsuspended standby tasks: {}",
time.milliseconds() - start, time.milliseconds() - start,
taskManager.suspendedActiveTaskIds(), taskManager.suspendedActiveTaskIds(),
taskManager.suspendedStandbyTaskIds()); taskManager.suspendedStandbyTaskIds());
} }
} }
} }
@ -346,12 +357,13 @@ public class StreamThread extends Thread {
return stateDirectory; 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<>(); final List<T> createdTasks = new ArrayList<>();
for (final Map.Entry<TaskId, Set<TopicPartition>> newTaskAndPartitions : tasksToBeCreated.entrySet()) { for (final Map.Entry<TaskId, Set<TopicPartition>> newTaskAndPartitions : tasksToBeCreated.entrySet()) {
final TaskId taskId = newTaskAndPartitions.getKey(); final TaskId taskId = newTaskAndPartitions.getKey();
final Set<TopicPartition> partitions = newTaskAndPartitions.getValue(); final Set<TopicPartition> partitions = newTaskAndPartitions.getValue();
T task = createTask(consumer, taskId, partitions); final T task = createTask(consumer, taskId, partitions);
if (task != null) { if (task != null) {
log.trace("Created task {} with assigned partitions {}", taskId, partitions); log.trace("Created task {} with assigned partitions {}", taskId, partitions);
createdTasks.add(task); createdTasks.add(task);
@ -384,14 +396,15 @@ public class StreamThread extends Thread {
final Producer<byte[], byte[]> threadProducer, final Producer<byte[], byte[]> threadProducer,
final String threadClientId, final String threadClientId,
final Logger log) { final Logger log) {
super(builder, super(
config, builder,
streamsMetrics, config,
stateDirectory, streamsMetrics,
taskCreatedSensor, stateDirectory,
storeChangelogReader, taskCreatedSensor,
time, storeChangelogReader,
log); time,
log);
this.cache = cache; this.cache = cache;
this.clientSupplier = clientSupplier; this.clientSupplier = clientSupplier;
this.threadProducer = threadProducer; this.threadProducer = threadProducer;
@ -399,21 +412,23 @@ public class StreamThread extends Thread {
} }
@Override @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(); taskCreatedSensor.record();
return new StreamTask( return new StreamTask(
taskId, taskId,
partitions, partitions,
builder.build(taskId.topicGroupId), builder.build(taskId.topicGroupId),
consumer, consumer,
storeChangelogReader, storeChangelogReader,
config, config,
streamsMetrics, streamsMetrics,
stateDirectory, stateDirectory,
cache, cache,
time, time,
createProducer(taskId)); createProducer(taskId));
} }
@ -450,14 +465,15 @@ public class StreamThread extends Thread {
final ChangelogReader storeChangelogReader, final ChangelogReader storeChangelogReader,
final Time time, final Time time,
final Logger log) { final Logger log) {
super(builder, super(
config, builder,
streamsMetrics, config,
stateDirectory, streamsMetrics,
taskCreatedSensor, stateDirectory,
storeChangelogReader, taskCreatedSensor,
time, storeChangelogReader,
log); time,
log);
} }
@Override @Override
@ -469,17 +485,21 @@ public class StreamThread extends Thread {
final ProcessorTopology topology = builder.build(taskId.topicGroupId); final ProcessorTopology topology = builder.build(taskId.topicGroupId);
if (!topology.stateStores().isEmpty()) { if (!topology.stateStores().isEmpty()) {
return new StandbyTask(taskId, return new StandbyTask(
partitions, taskId,
topology, partitions,
consumer, topology,
storeChangelogReader, consumer,
config, storeChangelogReader,
streamsMetrics, config,
stateDirectory); streamsMetrics,
stateDirectory);
} else { } else {
log.trace("Skipped standby task {} with assigned partitions {} " + log.trace(
"since it does not have any state stores to materialize", taskId, partitions); "Skipped standby task {} with assigned partitions {} " +
"since it does not have any state stores to materialize",
taskId, partitions
);
return null; return null;
} }
} }
@ -531,11 +551,22 @@ public class StreamThread extends Thread {
} }
private Meter createMeter(Metrics metrics, SampledStat stat, String baseName, String descriptiveName) { private Meter createMeter(final Metrics metrics,
MetricName rateMetricName = metrics.metricName(baseName + "-rate", groupName, final SampledStat stat,
String.format("The average per-second number of %s", descriptiveName), tags); final String baseName,
MetricName totalMetricName = metrics.metricName(baseName + "-total", groupName, final String descriptiveName) {
String.format("The total number of %s", descriptiveName), tags); 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); return new Meter(stat, rateMetricName, totalMetricName);
} }
@ -562,7 +593,7 @@ public class StreamThread extends Thread {
private long lastCommitMs; private long lastCommitMs;
private long timerStartedMs; private long timerStartedMs;
private String originalReset; private final String originalReset;
private Throwable rebalanceException = null; private Throwable rebalanceException = null;
private boolean processStandbyRecords = false; private boolean processStandbyRecords = false;
private volatile State state = State.CREATED; private volatile State state = State.CREATED;
@ -607,44 +638,47 @@ public class StreamThread extends Thread {
threadProducer = clientSupplier.getProducer(producerConfigs); threadProducer = clientSupplier.getProducer(producerConfigs);
} }
StreamsMetricsThreadImpl streamsMetrics = new StreamsMetricsThreadImpl( final StreamsMetricsThreadImpl streamsMetrics = new StreamsMetricsThreadImpl(
metrics, metrics,
"stream-metrics", "stream-metrics",
"thread." + threadClientId, "thread." + threadClientId,
Collections.singletonMap("client-id", threadClientId)); Collections.singletonMap("client-id", threadClientId));
final ThreadCache cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics); final ThreadCache cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics);
final AbstractTaskCreator<StreamTask> activeTaskCreator = new TaskCreator(builder, final AbstractTaskCreator<StreamTask> activeTaskCreator = new TaskCreator(
config, builder,
streamsMetrics, config,
stateDirectory, streamsMetrics,
streamsMetrics.taskCreatedSensor, stateDirectory,
changelogReader, streamsMetrics.taskCreatedSensor,
cache, changelogReader,
time, cache,
clientSupplier, time,
threadProducer, clientSupplier,
threadClientId, threadProducer,
log); threadClientId,
final AbstractTaskCreator<StandbyTask> standbyTaskCreator = new StandbyTaskCreator(builder, log);
config, final AbstractTaskCreator<StandbyTask> standbyTaskCreator = new StandbyTaskCreator(
streamsMetrics, builder,
stateDirectory, config,
streamsMetrics.taskCreatedSensor, streamsMetrics,
changelogReader, stateDirectory,
time, streamsMetrics.taskCreatedSensor,
log); changelogReader,
TaskManager taskManager = new TaskManager(changelogReader, time,
processId, log);
logPrefix, final TaskManager taskManager = new TaskManager(
restoreConsumer, changelogReader,
streamsMetadataState, processId,
activeTaskCreator, logPrefix,
standbyTaskCreator, restoreConsumer,
adminClient, streamsMetadataState,
new AssignedStreamsTasks(logContext), activeTaskCreator,
new AssignedStandbyTasks(logContext)); standbyTaskCreator,
adminClient,
new AssignedStreamsTasks(logContext),
new AssignedStandbyTasks(logContext));
log.info("Creating consumer client"); log.info("Creating consumer client");
final String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG); 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); final Consumer<byte[], byte[]> consumer = clientSupplier.getConsumer(consumerConfigs);
taskManager.setConsumer(consumer); taskManager.setConsumer(consumer);
return new StreamThread(time, return new StreamThread(
config, time,
restoreConsumer, config,
consumer, restoreConsumer,
originalReset, consumer,
taskManager, originalReset,
streamsMetrics, taskManager,
builder, streamsMetrics,
threadClientId, builder,
logContext); threadClientId,
logContext);
} }
public StreamThread(final Time time, public StreamThread(final Time time,
@ -705,8 +740,8 @@ public class StreamThread extends Thread {
/** /**
* Execute the stream processors * Execute the stream processors
* *
* @throws KafkaException for any Kafka-related exceptions * @throws KafkaException for any Kafka-related exceptions
* @throws RuntimeException for any other non-Kafka exceptions * @throws RuntimeException for any other non-Kafka exceptions
*/ */
@Override @Override
public void run() { 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; this.rebalanceException = rebalanceException;
} }
/** /**
* Main event loop for polling, and processing records through topologies. * Main event loop for polling, and processing records through topologies.
*
* @throws IllegalStateException If store gets registered after initialized is already finished * @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() { private void runLoop() {
long recordsProcessedBeforeCommit = UNLIMITED_RECORDS; long recordsProcessedBeforeCommit = UNLIMITED_RECORDS;
@ -750,9 +786,9 @@ public class StreamThread extends Thread {
recordsProcessedBeforeCommit = runOnce(recordsProcessedBeforeCommit); recordsProcessedBeforeCommit = runOnce(recordsProcessedBeforeCommit);
} catch (final TaskMigratedException ignoreAndRejoinGroup) { } catch (final TaskMigratedException ignoreAndRejoinGroup) {
log.warn("Detected task {} that got migrated to another thread. " + 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. " + "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{}", "Will try to rejoin the consumer group. Below is the detailed description of the task:\n{}",
ignoreAndRejoinGroup.migratedTask().id(), ignoreAndRejoinGroup.migratedTask().toString(">")); ignoreAndRejoinGroup.migratedTask().id(), ignoreAndRejoinGroup.migratedTask().toString(">"));
// re-subscribe to enforce a rebalance in the next poll call // re-subscribe to enforce a rebalance in the next poll call
consumer.unsubscribe(); consumer.unsubscribe();
@ -763,8 +799,8 @@ public class StreamThread extends Thread {
/** /**
* @throws IllegalStateException If store gets registered after initialized is already finished * @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
* @throws TaskMigratedException if another thread wrote to the changelog topic that is currently restored * @throws TaskMigratedException If another thread wrote to the changelog topic that is currently restored
* or if committing offsets failed (non-EOS) * or if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS) * or if the task producer got fenced (EOS)
*/ */
@ -772,7 +808,7 @@ public class StreamThread extends Thread {
long runOnce(final long recordsProcessedBeforeCommit) { long runOnce(final long recordsProcessedBeforeCommit) {
long processedBeforeCommit = recordsProcessedBeforeCommit; long processedBeforeCommit = recordsProcessedBeforeCommit;
ConsumerRecords<byte[], byte[]> records; final ConsumerRecords<byte[], byte[]> records;
timerStartedMs = time.milliseconds(); timerStartedMs = time.milliseconds();
@ -803,12 +839,12 @@ public class StreamThread extends Thread {
final long totalProcessed = processAndMaybeCommit(recordsProcessedBeforeCommit); final long totalProcessed = processAndMaybeCommit(recordsProcessedBeforeCommit);
if (totalProcessed > 0) { if (totalProcessed > 0) {
final long processLatency = computeLatency(); final long processLatency = computeLatency();
streamsMetrics.processTimeSensor.record(processLatency / (double) totalProcessed, streamsMetrics.processTimeSensor.record(processLatency / (double) totalProcessed, timerStartedMs);
timerStartedMs); processedBeforeCommit = adjustRecordsProcessedBeforeCommit(
processedBeforeCommit = adjustRecordsProcessedBeforeCommit(recordsProcessedBeforeCommit, recordsProcessedBeforeCommit,
totalProcessed, totalProcessed,
processLatency, processLatency,
commitTimeMs); commitTimeMs);
} }
} }
@ -890,6 +926,7 @@ public class StreamThread extends Thread {
/** /**
* Take records and add them to each respective task * Take records and add them to each respective task
*
* @param records Records, can be null * @param records Records, can be null
*/ */
private void addRecordsToTasks(final ConsumerRecords<byte[], byte[]> records) { private void addRecordsToTasks(final ConsumerRecords<byte[], byte[]> records) {
@ -900,7 +937,7 @@ public class StreamThread extends Thread {
if (task.isClosed()) { if (task.isClosed()) {
log.info("Stream task {} is already closed, probably because it got unexpectedly migrated to another thread already. " + 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); 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 * Schedule the records processing by selecting which record is processed next. Commits may
* happen as records are processed. * happen as records are processed.
*
* @param recordsProcessedBeforeCommit number of records to be processed before commit is called. * @param recordsProcessedBeforeCommit number of records to be processed before commit is called.
* if UNLIMITED_RECORDS, then commit is never called * if UNLIMITED_RECORDS, then commit is never called
* @return Number of records processed since last commit. * @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 * Adjust the number of records that should be processed by scheduler. This avoids
* scenarios where the processing time is higher than the commit time. * scenarios where the processing time is higher than the commit time.
*
* @param prevRecordsProcessedBeforeCommit Previous number of records processed by scheduler. * @param prevRecordsProcessedBeforeCommit Previous number of records processed by scheduler.
* @param totalProcessed Total number of records processed in this last round. * @param totalProcessed Total number of records processed in this last round.
* @param processLatency Total processing latency in ms 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 commitTime Desired commit time in ms.
* @return An adjusted number of records to be processed in the next round. * @return An adjusted number of records to be processed in the next round.
*/ */
private long adjustRecordsProcessedBeforeCommit(final long prevRecordsProcessedBeforeCommit, final long totalProcessed, private long adjustRecordsProcessedBeforeCommit(final long prevRecordsProcessedBeforeCommit, final long totalProcessed,
@ -978,12 +1017,12 @@ public class StreamThread extends Thread {
// push down // push down
recordsProcessedBeforeCommit = Math.max(1, (commitTime * totalProcessed) / processLatency); recordsProcessedBeforeCommit = Math.max(1, (commitTime * totalProcessed) / processLatency);
log.debug("processing latency {} > commit time {} for {} records. Adjusting down recordsProcessedBeforeCommit={}", 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) { } else if (prevRecordsProcessedBeforeCommit != UNLIMITED_RECORDS && processLatency > 0) {
// push up // push up
recordsProcessedBeforeCommit = Math.max(1, (commitTime * totalProcessed) / processLatency); recordsProcessedBeforeCommit = Math.max(1, (commitTime * totalProcessed) / processLatency);
log.debug("processing latency {} < commit time {} for {} records. Adjusting up recordsProcessedBeforeCommit={}", log.debug("processing latency {} < commit time {} for {} records. Adjusting up recordsProcessedBeforeCommit={}",
processLatency, commitTime, totalProcessed, recordsProcessedBeforeCommit); processLatency, commitTime, totalProcessed, recordsProcessedBeforeCommit);
} }
return 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 * Commit all tasks owned by this thread if specified interval time has elapsed
*
* @throws TaskMigratedException if committing offsets failed (non-EOS) * @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (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 (commitTimeMs >= 0 && lastCommitMs + commitTimeMs < now) {
if (log.isTraceEnabled()) { if (log.isTraceEnabled()) {
log.trace("Committing all active tasks {} and standby tasks {} since {}ms has elapsed (commit interval is {}ms)", 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) { if (committed > 0) {
streamsMetrics.commitTimeSensor.record(computeLatency() / (double) committed, timerStartedMs); streamsMetrics.commitTimeSensor.record(computeLatency() / (double) committed, timerStartedMs);
@ -1010,7 +1050,7 @@ public class StreamThread extends Thread {
} }
if (log.isDebugEnabled()) { if (log.isDebugEnabled()) {
log.debug("Committed all active tasks {} and standby tasks {} in {}ms", 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; lastCommitMs = now;
@ -1033,7 +1073,7 @@ public class StreamThread extends Thread {
if (task.isClosed()) { if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " + 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); throw new TaskMigratedException(task);
} }
@ -1066,7 +1106,7 @@ public class StreamThread extends Thread {
if (task.isClosed()) { if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " + 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); throw new TaskMigratedException(task);
} }
@ -1085,7 +1125,7 @@ public class StreamThread extends Thread {
if (task.isClosed()) { if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " + 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); throw new TaskMigratedException(task);
} }
@ -1112,13 +1152,13 @@ public class StreamThread extends Thread {
/** /**
* Shutdown this stream thread. * Shutdown this stream thread.
* * <p>
* Note that there is nothing to prevent this function from being called multiple times * 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 * (e.g., in testing), hence the state is set only the first time
*/ */
public void shutdown() { public void shutdown() {
log.info("Informed to shut down"); log.info("Informed to shut down");
State oldState = setState(State.PENDING_SHUTDOWN); final State oldState = setState(State.PENDING_SHUTDOWN);
if (oldState == State.CREATED) { if (oldState == State.CREATED) {
// The thread may not have been started. Take responsibility for shutting down // The thread may not have been started. Take responsibility for shutting down
completeShutdown(true); completeShutdown(true);
@ -1169,11 +1209,11 @@ public class StreamThread extends Thread {
private void updateThreadMetadata(final Map<TaskId, StreamTask> activeTasks, final Map<TaskId, StandbyTask> standbyTasks) { private void updateThreadMetadata(final Map<TaskId, StreamTask> activeTasks, final Map<TaskId, StandbyTask> standbyTasks) {
final Set<TaskMetadata> activeTasksMetadata = new HashSet<>(); 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())); activeTasksMetadata.add(new TaskMetadata(task.getKey().toString(), task.getValue().partitions()));
} }
final Set<TaskMetadata> standbyTasksMetadata = new HashSet<>(); 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())); 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. * Produces a string representation containing useful information about a StreamThread.
* This is useful in debugging scenarios. * This is useful in debugging scenarios.
*
* @return A string representation of the StreamThread instance. * @return A string representation of the StreamThread instance.
*/ */
@Override @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. * Produces a string representation containing useful information about a StreamThread, starting with the given indent.
* This is useful in debugging scenarios. * This is useful in debugging scenarios.
*
* @return A string representation of the StreamThread instance. * @return A string representation of the StreamThread instance.
*/ */
public String toString(final String indent) { public String toString(final String indent) {
final StringBuilder sb = new StringBuilder() return indent + "\tStreamsThread threadId: " + getName() + "\n" + taskManager.toString(indent);
.append(indent).append("\tStreamsThread threadId: ").append(getName()).append("\n");
sb.append(taskManager.toString(indent));
return sb.toString();
} }
// the following are for testing only // the following are for testing only

View File

@ -30,13 +30,11 @@ import org.apache.kafka.streams.StreamsMetrics;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.List;
import java.util.ArrayList;
import java.util.Arrays;
public class StreamsMetricsImpl implements StreamsMetrics { public class StreamsMetricsImpl implements StreamsMetrics {
private static final Logger log = LoggerFactory.getLogger(StreamsMetricsImpl.class); private static final Logger log = LoggerFactory.getLogger(StreamsMetricsImpl.class);
@ -44,9 +42,9 @@ public class StreamsMetricsImpl implements StreamsMetrics {
final Metrics metrics; final Metrics metrics;
final String groupName; final String groupName;
final Map<String, String> tags; 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"); Objects.requireNonNull(metrics, "Metrics cannot be null");
this.metrics = metrics; this.metrics = metrics;
@ -60,12 +58,12 @@ public class StreamsMetricsImpl implements StreamsMetrics {
} }
@Override @Override
public Sensor addSensor(String name, Sensor.RecordingLevel recordingLevel) { public Sensor addSensor(final String name, final Sensor.RecordingLevel recordingLevel) {
return metrics.sensor(name, recordingLevel); return metrics.sensor(name, recordingLevel);
} }
@Override @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); return metrics.sensor(name, recordingLevel, parents);
} }
@ -75,21 +73,21 @@ public class StreamsMetricsImpl implements StreamsMetrics {
} }
@Override @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); sensor.record(endNs - startNs);
} }
@Override @Override
public void recordThroughput(Sensor sensor, long value) { public void recordThroughput(final Sensor sensor, final long value) {
sensor.record(value); sensor.record(value);
} }
private String groupNameFromScope(String scopeName) { private String groupNameFromScope(final String scopeName) {
return "stream-" + scopeName + "-metrics"; return "stream-" + scopeName + "-metrics";
} }
private String sensorName(String operationName, String entityName) { private String sensorName(final String operationName, final String entityName) {
if (entityName == null) { if (entityName == null) {
return operationName; return operationName;
} else { } 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 // 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 != null) {
if ((tags.length % 2) != 0) if ((tags.length % 2) != 0) {
throw new IllegalArgumentException("Tags needs to be specified in key-value pairs"); throw new IllegalArgumentException("Tags needs to be specified in key-value pairs");
}
for (int i = 0; i < tags.length; i += 2) for (int i = 0; i < tags.length; i += 2)
tagMap.put(tags[i], tags[i + 1]); 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) { private Map<String, String> constructTags(final String scopeName, final String entityName, final String... tags) {
List<String> updatedTagList = new ArrayList<>(Arrays.asList(tags)); final String[] updatedTags = Arrays.copyOf(tags, tags.length + 2);
updatedTagList.add(scopeName + "-id"); updatedTags[tags.length] = scopeName + "-id";
updatedTagList.add(entityName); updatedTags[tags.length + 1] = entityName;
return tagMap(updatedTagList.toArray(new String[updatedTagList.size()])); return tagMap(updatedTags);
} }
/** /**
* @throws IllegalArgumentException if tags is not constructed in key-value pairs * @throws IllegalArgumentException if tags is not constructed in key-value pairs
*/ */
@Override @Override
public Sensor addLatencyAndThroughputSensor(String scopeName, String entityName, String operationName, public Sensor addLatencyAndThroughputSensor(final String scopeName,
Sensor.RecordingLevel recordingLevel, String... tags) { 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> tagMap = constructTags(scopeName, entityName, tags);
final Map<String, String> allTagMap = constructTags(scopeName, "all", tags); final Map<String, String> allTagMap = constructTags(scopeName, "all", tags);
// first add the global operation metrics if not yet, with the global tags only // 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); addLatencyAndThroughputMetrics(scopeName, parent, operationName, allTagMap);
// add the operation metrics with additional tags // 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); addLatencyAndThroughputMetrics(scopeName, sensor, operationName, tagMap);
parentSensors.put(sensor, parent); parentSensors.put(sensor, parent);
@ -144,16 +146,20 @@ public class StreamsMetricsImpl implements StreamsMetrics {
* @throws IllegalArgumentException if tags is not constructed in key-value pairs * @throws IllegalArgumentException if tags is not constructed in key-value pairs
*/ */
@Override @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> tagMap = constructTags(scopeName, entityName, tags);
final Map<String, String> allTagMap = constructTags(scopeName, "all", tags); final Map<String, String> allTagMap = constructTags(scopeName, "all", tags);
// first add the global operation metrics if not yet, with the global tags only // 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); addThroughputMetrics(scopeName, parent, operationName, allTagMap);
// add the operation metrics with additional tags // 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); addThroughputMetrics(scopeName, sensor, operationName, tagMap);
parentSensors.put(sensor, parent); parentSensors.put(sensor, parent);
@ -161,8 +167,10 @@ public class StreamsMetricsImpl implements StreamsMetrics {
return sensor; 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), maybeAddMetric(sensor, metrics.metricName(opName + "-latency-avg", groupNameFromScope(scopeName),
"The average latency of " + opName + " operation.", tags), new Avg()); "The average latency of " + opName + " operation.", tags), new Avg());
maybeAddMetric(sensor, metrics.metricName(opName + "-latency-max", groupNameFromScope(scopeName), maybeAddMetric(sensor, metrics.metricName(opName + "-latency-max", groupNameFromScope(scopeName),
@ -170,11 +178,14 @@ public class StreamsMetricsImpl implements StreamsMetrics {
addThroughputMetrics(scopeName, sensor, opName, tags); addThroughputMetrics(scopeName, sensor, opName, tags);
} }
private void addThroughputMetrics(String scopeName, Sensor sensor, String opName, Map<String, String> tags) { private void addThroughputMetrics(final String scopeName,
MetricName rateMetricName = metrics.metricName(opName + "-rate", groupNameFromScope(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); "The average number of occurrence of " + opName + " operation per second.", tags);
MetricName totalMetricName = metrics.metricName(opName + "-total", groupNameFromScope(scopeName), final MetricName totalMetricName = metrics.metricName(opName + "-total", groupNameFromScope(scopeName),
"The total number of occurrence of " + opName + " operations.", tags); "The total number of occurrence of " + opName + " operations.", tags);
if (!metrics.metrics().containsKey(rateMetricName) && !metrics.metrics().containsKey(totalMetricName)) { if (!metrics.metrics().containsKey(rateMetricName) && !metrics.metrics().containsKey(totalMetricName)) {
sensor.add(new Meter(new Count(), rateMetricName, totalMetricName)); sensor.add(new Meter(new Count(), rateMetricName, totalMetricName));
} else { } else {
@ -182,12 +193,16 @@ public class StreamsMetricsImpl implements StreamsMetrics {
} }
} }
public void maybeAddMetric(Sensor sensor, MetricName name, MeasurableStat stat) { /**
if (!metrics.metrics().containsKey(name)) { * Register a metric on the sensor if it isn't already there.
sensor.add(name, stat); *
} else { * @param sensor The sensor on which to register the metric
log.trace("Trying to add metric twice: {}", name); * @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() * action.run()
* endTs = time.nanoseconds() * endTs = time.nanoseconds()
* sensor.record(endTs - startTs) * sensor.record(endTs - startTs)
* @param time Time object. *
* @param action Action to run. * @param time Time object.
* @param sensor Sensor to record value. * @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; long startNs = -1;
if (sensor.shouldRecord()) { if (sensor.shouldRecord()) {
startNs = time.nanoseconds(); startNs = time.nanoseconds();
@ -215,7 +231,7 @@ public class StreamsMetricsImpl implements StreamsMetrics {
* Deletes a sensor and its parents, if any * Deletes a sensor and its parents, if any
*/ */
@Override @Override
public void removeSensor(Sensor sensor) { public void removeSensor(final Sensor sensor) {
Objects.requireNonNull(sensor, "Sensor is null"); Objects.requireNonNull(sensor, "Sensor is null");
metrics.removeSensor(sensor.name()); metrics.removeSensor(sensor.name());
@ -225,5 +241,4 @@ public class StreamsMetricsImpl implements StreamsMetrics {
} }
} }
} }

View File

@ -32,7 +32,6 @@ import java.util.Iterator;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.NavigableSet;
import java.util.Set; import java.util.Set;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.TreeSet; import java.util.TreeSet;
@ -46,7 +45,7 @@ class NamedCache {
private LRUNode tail; private LRUNode tail;
private LRUNode head; private LRUNode head;
private long currentSizeBytes; private long currentSizeBytes;
private NamedCacheMetrics namedCacheMetrics; private final NamedCacheMetrics namedCacheMetrics;
// internal stats // internal stats
private long numReadHits = 0; private long numReadHits = 0;
@ -56,7 +55,7 @@ class NamedCache {
NamedCache(final String name, final StreamsMetrics metrics) { NamedCache(final String name, final StreamsMetrics metrics) {
this.name = name; this.name = name;
this.namedCacheMetrics = new NamedCacheMetrics(metrics); this.namedCacheMetrics = new NamedCacheMetrics(metrics, name);
} }
synchronized final String name() { synchronized final String name() {
@ -105,7 +104,7 @@ class NamedCache {
if (log.isTraceEnabled()) { if (log.isTraceEnabled()) {
log.trace("Named cache {} stats on flush: #hits={}, #misses={}, #overwrites={}, #flushes={}", log.trace("Named cache {} stats on flush: #hits={}, #misses={}, #overwrites={}, #flushes={}",
name, hits(), misses(), overwrites(), flushes()); name, hits(), misses(), overwrites(), flushes());
} }
if (listener == null) { if (listener == null) {
@ -116,7 +115,7 @@ class NamedCache {
return; return;
} }
final List<ThreadCache.DirtyEntry> entries = new ArrayList<>(); final List<ThreadCache.DirtyEntry> entries = new ArrayList<>();
final List<Bytes> deleted = new ArrayList<>(); final List<Bytes> deleted = new ArrayList<>();
// evicted already been removed from the cache so add it to the list of // evicted already been removed from the cache so add it to the list of
@ -126,7 +125,7 @@ class NamedCache {
dirtyKeys.remove(evicted.key); dirtyKeys.remove(evicted.key);
} }
for (Bytes key : dirtyKeys) { for (final Bytes key : dirtyKeys) {
final LRUNode node = getInternal(key); final LRUNode node = getInternal(key);
if (node == null) { if (node == null) {
throw new IllegalStateException("Key = " + key + " found in dirty key set, but entry is 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. // clear dirtyKeys before the listener is applied as it may be re-entrant.
dirtyKeys.clear(); dirtyKeys.clear();
listener.apply(entries); listener.apply(entries);
for (Bytes key : deleted) { for (final Bytes key : deleted) {
delete(key); delete(key);
} }
} }
synchronized void put(final Bytes key, final LRUCacheEntry value) { synchronized void put(final Bytes key, final LRUCacheEntry value) {
if (!value.isDirty() && dirtyKeys.contains(key)) { if (!value.isDirty() && dirtyKeys.contains(key)) {
throw new IllegalStateException(String.format("Attempting to put a clean entry for key [%s] " + throw new IllegalStateException(
"into NamedCache [%s] when it already contains " + String.format(
"a dirty entry for the same key", "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)); key, name
)
);
} }
LRUNode node = cache.get(key); LRUNode node = cache.get(key);
if (node != null) { if (node != null) {
@ -190,13 +191,13 @@ class NamedCache {
return node; return node;
} }
private void updateLRU(LRUNode node) { private void updateLRU(final LRUNode node) {
remove(node); remove(node);
putHead(node); putHead(node);
} }
private void remove(LRUNode node) { private void remove(final LRUNode node) {
if (node.previous != null) { if (node.previous != null) {
node.previous.next = node.next; node.previous.next = node.next;
} else { } else {
@ -209,7 +210,7 @@ class NamedCache {
} }
} }
private void putHead(LRUNode node) { private void putHead(final LRUNode node) {
node.next = head; node.next = head;
node.previous = null; node.previous = null;
if (head != null) { if (head != null) {
@ -243,7 +244,7 @@ class NamedCache {
} }
synchronized void putAll(final List<KeyValue<byte[], LRUCacheEntry>> entries) { 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); put(Bytes.wrap(entry.key), entry.value);
} }
} }
@ -271,18 +272,12 @@ class NamedCache {
} }
private Iterator<Bytes> keySetIterator(final Set<Bytes> keySet) { private Iterator<Bytes> keySetIterator(final Set<Bytes> keySet) {
final TreeSet<Bytes> copy = new TreeSet<>(); return new TreeSet<>(keySet).iterator();
copy.addAll(keySet);
return copy.iterator();
} }
synchronized Iterator<Bytes> allKeys() { synchronized Iterator<Bytes> allKeys() {
return keySetIterator(cache.navigableKeySet()); return keySetIterator(cache.navigableKeySet());
} }
synchronized NavigableSet<Bytes> keySet() {
return cache.navigableKeySet();
}
synchronized LRUCacheEntry first() { synchronized LRUCacheEntry first() {
if (head == null) { if (head == null) {
@ -338,11 +333,11 @@ class NamedCache {
} }
long size() { long size() {
return key.get().length + return key.get().length +
8 + // entry 8 + // entry
8 + // previous 8 + // previous
8 + // next 8 + // next
entry.size(); entry.size();
} }
LRUNode next() { LRUNode next() {
@ -353,19 +348,19 @@ class NamedCache {
return previous; return previous;
} }
private void update(LRUCacheEntry entry) { private void update(final LRUCacheEntry entry) {
this.entry = entry; this.entry = entry;
} }
} }
class NamedCacheMetrics { private static class NamedCacheMetrics {
final StreamsMetricsImpl metrics; private final StreamsMetricsImpl metrics;
final String groupName; private final String groupName;
final Map<String, String> metricTags; private final Map<String, String> metricTags;
final Map<String, String> allMetricTags; private final Map<String, String> allMetricTags;
final Sensor hitRatioSensor; private final Sensor hitRatioSensor;
public NamedCacheMetrics(StreamsMetrics metrics) { private NamedCacheMetrics(final StreamsMetrics metrics, final String name) {
final String scope = "record-cache"; final String scope = "record-cache";
final String opName = "hitRatio"; final String opName = "hitRatio";
final String tagKey = scope + "-id"; final String tagKey = scope + "-id";
@ -373,18 +368,18 @@ class NamedCache {
this.groupName = "stream-" + scope + "-metrics"; this.groupName = "stream-" + scope + "-metrics";
this.metrics = (StreamsMetricsImpl) metrics; this.metrics = (StreamsMetricsImpl) metrics;
this.allMetricTags = ((StreamsMetricsImpl) metrics).tagMap(tagKey, "all", this.allMetricTags = ((StreamsMetricsImpl) metrics).tagMap(tagKey, "all",
"task-id", ThreadCache.taskIDfromCacheName(name)); "task-id", ThreadCache.taskIDfromCacheName(name));
this.metricTags = ((StreamsMetricsImpl) metrics).tagMap(tagKey, tagValue, this.metricTags = ((StreamsMetricsImpl) metrics).tagMap(tagKey, tagValue,
"task-id", ThreadCache.taskIDfromCacheName(name)); "task-id", ThreadCache.taskIDfromCacheName(name));
// add parent // 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, ((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, ((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, ((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 // add child
hitRatioSensor = this.metrics.registry().sensor(opName, Sensor.RecordingLevel.DEBUG, parent); 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); metrics.removeSensor(hitRatioSensor);
} }
} }

View File

@ -73,40 +73,42 @@ public class GlobalKTableJoinsTest {
@Test @Test
public void shouldLeftJoinWithStream() { public void shouldLeftJoinWithStream() {
stream.leftJoin(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER) stream
.foreach(action); .leftJoin(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER)
.foreach(action);
final Map<String, String> expected = new HashMap<>(); final Map<String, String> expected = new HashMap<>();
expected.put("1", "a+A"); expected.put("1", "a+A");
expected.put("2", "b+B"); expected.put("2", "b+B");
expected.put("3", "c+null"); expected.put("3", "c+null");
verifyJoin(expected, streamTopic); verifyJoin(expected);
} }
@Test @Test
public void shouldInnerJoinWithStream() { public void shouldInnerJoinWithStream() {
stream.join(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER) stream
.foreach(action); .join(global, keyValueMapper, MockValueJoiner.TOSTRING_JOINER)
.foreach(action);
final Map<String, String> expected = new HashMap<>(); final Map<String, String> expected = new HashMap<>();
expected.put("1", "a+A"); expected.put("1", "a+A");
expected.put("2", "b+B"); 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.setUp(builder, stateDir);
driver.setTime(0L); driver.setTime(0L);
// write some data to the global table // write some data to the global table
driver.process(globalTopic, "a", "A"); driver.process(globalTopic, "a", "A");
driver.process(globalTopic, "b", "B"); driver.process(globalTopic, "b", "B");
//write some data to the stream //write some data to the stream
driver.process(joinInput, "1", "a"); driver.process(streamTopic, "1", "a");
driver.process(joinInput, "2", "b"); driver.process(streamTopic, "2", "b");
driver.process(joinInput, "3", "c"); driver.process(streamTopic, "3", "c");
driver.flushState(); driver.flushState();
assertEquals(expected, results); assertEquals(expected, results);

View File

@ -220,7 +220,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldAggregateSessionWindows() { public void shouldAggregateSessionWindows() {
final Map<Windowed<String>, Integer> results = new HashMap<>(); 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 @Override
public Integer apply() { public Integer apply() {
return 0; return 0;
@ -251,7 +251,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldAggregateSessionWindowsWithInternalStoreName() { public void shouldAggregateSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, Integer> results = new HashMap<>(); 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 @Override
public Integer apply() { public Integer apply() {
return 0; return 0;
@ -301,7 +301,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldCountSessionWindows() { public void shouldCountSessionWindows() {
final Map<Windowed<String>, Long> results = new HashMap<>(); 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>() { table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() {
@Override @Override
public void apply(final Windowed<String> key, final Long value) { public void apply(final Windowed<String> key, final Long value) {
@ -316,7 +316,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldCountSessionWindowsWithInternalStoreName() { public void shouldCountSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, Long> results = new HashMap<>(); 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>() { table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() {
@Override @Override
public void apply(final Windowed<String> key, final Long value) { public void apply(final Windowed<String> key, final Long value) {
@ -351,14 +351,16 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldReduceSessionWindows() { public void shouldReduceSessionWindows() {
final Map<Windowed<String>, String> results = new HashMap<>(); final Map<Windowed<String>, String> results = new HashMap<>();
KTable<Windowed<String>, String> table = groupedStream.reduce( final KTable<Windowed<String>, String> table = groupedStream.reduce(
new Reducer<String>() { new Reducer<String>() {
@Override @Override
public String apply(final String value1, final String value2) { public String apply(final String value1, final String value2) {
return value1 + ":" + value2; return value1 + ":" + value2;
} }
}, SessionWindows.with(30), },
"session-store"); SessionWindows.with(30),
"session-store"
);
table.toStream().foreach(new ForeachAction<Windowed<String>, String>() { table.toStream().foreach(new ForeachAction<Windowed<String>, String>() {
@Override @Override
public void apply(final Windowed<String> key, final String value) { public void apply(final Windowed<String> key, final String value) {
@ -373,13 +375,15 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldReduceSessionWindowsWithInternalStoreName() { public void shouldReduceSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, String> results = new HashMap<>(); final Map<Windowed<String>, String> results = new HashMap<>();
KTable<Windowed<String>, String> table = groupedStream.reduce( final KTable<Windowed<String>, String> table = groupedStream.reduce(
new Reducer<String>() { new Reducer<String>() {
@Override @Override
public String apply(final String value1, final String value2) { public String apply(final String value1, final String value2) {
return value1 + ":" + value2; return value1 + ":" + value2;
} }
}, SessionWindows.with(30)); },
SessionWindows.with(30)
);
table.toStream().foreach(new ForeachAction<Windowed<String>, String>() { table.toStream().foreach(new ForeachAction<Windowed<String>, String>() {
@Override @Override
public void apply(final Windowed<String> key, final String value) { public void apply(final Windowed<String> key, final String value) {
@ -445,12 +449,13 @@ public class KGroupedStreamImplTest {
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldNotAcceptNullSessionMergerWhenAggregatingSessionWindows() { public void shouldNotAcceptNullSessionMergerWhenAggregatingSessionWindows() {
groupedStream.aggregate(MockInitializer.STRING_INIT, groupedStream.aggregate(
MockAggregator.TOSTRING_ADDER, MockInitializer.STRING_INIT,
null, MockAggregator.TOSTRING_ADDER,
SessionWindows.with(10), null,
Serdes.String(), SessionWindows.with(10),
"storeName"); Serdes.String(),
"storeName");
} }
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@ -539,15 +544,14 @@ public class KGroupedStreamImplTest {
groupedStream.count((Materialized) null); groupedStream.count((Materialized) null);
} }
@SuppressWarnings("unchecked")
@Test @Test
public void shouldCountAndMaterializeResults() { public void shouldCountAndMaterializeResults() {
groupedStream.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("count") groupedStream.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("count").withKeySerde(Serdes.String()));
.withKeySerde(Serdes.String()));
processData(); 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("1"), equalTo(3L));
assertThat(count.get("2"), equalTo(1L)); assertThat(count.get("2"), equalTo(1L));
@ -555,14 +559,14 @@ public class KGroupedStreamImplTest {
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void shouldReduceAndMaterializeResults() { public void shouldReduceAndMaterializeResults() {
groupedStream.reduce(MockReducer.STRING_ADDER, groupedStream.reduce(
Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("reduce") MockReducer.STRING_ADDER,
.withKeySerde(Serdes.String()) Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("reduce")
.withValueSerde(Serdes.String())); .withKeySerde(Serdes.String())
.withValueSerde(Serdes.String()));
processData(); processData();
@ -576,11 +580,12 @@ public class KGroupedStreamImplTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void shouldAggregateAndMaterializeResults() { public void shouldAggregateAndMaterializeResults() {
groupedStream.aggregate(MockInitializer.STRING_INIT, groupedStream.aggregate(
MockAggregator.TOSTRING_ADDER, MockInitializer.STRING_INIT,
Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("aggregate") MockAggregator.TOSTRING_ADDER,
.withKeySerde(Serdes.String()) Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("aggregate")
.withValueSerde(Serdes.String())); .withKeySerde(Serdes.String())
.withValueSerde(Serdes.String()));
processData(); processData();
@ -595,15 +600,16 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldAggregateWithDefaultSerdes() { public void shouldAggregateWithDefaultSerdes() {
final Map<String, String> results = new HashMap<>(); final Map<String, String> results = new HashMap<>();
groupedStream.aggregate(MockInitializer.STRING_INIT, groupedStream.aggregate(
MockAggregator.TOSTRING_ADDER) MockInitializer.STRING_INIT,
.toStream() MockAggregator.TOSTRING_ADDER)
.foreach(new ForeachAction<String, String>() { .toStream()
@Override .foreach(new ForeachAction<String, String>() {
public void apply(final String key, final String value) { @Override
results.put(key, value); public void apply(final String key, final String value) {
} results.put(key, value);
}); }
});
processData(); processData();
@ -637,13 +643,13 @@ public class KGroupedStreamImplTest {
driver.process(TOPIC, "2", "B"); driver.process(TOPIC, "2", "B");
driver.process(TOPIC, "2", "B"); driver.process(TOPIC, "2", "B");
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 1L), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 1L),
KeyValue.pair(new Windowed<>("2", 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<>("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)), 1L),
KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 2L), 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)), 1L),
KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 2L) KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 2L)
))); )));
} }
@ -652,15 +658,15 @@ public class KGroupedStreamImplTest {
public void shouldCountWindowed() { public void shouldCountWindowed() {
final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>(); final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
groupedStream.count( groupedStream.count(
TimeWindows.of(500L), TimeWindows.of(500L),
"aggregate-by-key-windowed") "aggregate-by-key-windowed")
.toStream() .toStream()
.foreach(new ForeachAction<Windowed<String>, Long>() { .foreach(new ForeachAction<Windowed<String>, Long>() {
@Override @Override
public void apply(final Windowed<String> key, final Long value) { public void apply(final Windowed<String> key, final Long value) {
results.add(KeyValue.pair(key, value)); results.add(KeyValue.pair(key, value));
} }
}); });
doCountWindowed(results); doCountWindowed(results);
} }
@ -670,14 +676,14 @@ public class KGroupedStreamImplTest {
public void shouldCountWindowedWithInternalStoreName() { public void shouldCountWindowedWithInternalStoreName() {
final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>(); final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
groupedStream.count( groupedStream.count(
TimeWindows.of(500L)) TimeWindows.of(500L))
.toStream() .toStream()
.foreach(new ForeachAction<Windowed<String>, Long>() { .foreach(new ForeachAction<Windowed<String>, Long>() {
@Override @Override
public void apply(final Windowed<String> key, final Long value) { public void apply(final Windowed<String> key, final Long value) {
results.add(KeyValue.pair(key, value)); results.add(KeyValue.pair(key, value));
} }
}); });
doCountWindowed(results); doCountWindowed(results);
} }

View File

@ -25,8 +25,8 @@ import org.apache.kafka.streams.kstream.JoinWindows;
import org.apache.kafka.streams.kstream.Joined; import org.apache.kafka.streams.kstream.Joined;
import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; 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.InternalMockProcessorContext;
import org.apache.kafka.test.KStreamTestDriver;
import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockProcessorSupplier;
import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.MockValueJoiner;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
@ -35,7 +35,6 @@ import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashSet; import java.util.HashSet;
@ -57,31 +56,32 @@ public class KStreamKStreamJoinTest {
private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde); private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde);
@Before @Before
public void setUp() throws IOException { public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test"); stateDir = TestUtils.tempDirectory("kafka-test");
} }
@Test @Test
public void testJoin() { public void testJoin() {
StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KStream<Integer, String> stream1; final KStream<Integer, String> stream1;
KStream<Integer, String> stream2; final KStream<Integer, String> stream2;
KStream<Integer, String> joined; final KStream<Integer, String> joined;
MockProcessorSupplier<Integer, String> processor; final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>(); processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed); stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed); stream2 = builder.stream(topic2, consumed);
joined = stream1.join(stream2, joined = stream1.join(
MockValueJoiner.TOSTRING_JOINER, stream2,
JoinWindows.of(100), MockValueJoiner.TOSTRING_JOINER,
Joined.with(intSerde, stringSerde, stringSerde)); JoinWindows.of(100),
Joined.with(intSerde, stringSerde, stringSerde));
joined.process(processor); joined.process(processor);
Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder); final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size()); assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); 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 } // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
// w2 = { 0:Y0, 1:Y1 } // w2 = { 0:Y0, 1:Y1 }
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey); 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 } // --> 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 } // 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); 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 } // --> 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 } // 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); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
@ -164,25 +164,26 @@ public class KStreamKStreamJoinTest {
@Test @Test
public void testOuterJoin() { public void testOuterJoin() {
StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KStream<Integer, String> stream1; final KStream<Integer, String> stream1;
KStream<Integer, String> stream2; final KStream<Integer, String> stream2;
KStream<Integer, String> joined; final KStream<Integer, String> joined;
MockProcessorSupplier<Integer, String> processor; final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>(); processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed); stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed); stream2 = builder.stream(topic2, consumed);
joined = stream1.outerJoin(stream2, joined = stream1.outerJoin(
MockValueJoiner.TOSTRING_JOINER, stream2,
JoinWindows.of(100), MockValueJoiner.TOSTRING_JOINER,
Joined.with(intSerde, stringSerde, stringSerde)); JoinWindows.of(100),
Joined.with(intSerde, stringSerde, stringSerde));
joined.process(processor); joined.process(processor);
Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder); final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size()); assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); 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 } // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
// w2 = { 0:Y0, 1:Y1 } // w2 = { 0:Y0, 1:Y1 }
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "X" + expectedKey); 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 } // --> 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 } // 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); 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 } // --> 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 } // 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); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
@ -267,26 +268,27 @@ public class KStreamKStreamJoinTest {
public void testWindowing() { public void testWindowing() {
long time = 0L; long time = 0L;
StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KStream<Integer, String> stream1; final KStream<Integer, String> stream1;
KStream<Integer, String> stream2; final KStream<Integer, String> stream2;
KStream<Integer, String> joined; final KStream<Integer, String> joined;
MockProcessorSupplier<Integer, String> processor; final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>(); processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed); stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed); stream2 = builder.stream(topic2, consumed);
joined = stream1.join(stream2, joined = stream1.join(
MockValueJoiner.TOSTRING_JOINER, stream2,
JoinWindows.of(100), MockValueJoiner.TOSTRING_JOINER,
Joined.with(intSerde, stringSerde, stringSerde)); JoinWindows.of(100),
Joined.with(intSerde, stringSerde, stringSerde));
joined.process(processor); joined.process(processor);
Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder); final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size()); assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
@ -334,35 +336,35 @@ public class KStreamKStreamJoinTest {
time = 1000 + 100L; time = 1000 + 100L;
setRecordContext(time, topic2); setRecordContext(time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("3:X3+YY3"); processor.checkAndClearProcessResult("3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
@ -372,35 +374,35 @@ public class KStreamKStreamJoinTest {
time = 1000L - 100L - 1L; time = 1000L - 100L - 1L;
setRecordContext(time, topic2); setRecordContext(time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult(); processor.checkAndClearProcessResult();
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0"); processor.checkAndClearProcessResult("0:X0+YY0");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
@ -420,35 +422,35 @@ public class KStreamKStreamJoinTest {
time = 2000L + 100L; time = 2000L + 100L;
setRecordContext(time, topic1); setRecordContext(time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
setRecordContext(++time, topic1); setRecordContext(++time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
setRecordContext(++time, topic1); setRecordContext(++time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3"); processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3");
setRecordContext(++time, topic1); setRecordContext(++time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
processor.checkAndClearProcessResult("3:XX3+Y3"); processor.checkAndClearProcessResult("3:XX3+Y3");
setRecordContext(++time, topic1); setRecordContext(++time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
@ -458,35 +460,35 @@ public class KStreamKStreamJoinTest {
time = 2000L - 100L - 1L; time = 2000L - 100L - 1L;
setRecordContext(time, topic1); setRecordContext(time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
processor.checkAndClearProcessResult(); processor.checkAndClearProcessResult();
setRecordContext(++time, topic1); setRecordContext(++time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
processor.checkAndClearProcessResult("0:XX0+Y0"); processor.checkAndClearProcessResult("0:XX0+Y0");
setRecordContext(++time, topic1); setRecordContext(++time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1"); processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1");
setRecordContext(++time, topic1); setRecordContext(++time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2"); processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2");
setRecordContext(++time, topic1); setRecordContext(++time, topic1);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic1, expectedKey, "XX" + expectedKey); driver.process(topic1, expectedKey, "XX" + expectedKey);
} }
@ -494,31 +496,32 @@ public class KStreamKStreamJoinTest {
} }
@Test @Test
public void testAsymetricWindowingAfter() { public void testAsymmetricWindowingAfter() {
long time = 1000L; long time = 1000L;
StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KStream<Integer, String> stream1; final KStream<Integer, String> stream1;
KStream<Integer, String> stream2; final KStream<Integer, String> stream2;
KStream<Integer, String> joined; final KStream<Integer, String> joined;
MockProcessorSupplier<Integer, String> processor; final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>(); processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed); stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed); stream2 = builder.stream(topic2, consumed);
joined = stream1.join(stream2, joined = stream1.join(
MockValueJoiner.TOSTRING_JOINER, stream2,
JoinWindows.of(0).after(100), MockValueJoiner.TOSTRING_JOINER,
Joined.with(intSerde, JoinWindows.of(0).after(100),
stringSerde, Joined.with(intSerde,
stringSerde)); stringSerde,
stringSerde));
joined.process(processor); joined.process(processor);
Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder); final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size()); assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
@ -535,35 +538,35 @@ public class KStreamKStreamJoinTest {
time = 1000L - 1L; time = 1000L - 1L;
setRecordContext(time, topic2); setRecordContext(time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult(); processor.checkAndClearProcessResult();
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0"); processor.checkAndClearProcessResult("0:X0+YY0");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
@ -572,35 +575,35 @@ public class KStreamKStreamJoinTest {
time = 1000 + 100L; time = 1000 + 100L;
setRecordContext(time, topic2); setRecordContext(time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("3:X3+YY3"); processor.checkAndClearProcessResult("3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
@ -608,29 +611,30 @@ public class KStreamKStreamJoinTest {
} }
@Test @Test
public void testAsymetricWindowingBefore() { public void testAsymmetricWindowingBefore() {
long time = 1000L; long time = 1000L;
StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KStream<Integer, String> stream1; final KStream<Integer, String> stream1;
KStream<Integer, String> stream2; final KStream<Integer, String> stream2;
KStream<Integer, String> joined; final KStream<Integer, String> joined;
MockProcessorSupplier<Integer, String> processor; final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>(); processor = new MockProcessorSupplier<>();
stream1 = builder.stream(topic1, consumed); stream1 = builder.stream(topic1, consumed);
stream2 = builder.stream(topic2, consumed); stream2 = builder.stream(topic2, consumed);
joined = stream1.join(stream2, joined = stream1.join(
MockValueJoiner.TOSTRING_JOINER, stream2,
JoinWindows.of(0).before(100), MockValueJoiner.TOSTRING_JOINER,
Joined.with(intSerde, stringSerde, stringSerde)); JoinWindows.of(0).before(100),
Joined.with(intSerde, stringSerde, stringSerde));
joined.process(processor); joined.process(processor);
Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder); final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size()); assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
@ -647,35 +651,35 @@ public class KStreamKStreamJoinTest {
time = 1000L - 100L - 1L; time = 1000L - 100L - 1L;
setRecordContext(time, topic2); setRecordContext(time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult(); processor.checkAndClearProcessResult();
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0"); processor.checkAndClearProcessResult("0:X0+YY0");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
@ -683,35 +687,35 @@ public class KStreamKStreamJoinTest {
time = 1000L; time = 1000L;
setRecordContext(time, topic2); setRecordContext(time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3"); processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }
processor.checkAndClearProcessResult("3:X3+YY3"); processor.checkAndClearProcessResult("3:X3+YY3");
setRecordContext(++time, topic2); setRecordContext(++time, topic2);
for (int expectedKey : expectedKeys) { for (final int expectedKey : expectedKeys) {
driver.process(topic2, expectedKey, "YY" + expectedKey); driver.process(topic2, expectedKey, "YY" + expectedKey);
} }

View File

@ -32,7 +32,6 @@ import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashSet; import java.util.HashSet;
@ -49,14 +48,13 @@ public class KStreamKTableJoinTest {
final private Serde<String> stringSerde = Serdes.String(); final private Serde<String> stringSerde = Serdes.String();
@Rule @Rule
public final KStreamTestDriver driver = new KStreamTestDriver(); public final KStreamTestDriver driver = new KStreamTestDriver();
private File stateDir = null;
private MockProcessorSupplier<Integer, String> processor; private MockProcessorSupplier<Integer, String> processor;
private final int[] expectedKeys = {0, 1, 2, 3}; private final int[] expectedKeys = {0, 1, 2, 3};
private StreamsBuilder builder; private StreamsBuilder builder;
@Before @Before
public void setUp() throws IOException { public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test"); final File stateDir = TestUtils.tempDirectory("kafka-test");
builder = new StreamsBuilder(); builder = new StreamsBuilder();
@ -86,8 +84,8 @@ public class KStreamKTableJoinTest {
} }
} }
private void pushNullValueToTable(final int messageCount) { private void pushNullValueToTable() {
for (int i = 0; i < messageCount; i++) { for (int i = 0; i < 2; i++) {
driver.process(tableTopic, expectedKeys[i], null); 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. // push two items with null to the table as deletes. this should not produce any item.
pushNullValueToTable(2); pushNullValueToTable();
processor.checkAndClearProcessResult(); processor.checkAndClearProcessResult();
// push all four items to the primary stream. this should produce two items. // push all four items to the primary stream. this should produce two items.

View File

@ -28,8 +28,8 @@ import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
import org.apache.kafka.streams.state.KeyValueIterator; 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.SessionStore;
import org.apache.kafka.streams.state.internals.RocksDBSessionStoreSupplier;
import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.streams.state.internals.ThreadCache;
import org.apache.kafka.test.InternalMockProcessorContext; import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.NoOpRecordCollector; import org.apache.kafka.test.NoOpRecordCollector;
@ -72,14 +72,15 @@ public class KStreamSessionWindowAggregateProcessorTest {
} }
}; };
private final KStreamSessionWindowAggregate<String, String, Long> sessionAggregator = private final KStreamSessionWindowAggregate<String, String, Long> sessionAggregator =
new KStreamSessionWindowAggregate<>(SessionWindows.with(GAP_MS).until(3 * GAP_MS), new KStreamSessionWindowAggregate<>(
STORE_NAME, SessionWindows.with(GAP_MS).until(3 * GAP_MS),
initializer, STORE_NAME,
aggregator, initializer,
sessionMerger); aggregator,
sessionMerger);
private final List<KeyValue> results = new ArrayList<>(); 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 SessionStore<String, Long> sessionStore;
private InternalMockProcessorContext context; private InternalMockProcessorContext context;
@ -101,14 +102,15 @@ public class KStreamSessionWindowAggregateProcessorTest {
private void initStore(final boolean enableCaching) { private void initStore(final boolean enableCaching) {
final RocksDBSessionStoreSupplier<String, Long> supplier = final RocksDBSessionStoreSupplier<String, Long> supplier =
new RocksDBSessionStoreSupplier<>(STORE_NAME, new RocksDBSessionStoreSupplier<>(
GAP_MS * 3, STORE_NAME,
Serdes.String(), GAP_MS * 3,
Serdes.Long(), Serdes.String(),
false, Serdes.Long(),
Collections.<String, String>emptyMap(), false,
enableCaching); Collections.<String, String>emptyMap(),
sessionStore = (SessionStore<String, Long>) supplier.get(); enableCaching);
sessionStore = supplier.get();
sessionStore.init(context, sessionStore); sessionStore.init(context, sessionStore);
} }
@ -180,12 +182,14 @@ public class KStreamSessionWindowAggregateProcessorTest {
processor.process(sessionId, "third"); processor.process(sessionId, "third");
sessionStore.flush(); 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(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(GAP_MS + 1, GAP_MS + 1)), new Change<>(2L, null)),
KeyValue.pair(new Windowed<>(sessionId, new SessionWindow(time, time)), new Change<>(3L, 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(); sessionStore.flush();
assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)), assertEquals(
KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), new Change<>(1L, null)), Arrays.asList(
KeyValue.pair(new Windowed<>("c", new SessionWindow(0, 0)), new Change<>(1L, null)), KeyValue.pair(new Windowed<>("a", 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(0, 0)), new Change<>(1L, null)),
KeyValue.pair(new Windowed<>("b", new SessionWindow(GAP_MS + 1, GAP_MS + 1)), new Change<>(1L, null)), KeyValue.pair(new Windowed<>("c", new SessionWindow(0, 0)), 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<>("d", new SessionWindow(0, 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)) 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)),
results); 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("b", "1");
processor.process("c", "1"); processor.process("c", "1");
assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)), assertEquals(
KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), new Change<>(1L, null)), Arrays.asList(
KeyValue.pair(new Windowed<>("c", new SessionWindow(0, 0)), new Change<>(1L, null))), results); 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 @Test
@ -277,9 +289,14 @@ public class KStreamSessionWindowAggregateProcessorTest {
processor.process("a", "1"); processor.process("a", "1");
context.setTime(5); context.setTime(5);
processor.process("a", "1"); processor.process("a", "1");
assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(1L, null)), assertEquals(
KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), new Change<>(null, null)), Arrays.asList(
KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 5)), new Change<>(2L, null))), results); 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
);
} }

View File

@ -21,17 +21,16 @@ import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.Consumed; import org.apache.kafka.streams.Consumed;
import org.apache.kafka.streams.StreamsBuilder; 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.KTable;
import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.Serialized;
import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.TimeWindows;
import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueJoiner;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; 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.KStreamTestDriver;
import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockAggregator;
import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockInitializer;
import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockProcessorSupplier;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.Before; import org.junit.Before;
@ -39,7 +38,6 @@ import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.IOException;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -51,24 +49,21 @@ public class KStreamWindowAggregateTest {
public final KStreamTestDriver driver = new KStreamTestDriver(); public final KStreamTestDriver driver = new KStreamTestDriver();
@Before @Before
public void setUp() throws IOException { public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test"); stateDir = TestUtils.tempDirectory("kafka-test");
} }
@Test @Test
public void testAggBasic() { public void testAggBasic() {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
String topic1 = "topic1"; final String topic1 = "topic1";
KStream<String, String> stream1 = builder.stream(topic1, Consumed.with(strSerde, strSerde)); final KTable<Windowed<String>, String> table2 = builder
KTable<Windowed<String>, String> table2 = .stream(topic1, Consumed.with(strSerde, strSerde))
stream1.groupByKey(Serialized.with(strSerde, strSerde)) .groupByKey(Serialized.with(strSerde, strSerde))
.aggregate(MockInitializer.STRING_INIT, .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10).advanceBy(5), strSerde, "topic1-Canonized");
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); table2.toStream().process(proc2);
driver.setUp(builder, stateDir); driver.setUp(builder, stateDir);
@ -122,7 +117,8 @@ public class KStreamWindowAggregateTest {
driver.flushState(); driver.flushState();
assertEquals(Utils.mkList( assertEquals(
Utils.mkList(
"[A@0/10]:0+1", "[A@0/10]:0+1",
"[B@0/10]:0+2", "[B@0/10]:0+2",
"[C@0/10]:0+3", "[C@0/10]:0+3",
@ -130,55 +126,51 @@ public class KStreamWindowAggregateTest {
"[A@0/10]:0+1+1", "[A@0/10]:0+1+1",
"[A@0/10]:0+1+1+1", "[A@5/15]:0+1", "[A@0/10]:0+1+1+1", "[A@5/15]:0+1",
"[B@0/10]:0+2+2", "[B@5/15]:0+2", "[B@0/10]:0+2+2", "[B@5/15]:0+2",
"[D@0/10]:0+4+4", "[D@5/15]:0+4", "[D@0/10]:0+4+4", "[D@5/15]:0+4",
"[B@0/10]:0+2+2+2", "[B@5/15]:0+2+2", "[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", "[A@5/15]:0+1+1", "[A@10/20]:0+1",
"[B@5/15]:0+2+2+2", "[B@10/20]:0+2", "[B@5/15]:0+2+2+2", "[B@10/20]:0+2",
"[D@5/15]:0+4+4", "[D@10/20]:0+4", "[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", "[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)); ((InternalMockProcessorContext) driver.context()).setRecordContext(new ProcessorRecordContext(time, 0, 0, topic));
} }
@Test @Test
public void testJoin() { public void testJoin() {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
String topic1 = "topic1"; final String topic1 = "topic1";
String topic2 = "topic2"; final String topic2 = "topic2";
KStream<String, String> stream1 = builder.stream(topic1, Consumed.with(strSerde, strSerde)); final KTable<Windowed<String>, String> table1 = builder
KTable<Windowed<String>, String> table1 = .stream(topic1, Consumed.with(strSerde, strSerde))
stream1.groupByKey(Serialized.with(strSerde, strSerde)) .groupByKey(Serialized.with(strSerde, strSerde))
.aggregate(MockInitializer.STRING_INIT, .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10).advanceBy(5), strSerde, "topic1-Canonized");
MockAggregator.TOSTRING_ADDER,
TimeWindows.of(10).advanceBy(5),
strSerde, "topic1-Canonized");
MockProcessorSupplier<Windowed<String>, String> proc1 = new MockProcessorSupplier<>(); final MockProcessorSupplier<Windowed<String>, String> proc1 = new MockProcessorSupplier<>();
table1.toStream().process(proc1); table1.toStream().process(proc1);
KStream<String, String> stream2 = builder.stream(topic2, Consumed.with(strSerde, strSerde)); final KTable<Windowed<String>, String> table2 = builder
KTable<Windowed<String>, String> table2 = .stream(topic2, Consumed.with(strSerde, strSerde)).groupByKey(Serialized.with(strSerde, strSerde))
stream2.groupByKey(Serialized.with(strSerde, strSerde)) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10).advanceBy(5), strSerde, "topic2-Canonized");
.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); 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>() { table1.join(table2, new ValueJoiner<String, String, String>() {
@Override @Override
public String apply(String p1, String p2) { public String apply(final String p1, final String p2) {
return p1 + "%" + p2; return p1 + "%" + p2;
} }
}).toStream().process(proc3); }).toStream().process(proc3);
@ -202,11 +194,11 @@ public class KStreamWindowAggregateTest {
driver.flushState(); driver.flushState();
proc1.checkAndClearProcessResult( proc1.checkAndClearProcessResult(
"[A@0/10]:0+1", "[A@0/10]:0+1",
"[B@0/10]:0+2", "[B@0/10]:0+2",
"[C@0/10]:0+3", "[C@0/10]:0+3",
"[D@0/10]:0+4", "[D@0/10]:0+4",
"[A@0/10]:0+1+1" "[A@0/10]:0+1+1"
); );
proc2.checkAndClearProcessResult(); proc2.checkAndClearProcessResult();
proc3.checkAndClearProcessResult(); proc3.checkAndClearProcessResult();
@ -228,11 +220,11 @@ public class KStreamWindowAggregateTest {
driver.flushState(); driver.flushState();
proc1.checkAndClearProcessResult( proc1.checkAndClearProcessResult(
"[A@0/10]:0+1+1+1", "[A@5/15]:0+1", "[A@0/10]:0+1+1+1", "[A@5/15]:0+1",
"[B@0/10]:0+2+2", "[B@5/15]:0+2", "[B@0/10]:0+2+2", "[B@5/15]:0+2",
"[D@0/10]:0+4+4", "[D@5/15]:0+4", "[D@0/10]:0+4+4", "[D@5/15]:0+4",
"[B@0/10]:0+2+2+2", "[B@5/15]:0+2+2", "[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"
); );
proc2.checkAndClearProcessResult(); proc2.checkAndClearProcessResult();
proc3.checkAndClearProcessResult(); proc3.checkAndClearProcessResult();
@ -255,18 +247,18 @@ public class KStreamWindowAggregateTest {
proc1.checkAndClearProcessResult(); proc1.checkAndClearProcessResult();
proc2.checkAndClearProcessResult( proc2.checkAndClearProcessResult(
"[A@0/10]:0+a", "[A@0/10]:0+a",
"[B@0/10]:0+b", "[B@0/10]:0+b",
"[C@0/10]:0+c", "[C@0/10]:0+c",
"[D@0/10]:0+d", "[D@0/10]:0+d",
"[A@0/10]:0+a+a" "[A@0/10]:0+a+a"
); );
proc3.checkAndClearProcessResult( proc3.checkAndClearProcessResult(
"[A@0/10]:0+1+1+1%0+a", "[A@0/10]:0+1+1+1%0+a",
"[B@0/10]:0+2+2+2%0+b", "[B@0/10]:0+2+2+2%0+b",
"[C@0/10]:0+3+3%0+c", "[C@0/10]:0+3+3%0+c",
"[D@0/10]:0+4+4%0+d", "[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+a");
setRecordContext(5, topic1); setRecordContext(5, topic1);
driver.process(topic2, "A", "a"); driver.process(topic2, "A", "a");
@ -285,18 +277,18 @@ public class KStreamWindowAggregateTest {
driver.flushState(); driver.flushState();
proc1.checkAndClearProcessResult(); proc1.checkAndClearProcessResult();
proc2.checkAndClearProcessResult( proc2.checkAndClearProcessResult(
"[A@0/10]:0+a+a+a", "[A@5/15]:0+a", "[A@0/10]:0+a+a+a", "[A@5/15]:0+a",
"[B@0/10]:0+b+b", "[B@5/15]:0+b", "[B@0/10]:0+b+b", "[B@5/15]:0+b",
"[D@0/10]:0+d+d", "[D@5/15]:0+d", "[D@0/10]:0+d+d", "[D@5/15]:0+d",
"[B@0/10]:0+b+b+b", "[B@5/15]:0+b+b", "[B@0/10]:0+b+b+b", "[B@5/15]:0+b+b",
"[C@0/10]:0+c+c", "[C@5/15]:0+c" "[C@0/10]:0+c+c", "[C@5/15]:0+c"
); );
proc3.checkAndClearProcessResult( proc3.checkAndClearProcessResult(
"[A@0/10]:0+1+1+1%0+a+a+a", "[A@5/15]:0+1%0+a", "[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", "[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", "[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", "[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" "[C@0/10]:0+3+3%0+c+c", "[C@5/15]:0+3%0+c"
); );
} }

View File

@ -101,7 +101,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce two items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -110,7 +110,7 @@ public class KTableKTableInnerJoinTest {
checkJoinedValues(getter, kv(0, "XX0+Y0"), kv(1, "XX1+Y1")); checkJoinedValues(getter, kv(0, "XX0+Y0"), kv(1, "XX1+Y1"));
// push all items to the other stream. this should produce four items. // 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.process(topic2, expectedKey, "YY" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -120,7 +120,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -140,7 +140,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce two items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -173,7 +173,6 @@ public class KTableKTableInnerJoinTest {
doTestJoin(builder, expectedKeys, processor, joined); doTestJoin(builder, expectedKeys, processor, joined);
} }
@Test @Test
public void testQueryableJoin() { public void testQueryableJoin() {
final StreamsBuilder builder = new StreamsBuilder(); 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. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -243,7 +242,7 @@ public class KTableKTableInnerJoinTest {
proc.checkAndClearProcessResult("0:(XX0+Y0<-null)", "1:(XX1+Y1<-null)"); proc.checkAndClearProcessResult("0:(XX0+Y0<-null)", "1:(XX1+Y1<-null)");
// push all items to the other stream. this should produce four items. // 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.process(topic2, expectedKey, "YY" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -251,7 +250,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -267,7 +266,7 @@ public class KTableKTableInnerJoinTest {
// push all four items to the primary stream. this should produce two items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); 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); return new KeyValue<>(key, value);
} }
private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) { @SafeVarargs
for (KeyValue<Integer, String> kv : expected) { private final void checkJoinedValues(final KTableValueGetter<Integer, String> getter, final KeyValue<Integer, String>... expected) {
String value = getter.get(kv.key); for (final KeyValue<Integer, String> kv : expected) {
final String value = getter.get(kv.key);
if (kv.value == null) { if (kv.value == null) {
assertNull(value); assertNull(value);
} else { } else {

View File

@ -36,7 +36,6 @@ import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashSet; import java.util.HashSet;
@ -53,8 +52,6 @@ public class KTableKTableLeftJoinTest {
final private String topic1 = "topic1"; final private String topic1 = "topic1";
final private String topic2 = "topic2"; 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<Integer> intSerde = Serdes.Integer();
final private Serde<String> stringSerde = Serdes.String(); final private Serde<String> stringSerde = Serdes.String();
@ -64,7 +61,7 @@ public class KTableKTableLeftJoinTest {
private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde); private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde);
@Before @Before
public void setUp() throws IOException { public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test"); stateDir = TestUtils.tempDirectory("kafka-test");
} }
@ -74,24 +71,24 @@ public class KTableKTableLeftJoinTest {
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KTable<Integer, String> table1 = builder.table(topic1, consumed); final KTable<Integer, String> table1 = builder.table(topic1, consumed);
KTable<Integer, String> table2 = builder.table(topic2, consumed); final KTable<Integer, String> table2 = builder.table(topic2, consumed);
KTable<Integer, String> joined = table1.leftJoin(table2, MockValueJoiner.TOSTRING_JOINER); final KTable<Integer, String> joined = table1.leftJoin(table2, MockValueJoiner.TOSTRING_JOINER);
MockProcessorSupplier<Integer, String> processor; final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>(); processor = new MockProcessorSupplier<>();
joined.toStream().process(processor); joined.toStream().process(processor);
Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder); final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size()); assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); 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.setUp(builder, stateDir);
driver.setTime(0L); driver.setTime(0L);
KTableValueGetter<Integer, String> getter = getterSupplier.get(); final KTableValueGetter<Integer, String> getter = getterSupplier.get();
getter.init(driver.context()); getter.init(driver.context());
// push two items to the primary stream. the other table is empty // 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. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); 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")); 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. // 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.process(topic2, expectedKey, "YY" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -136,7 +133,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -154,7 +151,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -205,14 +202,14 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); 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. // 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.process(topic2, expectedKey, "YY" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -220,7 +217,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -236,7 +233,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -249,10 +246,10 @@ public class KTableKTableLeftJoinTest {
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KTable<Integer, String> table1; final KTable<Integer, String> table1;
KTable<Integer, String> table2; final KTable<Integer, String> table2;
KTable<Integer, String> joined; final KTable<Integer, String> joined;
MockProcessorSupplier<Integer, String> proc; final MockProcessorSupplier<Integer, String> proc;
table1 = builder.table(topic1, consumed); table1 = builder.table(topic1, consumed);
table2 = builder.table(topic2, 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. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); 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. // 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.process(topic2, expectedKey, "YY" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -303,7 +300,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -319,7 +316,7 @@ public class KTableKTableLeftJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -344,13 +341,18 @@ public class KTableKTableLeftJoinTest {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
final Consumed<Long, String> consumed = Consumed.with(Serdes.Long(), Serdes.String()); final Consumed<Long, String> consumed = Consumed.with(Serdes.Long(), Serdes.String());
final KTable<Long, String> aggTable = builder.table(agg, consumed) final KTable<Long, String> aggTable = builder
.groupBy(new KeyValueMapper<Long, String, KeyValue<Long, String>>() { .table(agg, consumed)
.groupBy(
new KeyValueMapper<Long, String, KeyValue<Long, String>>() {
@Override @Override
public KeyValue<Long, String> apply(final Long key, final String value) { public KeyValue<Long, String> apply(final Long key, final String value) {
return new KeyValue<>(key, 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> one = builder.table(tableOne, consumed);
final KTable<Long, String> two = builder.table(tableTwo, 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); final KTable<Long, String> eight = six.leftJoin(seven, MockValueJoiner.TOSTRING_JOINER);
aggTable.leftJoin(one, MockValueJoiner.TOSTRING_JOINER) aggTable
.leftJoin(two, MockValueJoiner.TOSTRING_JOINER) .leftJoin(one, MockValueJoiner.TOSTRING_JOINER)
.leftJoin(three, MockValueJoiner.TOSTRING_JOINER) .leftJoin(two, MockValueJoiner.TOSTRING_JOINER)
.leftJoin(four, MockValueJoiner.TOSTRING_JOINER) .leftJoin(three, MockValueJoiner.TOSTRING_JOINER)
.leftJoin(five, MockValueJoiner.TOSTRING_JOINER) .leftJoin(four, MockValueJoiner.TOSTRING_JOINER)
.leftJoin(eight, MockValueJoiner.TOSTRING_JOINER) .leftJoin(five, MockValueJoiner.TOSTRING_JOINER)
.mapValues(mapper); .leftJoin(eight, MockValueJoiner.TOSTRING_JOINER)
.mapValues(mapper);
driver.setUp(builder, stateDir, 250); driver.setUp(builder, stateDir, 250);
final String[] values = {"a", "AA", "BBB", "CCCC", "DD", "EEEEEEEE", "F", "GGGGGGGGGGGGGGG", "HHH", "IIIIIIIIII", final String[] values = {
"J", "KK", "LLLL", "MMMMMMMMMMMMMMMMMMMMMM", "NNNNN", "O", "P", "QQQQQ", "R", "SSSS", "a", "AA", "BBB", "CCCC", "DD", "EEEEEEEE", "F", "GGGGGGGGGGGGGGG", "HHH", "IIIIIIIIII",
"T", "UU", "VVVVVVVVVVVVVVVVVVV"}; "J", "KK", "LLLL", "MMMMMMMMMMMMMMMMMMMMMM", "NNNNN", "O", "P", "QQQQQ", "R", "SSSS",
"T", "UU", "VVVVVVVVVVVVVVVVVVV"
};
final Random random = new Random(); final Random random = new Random();
for (int i = 0; i < 1000; i++) { for (int i = 0; i < 1000; i++) {
for (String input : inputs) { for (final String input : inputs) {
final Long key = (long) random.nextInt(1000); final Long key = (long) random.nextInt(1000);
final String value = values[random.nextInt(values.length)]; final String value = values[random.nextInt(values.length)];
driver.process(input, key, value); 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); return new KeyValue<>(key, value);
} }
private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) { @SafeVarargs
for (KeyValue<Integer, String> kv : expected) { private final void checkJoinedValues(final KTableValueGetter<Integer, String> getter, final KeyValue<Integer, String>... expected) {
String value = getter.get(kv.key); for (final KeyValue<Integer, String> kv : expected) {
final String value = getter.get(kv.key);
if (kv.value == null) { if (kv.value == null) {
assertNull(value); assertNull(value);
} else { } else {

View File

@ -32,7 +32,6 @@ import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashSet; import java.util.HashSet;
@ -47,8 +46,6 @@ public class KTableKTableOuterJoinTest {
final private String topic1 = "topic1"; final private String topic1 = "topic1";
final private String topic2 = "topic2"; 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<Integer> intSerde = Serdes.Integer();
final private Serde<String> stringSerde = Serdes.String(); final private Serde<String> stringSerde = Serdes.String();
@ -58,20 +55,20 @@ public class KTableKTableOuterJoinTest {
private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde); private final Consumed<Integer, String> consumed = Consumed.with(intSerde, stringSerde);
@Before @Before
public void setUp() throws IOException { public void setUp() {
stateDir = TestUtils.tempDirectory("kafka-test"); stateDir = TestUtils.tempDirectory("kafka-test");
} }
@Test @Test
public void testJoin() { public void testJoin() {
StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KTable<Integer, String> table1; final KTable<Integer, String> table1;
KTable<Integer, String> table2; final KTable<Integer, String> table2;
KTable<Integer, String> joined; final KTable<Integer, String> joined;
MockProcessorSupplier<Integer, String> processor; final MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>(); processor = new MockProcessorSupplier<>();
table1 = builder.table(topic1, consumed); table1 = builder.table(topic1, consumed);
@ -79,16 +76,16 @@ public class KTableKTableOuterJoinTest {
joined = table1.outerJoin(table2, MockValueJoiner.TOSTRING_JOINER); joined = table1.outerJoin(table2, MockValueJoiner.TOSTRING_JOINER);
joined.toStream().process(processor); joined.toStream().process(processor);
Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder); final Collection<Set<String>> copartitionGroups = StreamsBuilderTest.getCopartitionedGroups(builder);
assertEquals(1, copartitionGroups.size()); assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); 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.setUp(builder, stateDir);
KTableValueGetter<Integer, String> getter = getterSupplier.get(); final KTableValueGetter<Integer, String> getter = getterSupplier.get();
getter.init(driver.context()); getter.init(driver.context());
// push two items to the primary stream. the other table is empty // 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. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); 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")); 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. // 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.process(topic2, expectedKey, "YY" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -132,7 +129,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -150,7 +147,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -173,10 +170,10 @@ public class KTableKTableOuterJoinTest {
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KTable<Integer, String> table1; final KTable<Integer, String> table1;
KTable<Integer, String> table2; final KTable<Integer, String> table2;
KTable<Integer, String> joined; final KTable<Integer, String> joined;
MockProcessorSupplier<Integer, String> proc; final MockProcessorSupplier<Integer, String> proc;
table1 = builder.table(topic1, consumed); table1 = builder.table(topic1, consumed);
table2 = builder.table(topic2, 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. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); 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. // 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.process(topic2, expectedKey, "YY" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -224,7 +221,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -240,7 +237,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -261,10 +258,10 @@ public class KTableKTableOuterJoinTest {
final int[] expectedKeys = new int[]{0, 1, 2, 3}; final int[] expectedKeys = new int[]{0, 1, 2, 3};
KTable<Integer, String> table1; final KTable<Integer, String> table1;
KTable<Integer, String> table2; final KTable<Integer, String> table2;
KTable<Integer, String> joined; final KTable<Integer, String> joined;
MockProcessorSupplier<Integer, String> proc; final MockProcessorSupplier<Integer, String> proc;
table1 = builder.table(topic1, consumed); table1 = builder.table(topic1, consumed);
table2 = builder.table(topic2, 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. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); 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. // 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.process(topic2, expectedKey, "YY" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -314,7 +311,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "X" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -330,7 +327,7 @@ public class KTableKTableOuterJoinTest {
// push all four items to the primary stream. this should produce four items. // 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.process(topic1, expectedKey, "XX" + expectedKey);
} }
driver.flushState(); driver.flushState();
@ -345,13 +342,14 @@ public class KTableKTableOuterJoinTest {
proc.checkAndClearProcessResult("1:(null<-XX1+null)", "2:(null+YY2<-XX2+YY2)"); 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); return new KeyValue<>(key, value);
} }
private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) { @SafeVarargs
for (KeyValue<Integer, String> kv : expected) { private final void checkJoinedValues(final KTableValueGetter<Integer, String> getter, final KeyValue<Integer, String>... expected) {
String value = getter.get(kv.key); for (final KeyValue<Integer, String> kv : expected) {
final String value = getter.get(kv.key);
if (kv.value == null) { if (kv.value == null) {
assertNull(value); assertNull(value);
} else { } else {

View File

@ -53,11 +53,11 @@ public class KTableSourceTest {
public void testKTable() { public void testKTable() {
final StreamsBuilder builder = new StreamsBuilder(); 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); table1.toStream().process(proc1);
driver.setUp(builder, stateDir); driver.setUp(builder, stateDir);
@ -77,14 +77,14 @@ public class KTableSourceTest {
public void testValueGetter() { public void testValueGetter() {
final StreamsBuilder builder = new StreamsBuilder(); 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); driver.setUp(builder, stateDir);
KTableValueGetter<String, String> getter1 = getterSupplier1.get(); final KTableValueGetter<String, String> getter1 = getterSupplier1.get();
getter1.init(driver.context()); getter1.init(driver.context());
driver.process(topic1, "A", "01"); driver.process(topic1, "A", "01");
@ -121,11 +121,11 @@ public class KTableSourceTest {
public void testNotSendingOldValue() { public void testNotSendingOldValue() {
final StreamsBuilder builder = new StreamsBuilder(); 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); builder.build().addProcessor("proc1", proc1, table1.name);
@ -159,15 +159,15 @@ public class KTableSourceTest {
public void testSendingOldValue() { public void testSendingOldValue() {
final StreamsBuilder builder = new StreamsBuilder(); 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(); table1.enableSendingOldValues();
assertTrue(table1.sendingOldValueEnabled()); assertTrue(table1.sendingOldValueEnabled());
MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>(); final MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
builder.build().addProcessor("proc1", proc1, table1.name); builder.build().addProcessor("proc1", proc1, table1.name);

View File

@ -56,12 +56,14 @@ public class GlobalStateTaskTest {
private final String topic2 = "t2"; private final String topic2 = "t2";
private final TopicPartition t1 = new TopicPartition(topic1, 1); private final TopicPartition t1 = new TopicPartition(topic1, 1);
private final TopicPartition t2 = new TopicPartition(topic2, 1); private final TopicPartition t2 = new TopicPartition(topic2, 1);
private final MockSourceNode sourceOne = new MockSourceNode<>(new String[]{topic1}, private final MockSourceNode sourceOne = new MockSourceNode<>(
new StringDeserializer(), new String[]{topic1},
new StringDeserializer());; new StringDeserializer(),
private final MockSourceNode sourceTwo = new MockSourceNode<>(new String[]{topic2}, new StringDeserializer());
new IntegerDeserializer(), private final MockSourceNode sourceTwo = new MockSourceNode<>(
new IntegerDeserializer()); new String[]{topic2},
new IntegerDeserializer(),
new IntegerDeserializer());
private final MockProcessorNode processorOne = new MockProcessorNode<>(-1); private final MockProcessorNode processorOne = new MockProcessorNode<>(-1);
private final MockProcessorNode processorTwo = new MockProcessorNode<>(-1); private final MockProcessorNode processorTwo = new MockProcessorNode<>(-1);
@ -81,10 +83,11 @@ public class GlobalStateTaskTest {
final Map<String, String> storeToTopic = new HashMap<>(); final Map<String, String> storeToTopic = new HashMap<>();
storeToTopic.put("t1-store", topic1); storeToTopic.put("t1-store", topic1);
storeToTopic.put("t2-store", topic2); storeToTopic.put("t2-store", topic2);
topology = ProcessorTopology.with(Utils.mkList(sourceOne, sourceTwo, processorOne, processorTwo), topology = ProcessorTopology.with(
sourceByTopics, Utils.mkList(sourceOne, sourceTwo, processorOne, processorTwo),
Collections.<StateStore>emptyList(), sourceByTopics,
storeToTopic); Collections.<StateStore>emptyList(),
storeToTopic);
offsets.put(t1, 50L); offsets.put(t1, 50L);
offsets.put(t2, 100L); offsets.put(t2, 100L);
@ -134,17 +137,18 @@ public class GlobalStateTaskTest {
private void maybeDeserialize(final GlobalStateUpdateTask globalStateTask, private void maybeDeserialize(final GlobalStateUpdateTask globalStateTask,
final byte[] key, final byte[] key,
final byte[] recordValue, final byte[] recordValue,
boolean failExpected) { final boolean failExpected) {
final ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(topic2, 1, 1, final ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
0L, TimestampType.CREATE_TIME, 0L, 0, 0, topic2, 1, 1, 0L, TimestampType.CREATE_TIME,
key, recordValue); 0L, 0, 0, key, recordValue
);
globalStateTask.initialize(); globalStateTask.initialize();
try { try {
globalStateTask.update(record); globalStateTask.update(record);
if (failExpected) { if (failExpected) {
fail("Should have failed to deserialize."); fail("Should have failed to deserialize.");
} }
} catch (StreamsException e) { } catch (final StreamsException e) {
if (!failExpected) { if (!failExpected) {
fail("Shouldn't have failed to deserialize."); fail("Shouldn't have failed to deserialize.");
} }
@ -153,7 +157,7 @@ public class GlobalStateTaskTest {
@Test @Test
public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() throws Exception { public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() {
final byte[] key = new LongSerializer().serialize(topic2, 1L); final byte[] key = new LongSerializer().serialize(topic2, 1L);
final byte[] recordValue = new IntegerSerializer().serialize(topic2, 10); final byte[] recordValue = new IntegerSerializer().serialize(topic2, 10);
maybeDeserialize(globalStateTask, key, recordValue, true); maybeDeserialize(globalStateTask, key, recordValue, true);
@ -161,14 +165,14 @@ public class GlobalStateTaskTest {
@Test @Test
public void shouldThrowStreamsExceptionWhenValueDeserializationFails() throws Exception { public void shouldThrowStreamsExceptionWhenValueDeserializationFails() {
final byte[] key = new IntegerSerializer().serialize(topic2, 1); final byte[] key = new IntegerSerializer().serialize(topic2, 1);
final byte[] recordValue = new LongSerializer().serialize(topic2, 10L); final byte[] recordValue = new LongSerializer().serialize(topic2, 10L);
maybeDeserialize(globalStateTask, key, recordValue, true); maybeDeserialize(globalStateTask, key, recordValue, true);
} }
@Test @Test
public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() throws Exception { public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() {
final GlobalStateUpdateTask globalStateTask2 = new GlobalStateUpdateTask( final GlobalStateUpdateTask globalStateTask2 = new GlobalStateUpdateTask(
topology, topology,
context, context,
@ -182,7 +186,7 @@ public class GlobalStateTaskTest {
} }
@Test @Test
public void shouldNotThrowStreamsExceptionWhenValueDeserializationFails() throws Exception { public void shouldNotThrowStreamsExceptionWhenValueDeserializationFails() {
final GlobalStateUpdateTask globalStateTask2 = new GlobalStateUpdateTask( final GlobalStateUpdateTask globalStateTask2 = new GlobalStateUpdateTask(
topology, topology,
context, context,

View File

@ -16,12 +16,13 @@
*/ */
package org.apache.kafka.streams.processor.internals; package org.apache.kafka.streams.processor.internals;
import java.util.Collections;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import java.util.Collections;
public class MockStreamsMetrics extends StreamsMetricsImpl { public class MockStreamsMetrics extends StreamsMetricsImpl {
public MockStreamsMetrics(Metrics metrics) { public MockStreamsMetrics(final Metrics metrics) {
super(metrics, "mock-stream-metrics", super(metrics, "mock-stream-metrics",
Collections.<String, String>emptyMap()); Collections.<String, String>emptyMap());
} }

View File

@ -30,9 +30,10 @@ import org.apache.kafka.test.MockTimestampExtractor;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap;
import java.util.List; 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; import static org.junit.Assert.assertEquals;
public class PartitionGroupTest { public class PartitionGroupTest {
@ -61,12 +62,7 @@ public class PartitionGroupTest {
private final byte[] recordValue = intSerializer.serialize(null, 10); private final byte[] recordValue = intSerializer.serialize(null, 10);
private final byte[] recordKey = intSerializer.serialize(null, 1); private final byte[] recordKey = intSerializer.serialize(null, 1);
private final PartitionGroup group = new PartitionGroup(new HashMap<TopicPartition, RecordQueue>() { private final PartitionGroup group = new PartitionGroup(mkMap(mkEntry(partition1, queue1), mkEntry(partition2, queue2)));
{
put(partition1, queue1);
put(partition2, queue2);
}
});
@Test @Test
public void testTimeTracking() { public void testTimeTracking() {

View File

@ -28,22 +28,22 @@ import org.apache.kafka.test.InternalMockProcessorContext;
import org.junit.Test; import org.junit.Test;
import java.util.Collections; import java.util.Collections;
import java.util.Map;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.Map;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
public class ProcessorNodeTest { public class ProcessorNodeTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test (expected = StreamsException.class) @Test(expected = StreamsException.class)
public void shouldThrowStreamsExceptionIfExceptionCaughtDuringInit() { public void shouldThrowStreamsExceptionIfExceptionCaughtDuringInit() {
final ProcessorNode node = new ProcessorNode("name", new ExceptionalProcessor(), Collections.emptySet()); final ProcessorNode node = new ProcessorNode("name", new ExceptionalProcessor(), Collections.emptySet());
node.init(null); node.init(null);
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test (expected = StreamsException.class) @Test(expected = StreamsException.class)
public void shouldThrowStreamsExceptionIfExceptionCaughtDuringClose() { public void shouldThrowStreamsExceptionIfExceptionCaughtDuringClose() {
final ProcessorNode node = new ProcessorNode("name", new ExceptionalProcessor(), Collections.emptySet()); final ProcessorNode node = new ProcessorNode("name", new ExceptionalProcessor(), Collections.emptySet());
node.close(); node.close();
@ -71,7 +71,7 @@ public class ProcessorNodeTest {
} }
} }
private static class NoOpProcessor implements Processor { private static class NoOpProcessor implements Processor<Object, Object> {
@Override @Override
public void init(final ProcessorContext context) { 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 String opName,
final Map<String, String> metricTags) { final Map<String, String> metricTags) {
assertNotNull(metrics.metrics().get(metrics.metricName(opName + "-latency-avg", groupName, 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, 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, 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 StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
final Metrics metrics = new Metrics(); final Metrics metrics = new Metrics();
final InternalMockProcessorContext context = new InternalMockProcessorContext(anyStateSerde, new RecordCollectorImpl(null, null, new LogContext("processnode-test "), new DefaultProductionExceptionHandler()), metrics); final InternalMockProcessorContext context = new InternalMockProcessorContext(
final ProcessorNode node = new ProcessorNode("name", new NoOpProcessor(), Collections.emptySet()); 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); node.init(context);
String[] latencyOperations = {"process", "punctuate", "create", "destroy"}; final String[] latencyOperations = {"process", "punctuate", "create", "destroy"};
String throughputOperation = "forward"; final String throughputOperation = "forward";
String groupName = "stream-processor-node-metrics"; final String groupName = "stream-processor-node-metrics";
final Map<String, String> metricTags = new LinkedHashMap<>(); final Map<String, String> metricTags = new LinkedHashMap<>();
metricTags.put("processor-node-id", node.name()); metricTags.put("processor-node-id", node.name());
metricTags.put("task-id", context.taskId().toString()); metricTags.put("task-id", context.taskId().toString());
for (String operation : latencyOperations) { for (final String operation : latencyOperations) {
assertNotNull(metrics.getSensor(operation)); assertNotNull(metrics.getSensor(operation));
} }
assertNotNull(metrics.getSensor(throughputOperation)); assertNotNull(metrics.getSensor(throughputOperation));
for (String opName : latencyOperations) { for (final String opName : latencyOperations) {
testSpecificMetrics(metrics, groupName, opName, metricTags); testSpecificMetrics(metrics, groupName, opName, metricTags);
} }
assertNotNull(metrics.metrics().get(metrics.metricName(throughputOperation + "-rate", groupName, 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" // test "all"
metricTags.put("processor-node-id", "all"); metricTags.put("processor-node-id", "all");
for (String opName : latencyOperations) { for (final String opName : latencyOperations) {
testSpecificMetrics(metrics, groupName, opName, metricTags); testSpecificMetrics(metrics, groupName, opName, metricTags);
} }
assertNotNull(metrics.metrics().get(metrics.metricName(throughputOperation + "-rate", groupName, 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(); context.close();

View File

@ -49,13 +49,13 @@ public class RecordCollectorTest {
private final LogContext logContext = new LogContext("test "); private final LogContext logContext = new LogContext("test ");
private final List<PartitionInfo> infos = Arrays.asList( private final List<PartitionInfo> infos = Arrays.asList(
new PartitionInfo("topic1", 0, 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", 1, Node.noNode(), new Node[0], new Node[0]),
new PartitionInfo("topic1", 2, 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, 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(); private final ByteArraySerializer byteArraySerializer = new ByteArraySerializer();
@ -72,8 +72,10 @@ public class RecordCollectorTest {
public void testSpecificPartition() { public void testSpecificPartition() {
final RecordCollectorImpl collector = new RecordCollectorImpl( final RecordCollectorImpl collector = new RecordCollectorImpl(
new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer), new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer),
"RecordCollectorTest-TestSpecificPartition", new LogContext("RecordCollectorTest-TestSpecificPartition "), new DefaultProductionExceptionHandler()); "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);
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() { public void testStreamPartitioner() {
final RecordCollectorImpl collector = new RecordCollectorImpl( final RecordCollectorImpl collector = new RecordCollectorImpl(
new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer), new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer),
"RecordCollectorTest-TestStreamPartitioner", new LogContext("RecordCollectorTest-TestStreamPartitioner "), new DefaultProductionExceptionHandler()); "RecordCollectorTest-TestStreamPartitioner",
new LogContext("RecordCollectorTest-TestStreamPartitioner "),
new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner); collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
collector.send("topic1", "9", "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) @Test(expected = StreamsException.class)
public void shouldThrowStreamsExceptionOnAnyExceptionButProducerFencedException() { public void shouldThrowStreamsExceptionOnAnyExceptionButProducerFencedException() {
final RecordCollector collector = new RecordCollectorImpl( final RecordCollector collector = new RecordCollectorImpl(
new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
@Override @Override
public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) { public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
throw new KafkaException(); throw new KafkaException();
} }
}, },
"test", "test",
logContext, logContext,
new DefaultProductionExceptionHandler()); new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner); collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
} }
@ -147,16 +151,16 @@ public class RecordCollectorTest {
@Test @Test
public void shouldThrowStreamsExceptionOnSubsequentCallIfASendFailsWithDefaultExceptionHandler() { public void shouldThrowStreamsExceptionOnSubsequentCallIfASendFailsWithDefaultExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl( final RecordCollector collector = new RecordCollectorImpl(
new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
@Override @Override
public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) { public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
callback.onCompletion(null, new Exception()); callback.onCompletion(null, new Exception());
return null; return null;
} }
}, },
"test", "test",
logContext, logContext,
new DefaultProductionExceptionHandler()); new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner); collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
try { try {
@ -169,16 +173,16 @@ public class RecordCollectorTest {
@Test @Test
public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContinueExceptionHandler() { public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContinueExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl( final RecordCollector collector = new RecordCollectorImpl(
new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
@Override @Override
public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) { public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
callback.onCompletion(null, new Exception()); callback.onCompletion(null, new Exception());
return null; return null;
} }
}, },
"test", "test",
logContext, logContext,
new AlwaysContinueProductionExceptionHandler()); new AlwaysContinueProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner); collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
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 @Test
public void shouldThrowStreamsExceptionOnFlushIfASendFailedWithDefaultExceptionHandler() { public void shouldThrowStreamsExceptionOnFlushIfASendFailedWithDefaultExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl( final RecordCollector collector = new RecordCollectorImpl(
new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
@Override @Override
public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) { public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
callback.onCompletion(null, new Exception()); callback.onCompletion(null, new Exception());
return null; return null;
} }
}, },
"test", "test",
logContext, logContext,
new DefaultProductionExceptionHandler()); new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner); collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
try { try {
@ -210,16 +214,16 @@ public class RecordCollectorTest {
@Test @Test
public void shouldNotThrowStreamsExceptionOnFlushIfASendFailedWithContinueExceptionHandler() { public void shouldNotThrowStreamsExceptionOnFlushIfASendFailedWithContinueExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl( final RecordCollector collector = new RecordCollectorImpl(
new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
@Override @Override
public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) { public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
callback.onCompletion(null, new Exception()); callback.onCompletion(null, new Exception());
return null; return null;
} }
}, },
"test", "test",
logContext, logContext,
new AlwaysContinueProductionExceptionHandler()); new AlwaysContinueProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner); collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
collector.flush(); collector.flush();
@ -229,16 +233,16 @@ public class RecordCollectorTest {
@Test @Test
public void shouldThrowStreamsExceptionOnCloseIfASendFailedWithDefaultExceptionHandler() { public void shouldThrowStreamsExceptionOnCloseIfASendFailedWithDefaultExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl( final RecordCollector collector = new RecordCollectorImpl(
new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
@Override @Override
public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) { public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
callback.onCompletion(null, new Exception()); callback.onCompletion(null, new Exception());
return null; return null;
} }
}, },
"test", "test",
logContext, logContext,
new DefaultProductionExceptionHandler()); new DefaultProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner); collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
try { try {
@ -251,16 +255,16 @@ public class RecordCollectorTest {
@Test @Test
public void shouldNotThrowStreamsExceptionOnCloseIfASendFailedWithContinueExceptionHandler() { public void shouldNotThrowStreamsExceptionOnCloseIfASendFailedWithContinueExceptionHandler() {
final RecordCollector collector = new RecordCollectorImpl( final RecordCollector collector = new RecordCollectorImpl(
new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
@Override @Override
public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) { public synchronized Future<RecordMetadata> send(final ProducerRecord record, final Callback callback) {
callback.onCompletion(null, new Exception()); callback.onCompletion(null, new Exception());
return null; return null;
} }
}, },
"test", "test",
logContext, logContext,
new AlwaysContinueProductionExceptionHandler()); new AlwaysContinueProductionExceptionHandler());
collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner); collector.send("topic1", "3", "0", null, stringSerializer, stringSerializer, streamPartitioner);
collector.close(); collector.close();

View File

@ -56,22 +56,17 @@ public class RecordDeserializerTest {
assertEquals(TimestampType.CREATE_TIME, record.timestampType()); 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 keyThrowsException;
private final boolean valueThrowsException; private final boolean valueThrowsException;
private final Object key; private final Object key;
private final Object value; private final Object value;
TheSourceNode(final boolean keyThrowsException, final boolean valueThrowsException) {
this(keyThrowsException, valueThrowsException, null, null);
}
@SuppressWarnings("unchecked")
TheSourceNode(final boolean keyThrowsException, TheSourceNode(final boolean keyThrowsException,
final boolean valueThrowsException, final boolean valueThrowsException,
final Object key, final Object key,
final Object value) { final Object value) {
super("", Collections.EMPTY_LIST, null, null); super("", Collections.<String>emptyList(), null, null);
this.keyThrowsException = keyThrowsException; this.keyThrowsException = keyThrowsException;
this.valueThrowsException = valueThrowsException; this.valueThrowsException = valueThrowsException;
this.key = key; this.key = key;

View File

@ -55,7 +55,7 @@ public class RecordQueueTest {
private final String[] topics = {"topic"}; private final String[] topics = {"topic"};
final InternalMockProcessorContext context = new InternalMockProcessorContext(StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class), 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 MockSourceNode mockSourceNodeWithMetrics = new MockSourceNode<>(topics, intDeserializer, intDeserializer);
private final RecordQueue queue = new RecordQueue( private final RecordQueue queue = new RecordQueue(
new TopicPartition(topics[0], 1), new TopicPartition(topics[0], 1),
@ -93,7 +93,7 @@ public class RecordQueueTest {
assertEquals(TimestampTracker.NOT_KNOWN, queue.timestamp()); assertEquals(TimestampTracker.NOT_KNOWN, queue.timestamp());
// add three 3 out-of-order records with timestamp 2, 1, 3 // 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, 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, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 3, 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 // add three 3 out-of-order records with timestamp 4, 1, 2
// now with 3, 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, 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, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 2, 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()); assertEquals(0, queue.timeTracker().size());
// add three more records with 4, 5, 6 // 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, 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, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
new ConsumerRecord<>("topic", 1, 6, 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() { public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() {
final byte[] key = Serdes.Long().serializer().serialize("foo", 1L); final byte[] key = Serdes.Long().serializer().serialize("foo", 1L);
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList( 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); queue.addRawRecords(records);
} }
@ -194,24 +194,23 @@ public class RecordQueueTest {
public void shouldThrowStreamsExceptionWhenValueDeserializationFails() { public void shouldThrowStreamsExceptionWhenValueDeserializationFails() {
final byte[] value = Serdes.Long().serializer().serialize("foo", 1L); final byte[] value = Serdes.Long().serializer().serialize("foo", 1L);
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList( 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); queue.addRawRecords(records);
} }
@Test @Test
public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() throws Exception { public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() {
final byte[] key = Serdes.Long().serializer().serialize("foo", 1L); final byte[] key = Serdes.Long().serializer().serialize("foo", 1L);
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList( 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));
final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
queueThatSkipsDeserializeErrors.addRawRecords(records); queueThatSkipsDeserializeErrors.addRawRecords(records);
assertEquals(0, queueThatSkipsDeserializeErrors.size()); assertEquals(0, queueThatSkipsDeserializeErrors.size());
} }
@Test @Test
public void shouldNotThrowStreamsExceptionWhenValueDeserializationFailsWithSkipHandler() throws Exception { public void shouldNotThrowStreamsExceptionWhenValueDeserializationFailsWithSkipHandler() {
final byte[] value = Serdes.Long().serializer().serialize("foo", 1L); final byte[] value = Serdes.Long().serializer().serialize("foo", 1L);
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList( 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));
@ -224,28 +223,30 @@ public class RecordQueueTest {
@Test(expected = StreamsException.class) @Test(expected = StreamsException.class)
public void shouldThrowOnNegativeTimestamp() { public void shouldThrowOnNegativeTimestamp() {
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList( final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
final RecordQueue queue = new RecordQueue(new TopicPartition(topics[0], 1), final RecordQueue queue = new RecordQueue(
new MockSourceNode<>(topics, intDeserializer, intDeserializer), new TopicPartition(topics[0], 1),
new FailOnInvalidTimestamp(), new MockSourceNode<>(topics, intDeserializer, intDeserializer),
new LogAndContinueExceptionHandler(), new FailOnInvalidTimestamp(),
null, new LogAndContinueExceptionHandler(),
new LogContext()); null,
new LogContext());
queue.addRawRecords(records); queue.addRawRecords(records);
} }
@Test @Test
public void shouldDropOnNegativeTimestamp() { public void shouldDropOnNegativeTimestamp() {
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList( final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));
final RecordQueue queue = new RecordQueue(new TopicPartition(topics[0], 1), final RecordQueue queue = new RecordQueue(
new MockSourceNode<>(topics, intDeserializer, intDeserializer), new TopicPartition(topics[0], 1),
new LogAndSkipOnInvalidTimestamp(), new MockSourceNode<>(topics, intDeserializer, intDeserializer),
new LogAndContinueExceptionHandler(), new LogAndSkipOnInvalidTimestamp(),
null, new LogAndContinueExceptionHandler(),
new LogContext()); null,
new LogContext());
queue.addRawRecords(records); queue.addRawRecords(records);
assertEquals(0, queue.size()); assertEquals(0, queue.size());

View File

@ -35,10 +35,17 @@ import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
public class SinkNodeTest { public class SinkNodeTest {
private final Serializer anySerializer = Serdes.Bytes().serializer(); private final Serializer<byte[]> anySerializer = Serdes.ByteArray().serializer();
private final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class); private final StateSerdes<Bytes, Bytes> anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
private final InternalMockProcessorContext context = new InternalMockProcessorContext(anyStateSerde, private final InternalMockProcessorContext context = new InternalMockProcessorContext(
new RecordCollectorImpl(new MockProducer<byte[], byte[]>(true, anySerializer, anySerializer), null, new LogContext("sinknode-test "), new DefaultProductionExceptionHandler())); 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); private final SinkNode sink = new SinkNode<>("anyNodeName", "any-output-topic", anySerializer, anySerializer, null);
@Before @Before

View File

@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition; 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.metrics.Metrics;
import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.ByteArraySerializer;
@ -57,13 +58,14 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Map; import java.util.Map;
import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean; 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.equalTo;
import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.MatcherAssert.assertThat; 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> source2 = new MockSourceNode<>(new String[]{topic2}, intDeserializer, intDeserializer);
private final MockSourceNode<Integer, Integer> source3 = new MockSourceNode<Integer, Integer>(new String[]{topic2}, intDeserializer, intDeserializer) { private final MockSourceNode<Integer, Integer> source3 = new MockSourceNode<Integer, Integer>(new String[]{topic2}, intDeserializer, intDeserializer) {
@Override @Override
public void process(Integer key, Integer value) { public void process(final Integer key, final Integer value) {
throw new RuntimeException("KABOOM!"); throw new RuntimeException("KABOOM!");
} }
@ -106,13 +108,8 @@ public class StreamTaskTest {
private final Long offset = 543L; private final Long offset = 543L;
private final ProcessorTopology topology = ProcessorTopology.withSources( private final ProcessorTopology topology = ProcessorTopology.withSources(
Utils.<ProcessorNode>mkList(source1, source2, processorStreamTime, processorSystemTime), Utils.<ProcessorNode>mkList(source1, source2, processorStreamTime, processorSystemTime),
new HashMap<String, SourceNode>() { mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source2))
{
put(topic1, source1);
put(topic2, source2);
}
}
); );
private final MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); 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 StreamsMetrics streamsMetrics = new MockStreamsMetrics(metrics);
private final TaskId taskId00 = new TaskId(0, 0); private final TaskId taskId00 = new TaskId(0, 0);
private final MockTime time = new MockTime(); private final MockTime time = new MockTime();
private File baseDir = TestUtils.tempDirectory(); private final File baseDir = TestUtils.tempDirectory();
private StateDirectory stateDirectory; private StateDirectory stateDirectory;
private StreamsConfig config;
private StreamsConfig eosConfig;
private StreamTask task; private StreamTask task;
private long punctuatedAt; private long punctuatedAt;
private Punctuator punctuator = new Punctuator() { private final Punctuator punctuator = new Punctuator() {
@Override @Override
public void punctuate(long timestamp) { public void punctuate(final long timestamp) {
punctuatedAt = timestamp; punctuatedAt = timestamp;
} }
}; };
private StreamsConfig createConfig(final boolean enableEoS) throws IOException { private StreamsConfig createConfig(final boolean enableEoS) {
return new StreamsConfig(new Properties() { final String canonicalPath;
{ try {
setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "stream-task-test"); canonicalPath = baseDir.getCanonicalPath();
setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); } catch (final IOException e) {
setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); throw new RuntimeException(e);
setProperty(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); }
setProperty(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); return new StreamsConfig(mkProperties(mkMap(
if (enableEoS) { mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "stream-task-test"),
setProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); 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 @Before
public void setup() throws IOException { public void setup() {
consumer.assign(Arrays.asList(partition1, partition2)); consumer.assign(Arrays.asList(partition1, partition2));
config = createConfig(false); stateDirectory = new StateDirectory(createConfig(false), new MockTime());
eosConfig = createConfig(true);
stateDirectory = new StateDirectory(config, new MockTime());
} }
@After @After
@ -174,7 +169,7 @@ public class StreamTaskTest {
if (task != null) { if (task != null) {
try { try {
task.close(true, false); task.close(true, false);
} catch (Exception e) { } catch (final Exception e) {
// swallow // swallow
} }
} }
@ -186,18 +181,18 @@ public class StreamTaskTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testProcessOrder() { public void testProcessOrder() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.addRecords(partition1, records( task.addRecords(partition1, Arrays.asList(
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 10),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 20),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition1, 30)
)); ));
task.addRecords(partition2, records( task.addRecords(partition2, Arrays.asList(
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 25),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 35),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition2, 45)
)); ));
assertTrue(task.process()); 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 @Test
public void testMetrics() { public void testMetrics() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
final String name = task.id().toString(); assertNotNull(metrics.getSensor("commit"));
final Map<String, String> metricTags = new LinkedHashMap<>(); assertNotNull(getMetric("%s-latency-avg", "The average latency of %s operation.", task.id().toString()));
metricTags.put("task-id", name); assertNotNull(getMetric("%s-latency-max", "The max latency of %s operation.", task.id().toString()));
final String operation = "commit"; 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)); private KafkaMetric getMetric(final String nameFormat, final String descriptionFormat, final String taskId) {
testSpecificMetrics(operation, groupName, metricTags); return metrics.metrics().get(metrics.metricName(
metricTags.put("task-id", "all"); String.format(nameFormat, "commit"),
testSpecificMetrics(operation, groupName, metricTags); "stream-task-metrics",
String.format(descriptionFormat, "commit"),
mkMap(mkEntry("task-id", taskId))
));
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testPauseResume() { public void testPauseResume() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.addRecords(partition1, records( task.addRecords(partition1, Arrays.asList(
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 10),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition1, 20)
)); ));
task.addRecords(partition2, records( task.addRecords(partition2, Arrays.asList(
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 35),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 45),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 55),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition2, 65)
)); ));
assertTrue(task.process()); assertTrue(task.process());
@ -283,10 +274,10 @@ public class StreamTaskTest {
assertEquals(1, consumer.paused().size()); assertEquals(1, consumer.paused().size());
assertTrue(consumer.paused().contains(partition2)); assertTrue(consumer.paused().contains(partition2));
task.addRecords(partition1, records( task.addRecords(partition1, Arrays.asList(
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 30),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 40),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition1, 50)
)); ));
assertEquals(2, consumer.paused().size()); assertEquals(2, consumer.paused().size());
@ -317,23 +308,23 @@ public class StreamTaskTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testMaybePunctuateStreamTime() { public void testMaybePunctuateStreamTime() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.addRecords(partition1, records( task.addRecords(partition1, Arrays.asList(
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 0),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 20),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 32, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 32),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 40),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 60, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition1, 60)
)); ));
task.addRecords(partition2, records( task.addRecords(partition2, Arrays.asList(
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 25),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 35),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 45),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 61, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition2, 61)
)); ));
assertTrue(task.maybePunctuateStreamTime()); assertTrue(task.maybePunctuateStreamTime());
@ -408,22 +399,22 @@ public class StreamTaskTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void shouldPunctuateOnceStreamTimeAfterGap() { public void shouldPunctuateOnceStreamTimeAfterGap() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.addRecords(partition1, records( task.addRecords(partition1, Arrays.asList(
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 20),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 142, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 142),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 155, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 155),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 160, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition1, 160)
)); ));
task.addRecords(partition2, records( task.addRecords(partition2, Arrays.asList(
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 25),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 145, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 145),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 159, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 159),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 161, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition2, 161)
)); ));
assertTrue(task.maybePunctuateStreamTime()); // punctuate at 20 assertTrue(task.maybePunctuateStreamTime()); // punctuate at 20
@ -494,20 +485,20 @@ public class StreamTaskTest {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testCancelPunctuateStreamTime() { public void testCancelPunctuateStreamTime() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.addRecords(partition1, records( task.addRecords(partition1, Arrays.asList(
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 20),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition1, 30),
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition1, 40)
)); ));
task.addRecords(partition2, records( task.addRecords(partition2, Arrays.asList(
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 25),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), getConsumerRecord(partition2, 35),
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) getConsumerRecord(partition2, 45)
)); ));
assertTrue(task.maybePunctuateStreamTime()); assertTrue(task.maybePunctuateStreamTime());
@ -527,10 +518,10 @@ public class StreamTaskTest {
@Test @Test
public void shouldPunctuateSystemTimeWhenIntervalElapsed() { public void shouldPunctuateSystemTimeWhenIntervalElapsed() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
long now = time.milliseconds(); final long now = time.milliseconds();
time.sleep(10); time.sleep(10);
assertTrue(task.maybePunctuateSystemTime()); assertTrue(task.maybePunctuateSystemTime());
time.sleep(10); time.sleep(10);
@ -547,7 +538,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotPunctuateSystemTimeWhenIntervalNotElapsed() { public void shouldNotPunctuateSystemTimeWhenIntervalNotElapsed() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
assertFalse(task.maybePunctuateSystemTime()); assertFalse(task.maybePunctuateSystemTime());
@ -558,10 +549,10 @@ public class StreamTaskTest {
@Test @Test
public void shouldPunctuateOnceSystemTimeAfterGap() { public void shouldPunctuateOnceSystemTimeAfterGap() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
long now = time.milliseconds(); final long now = time.milliseconds();
time.sleep(100); time.sleep(100);
assertTrue(task.maybePunctuateSystemTime()); assertTrue(task.maybePunctuateSystemTime());
assertFalse(task.maybePunctuateSystemTime()); assertFalse(task.maybePunctuateSystemTime());
@ -584,10 +575,10 @@ public class StreamTaskTest {
@Test @Test
public void testCancelPunctuateSystemTime() { public void testCancelPunctuateSystemTime() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
long now = time.milliseconds(); final long now = time.milliseconds();
time.sleep(10); time.sleep(10);
assertTrue(task.maybePunctuateSystemTime()); assertTrue(task.maybePunctuateSystemTime());
processorSystemTime.supplier.scheduleCancellable.cancel(); processorSystemTime.supplier.scheduleCancellable.cancel();
@ -601,8 +592,7 @@ public class StreamTaskTest {
task = createTaskThatThrowsException(); task = createTaskThatThrowsException();
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.addRecords(partition2, Collections.singletonList( task.addRecords(partition2, singletonList(getConsumerRecord(partition2, 0)));
new ConsumerRecord<>(partition2.topic(), partition2.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
try { try {
task.process(); task.process();
@ -614,14 +604,14 @@ public class StreamTaskTest {
@Test @Test
public void shouldWrapKafkaExceptionsWithStreamsExceptionAndAddContextWhenPunctuatingStreamTime() { public void shouldWrapKafkaExceptionsWithStreamsExceptionAndAddContextWhenPunctuatingStreamTime() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
try { try {
task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, new Punctuator() { task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, new Punctuator() {
@Override @Override
public void punctuate(long timestamp) { public void punctuate(final long timestamp) {
throw new KafkaException("KABOOM!"); throw new KafkaException("KABOOM!");
} }
}); });
@ -635,14 +625,14 @@ public class StreamTaskTest {
@Test @Test
public void shouldWrapKafkaExceptionsWithStreamsExceptionAndAddContextWhenPunctuatingWallClockTimeTime() { public void shouldWrapKafkaExceptionsWithStreamsExceptionAndAddContextWhenPunctuatingWallClockTimeTime() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
try { try {
task.punctuate(processorSystemTime, 1, PunctuationType.WALL_CLOCK_TIME, new Punctuator() { task.punctuate(processorSystemTime, 1, PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
@Override @Override
public void punctuate(long timestamp) { public void punctuate(final long timestamp) {
throw new KafkaException("KABOOM!"); throw new KafkaException("KABOOM!");
} }
}); });
@ -659,7 +649,7 @@ public class StreamTaskTest {
final AtomicBoolean flushed = new AtomicBoolean(false); final AtomicBoolean flushed = new AtomicBoolean(false);
final StreamsMetrics streamsMetrics = new MockStreamsMetrics(new Metrics()); final StreamsMetrics streamsMetrics = new MockStreamsMetrics(new Metrics());
final StreamTask streamTask = new StreamTask(taskId00, partitions, topology, consumer, 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 @Override
RecordCollector createRecordCollector(final LogContext logContext, RecordCollector createRecordCollector(final LogContext logContext,
@ -678,31 +668,34 @@ public class StreamTaskTest {
@Test @Test
public void shouldCheckpointOffsetsOnCommit() throws IOException { public void shouldCheckpointOffsetsOnCommit() throws IOException {
task = createStatefulTask(false, true); task = createStatefulTask(createConfig(false), true);
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.commit(); task.commit();
final OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId00), final OffsetCheckpoint checkpoint = new OffsetCheckpoint(
ProcessorStateManager.CHECKPOINT_FILE_NAME)); new File(stateDirectory.directoryForTask(taskId00), ProcessorStateManager.CHECKPOINT_FILE_NAME)
);
assertThat(checkpoint.read(), equalTo(Collections.singletonMap(changelogPartition, offset))); assertThat(checkpoint.read(), equalTo(Collections.singletonMap(changelogPartition, offset)));
} }
@Test @Test
public void shouldNotCheckpointOffsetsOnCommitIfEosIsEnabled() { public void shouldNotCheckpointOffsetsOnCommitIfEosIsEnabled() {
task = createStatefulTask(true, true); task = createStatefulTask(createConfig(true), true);
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.commit(); task.commit();
final File checkpointFile = new File(stateDirectory.directoryForTask(taskId00), final File checkpointFile = new File(
ProcessorStateManager.CHECKPOINT_FILE_NAME); stateDirectory.directoryForTask(taskId00),
ProcessorStateManager.CHECKPOINT_FILE_NAME
);
assertFalse(checkpointFile.exists()); assertFalse(checkpointFile.exists());
} }
@Test @Test
public void shouldThrowIllegalStateExceptionIfCurrentNodeIsNotNullWhenPunctuateCalled() { public void shouldThrowIllegalStateExceptionIfCurrentNodeIsNotNullWhenPunctuateCalled() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.processorContext.setCurrentNode(processorStreamTime); task.processorContext.setCurrentNode(processorStreamTime);
@ -716,7 +709,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldCallPunctuateOnPassedInProcessorNode() { public void shouldCallPunctuateOnPassedInProcessorNode() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.punctuate(processorStreamTime, 5, PunctuationType.STREAM_TIME, punctuator); task.punctuate(processorStreamTime, 5, PunctuationType.STREAM_TIME, punctuator);
@ -726,8 +719,8 @@ public class StreamTaskTest {
} }
@Test @Test
public void shouldSetProcessorNodeOnContextBackToNullAfterSuccesfullPunctuate() { public void shouldSetProcessorNodeOnContextBackToNullAfterSuccessfulPunctuate() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.punctuate(processorStreamTime, 5, PunctuationType.STREAM_TIME, punctuator); task.punctuate(processorStreamTime, 5, PunctuationType.STREAM_TIME, punctuator);
@ -736,10 +729,10 @@ public class StreamTaskTest {
@Test(expected = IllegalStateException.class) @Test(expected = IllegalStateException.class)
public void shouldThrowIllegalStateExceptionOnScheduleIfCurrentNodeIsNull() { public void shouldThrowIllegalStateExceptionOnScheduleIfCurrentNodeIsNull() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.schedule(1, PunctuationType.STREAM_TIME, new Punctuator() { task.schedule(1, PunctuationType.STREAM_TIME, new Punctuator() {
@Override @Override
public void punctuate(long timestamp) { public void punctuate(final long timestamp) {
// no-op // no-op
} }
}); });
@ -747,11 +740,11 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotThrowExceptionOnScheduleIfCurrentNodeIsNotNull() { public void shouldNotThrowExceptionOnScheduleIfCurrentNodeIsNotNull() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.processorContext.setCurrentNode(processorStreamTime); task.processorContext.setCurrentNode(processorStreamTime);
task.schedule(1, PunctuationType.STREAM_TIME, new Punctuator() { task.schedule(1, PunctuationType.STREAM_TIME, new Punctuator() {
@Override @Override
public void punctuate(long timestamp) { public void punctuate(final long timestamp) {
// no-op // no-op
} }
}); });
@ -775,7 +768,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldInitAndBeginTransactionOnCreateIfEosEnabled() { public void shouldInitAndBeginTransactionOnCreateIfEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
task.initializeTopology(); task.initializeTopology();
assertTrue(producer.transactionInitialized()); assertTrue(producer.transactionInitialized());
@ -784,7 +777,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotThrowOnCloseIfTaskWasNotInitializedWithEosEnabled() { public void shouldNotThrowOnCloseIfTaskWasNotInitializedWithEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
assertTrue(!producer.transactionInFlight()); assertTrue(!producer.transactionInFlight());
task.close(false, false); task.close(false, false);
@ -792,7 +785,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotInitOrBeginTransactionOnCreateIfEosDisabled() { public void shouldNotInitOrBeginTransactionOnCreateIfEosDisabled() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
assertFalse(producer.transactionInitialized()); assertFalse(producer.transactionInitialized());
assertFalse(producer.transactionInFlight()); assertFalse(producer.transactionInFlight());
@ -800,11 +793,10 @@ public class StreamTaskTest {
@Test @Test
public void shouldSendOffsetsAndCommitTransactionButNotStartNewTransactionOnSuspendIfEosEnabled() { public void shouldSendOffsetsAndCommitTransactionButNotStartNewTransactionOnSuspendIfEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
task.initializeTopology(); task.initializeTopology();
task.addRecords(partition1, Collections.singletonList( task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
task.process(); task.process();
task.suspend(); task.suspend();
@ -815,7 +807,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldCommitTransactionOnSuspendEvenIfTransactionIsEmptyIfEosEnabled() { public void shouldCommitTransactionOnSuspendEvenIfTransactionIsEmptyIfEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
task.initializeTopology(); task.initializeTopology();
task.suspend(); task.suspend();
@ -825,9 +817,8 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotSendOffsetsAndCommitTransactionNorStartNewTransactionOnSuspendIfEosDisabled() { public void shouldNotSendOffsetsAndCommitTransactionNorStartNewTransactionOnSuspendIfEosDisabled() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.addRecords(partition1, Collections.singletonList( task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
task.process(); task.process();
task.suspend(); task.suspend();
@ -838,11 +829,10 @@ public class StreamTaskTest {
@Test @Test
public void shouldStartNewTransactionOnResumeIfEosEnabled() { public void shouldStartNewTransactionOnResumeIfEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
task.initializeTopology(); task.initializeTopology();
task.addRecords(partition1, Collections.singletonList( task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
task.process(); task.process();
task.suspend(); task.suspend();
@ -853,10 +843,9 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotStartNewTransactionOnResumeIfEosDisabled() { public void shouldNotStartNewTransactionOnResumeIfEosDisabled() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.addRecords(partition1, Collections.singletonList( task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
task.process(); task.process();
task.suspend(); task.suspend();
@ -866,11 +855,10 @@ public class StreamTaskTest {
@Test @Test
public void shouldStartNewTransactionOnCommitIfEosEnabled() { public void shouldStartNewTransactionOnCommitIfEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
task.initializeTopology(); task.initializeTopology();
task.addRecords(partition1, Collections.singletonList( task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
task.process(); task.process();
task.commit(); task.commit();
@ -879,10 +867,9 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotStartNewTransactionOnCommitIfEosDisabled() { public void shouldNotStartNewTransactionOnCommitIfEosDisabled() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.addRecords(partition1, Collections.singletonList( task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 0)));
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
task.process(); task.process();
task.commit(); task.commit();
@ -891,7 +878,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldAbortTransactionOnDirtyClosedIfEosEnabled() { public void shouldAbortTransactionOnDirtyClosedIfEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
task.initializeTopology(); task.initializeTopology();
task.close(false, false); task.close(false, false);
task = null; task = null;
@ -901,7 +888,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotAbortTransactionOnZombieClosedIfEosEnabled() { public void shouldNotAbortTransactionOnZombieClosedIfEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
task.close(false, true); task.close(false, true);
task = null; task = null;
@ -910,7 +897,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldNotAbortTransactionOnDirtyClosedIfEosDisabled() { public void shouldNotAbortTransactionOnDirtyClosedIfEosDisabled() {
task = createStatelessTask(false); task = createStatelessTask(createConfig(false));
task.close(false, false); task.close(false, false);
task = null; task = null;
@ -919,7 +906,7 @@ public class StreamTaskTest {
@Test @Test
public void shouldCloseProducerOnCloseWhenEosEnabled() { public void shouldCloseProducerOnCloseWhenEosEnabled() {
task = createStatelessTask(true); task = createStatelessTask(createConfig(true));
task.close(true, false); task.close(true, false);
task = null; task = null;
@ -935,7 +922,7 @@ public class StreamTaskTest {
try { try {
task.commit(); task.commit();
fail("should have thrown an exception"); fail("should have thrown an exception");
} catch (Exception e) { } catch (final Exception e) {
// all good // all good
} }
} }
@ -949,21 +936,21 @@ public class StreamTaskTest {
try { try {
task.suspend(); task.suspend();
fail("should have thrown an exception"); fail("should have thrown an exception");
} catch (Exception e) { } catch (final Exception e) {
// all good // all good
} }
} }
@Test @Test
public void shouldCloseStateManagerIfFailureOnTaskClose() { public void shouldCloseStateManagerIfFailureOnTaskClose() {
task = createStatefulTaskThatThrowsExceptionOnClose(true, false); task = createStatefulTaskThatThrowsExceptionOnClose();
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
try { try {
task.close(true, false); task.close(true, false);
fail("should have thrown an exception"); fail("should have thrown an exception");
} catch (Exception e) { } catch (final Exception e) {
// all good // all good
} }
@ -976,21 +963,21 @@ public class StreamTaskTest {
final StreamTask task = createTaskThatThrowsException(); final StreamTask task = createTaskThatThrowsException();
try { try {
task.close(false, false); task.close(false, false);
} catch (Exception e) { } catch (final Exception e) {
fail("should have not closed unitialized topology"); fail("should have not closed non-initialized topology");
} }
} }
@Test @Test
public void shouldBeInitializedIfChangelogPartitionsIsEmpty() { public void shouldBeInitializedIfChangelogPartitionsIsEmpty() {
final StreamTask task = createStatefulTask(false, false); final StreamTask task = createStatefulTask(createConfig(false), false);
assertTrue(task.initializeStateStores()); assertTrue(task.initializeStateStores());
} }
@Test @Test
public void shouldNotBeInitializedIfChangelogPartitionsIsNonEmpty() { public void shouldNotBeInitializedIfChangelogPartitionsIsNonEmpty() {
final StreamTask task = createStatefulTask(false, true); final StreamTask task = createStatefulTask(createConfig(false), true);
assertFalse(task.initializeStateStores()); assertFalse(task.initializeStateStores());
} }
@ -998,79 +985,91 @@ public class StreamTaskTest {
@Test @Test
public void shouldReturnOffsetsForRepartitionTopicsForPurging() { public void shouldReturnOffsetsForRepartitionTopicsForPurging() {
final TopicPartition repartition = new TopicPartition("repartition", 1); final TopicPartition repartition = new TopicPartition("repartition", 1);
final ProcessorTopology topology = ProcessorTopology.withRepartitionTopics( final ProcessorTopology topology = ProcessorTopology.withRepartitionTopics(
Utils.<ProcessorNode>mkList(source1, source2), Utils.<ProcessorNode>mkList(source1, source2),
new HashMap<String, SourceNode>() { mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(repartition.topic(), (SourceNode) source2)),
{ Collections.singleton(repartition.topic())
put(topic1, source1);
put(repartition.topic(), source2);
}
},
Collections.singleton(repartition.topic())
); );
consumer.assign(Arrays.asList(partition1, repartition)); consumer.assign(Arrays.asList(partition1, repartition));
task = new StreamTask(taskId00, Utils.mkSet(partition1, repartition), topology, consumer, changelogReader, config, task = new StreamTask(
streamsMetrics, stateDirectory, null, time, producer); taskId00,
Utils.mkSet(partition1, repartition),
topology,
consumer,
changelogReader,
createConfig(false),
streamsMetrics,
stateDirectory,
null,
time,
producer
);
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); task.initializeTopology();
task.addRecords(partition1, Collections.singletonList( task.addRecords(partition1, singletonList(getConsumerRecord(partition1, 5L)));
new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5L, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue))); task.addRecords(repartition, singletonList(getConsumerRecord(repartition, 10L)));
task.addRecords(repartition, Collections.singletonList(
new ConsumerRecord<>(repartition.topic(), repartition.partition(), 10L, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)));
assertTrue(task.process()); assertTrue(task.process());
assertTrue(task.process()); assertTrue(task.process());
task.commit(); task.commit();
Map<TopicPartition, Long> map = task.purgableOffsets(); final Map<TopicPartition, Long> map = task.purgableOffsets();
assertThat(map, equalTo(Collections.singletonMap(repartition, 11L))); 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( final ProcessorTopology topology = ProcessorTopology.with(
Utils.<ProcessorNode>mkList(source1, source2), Utils.<ProcessorNode>mkList(source1, source2),
new HashMap<String, SourceNode>() { mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source2)),
{ singletonList(stateStore),
put(topic1, source1); logged ? Collections.singletonMap(storeName, storeName + "-changelog") : Collections.<String, String>emptyMap());
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, return new StreamTask(
streamsMetrics, stateDirectory, null, time, producer); 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( final ProcessorTopology topology = ProcessorTopology.with(
Utils.<ProcessorNode>mkList(source1, source3), Utils.<ProcessorNode>mkList(source1, source3),
new HashMap<String, SourceNode>() { mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source3)),
{ singletonList(stateStore),
put(topic1, source1); Collections.<String, String>emptyMap());
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, return new StreamTask(
streamsMetrics, stateDirectory, null, time, producer); 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( final ProcessorTopology topology = ProcessorTopology.withSources(
Utils.<ProcessorNode>mkList(source1, source2, processorStreamTime, processorSystemTime), Utils.<ProcessorNode>mkList(source1, source2, processorStreamTime, processorSystemTime),
new HashMap<String, SourceNode>() { mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source2))
{
put(topic1, source1);
put(topic2, source2);
}
}
); );
source1.addChild(processorStreamTime); source1.addChild(processorStreamTime);
@ -1078,20 +1077,26 @@ public class StreamTaskTest {
source1.addChild(processorSystemTime); source1.addChild(processorSystemTime);
source2.addChild(processorSystemTime); source2.addChild(processorSystemTime);
return new StreamTask(taskId00, partitions, topology, consumer, changelogReader, eosEnabled ? eosConfig : config, return new StreamTask(
streamsMetrics, stateDirectory, null, time, producer); taskId00,
partitions,
topology,
consumer,
changelogReader,
streamsConfig,
streamsMetrics,
stateDirectory,
null,
time,
producer
);
} }
// this task will throw exception when processing (on partition2), flushing, suspending and closing // this task will throw exception when processing (on partition2), flushing, suspending and closing
private StreamTask createTaskThatThrowsException() { private StreamTask createTaskThatThrowsException() {
final ProcessorTopology topology = ProcessorTopology.withSources( final ProcessorTopology topology = ProcessorTopology.withSources(
Utils.<ProcessorNode>mkList(source1, source3, processorStreamTime, processorSystemTime), Utils.<ProcessorNode>mkList(source1, source3, processorStreamTime, processorSystemTime),
new HashMap<String, SourceNode>() { mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source3))
{
put(topic1, source1);
put(topic2, source3);
}
}
); );
source1.addChild(processorStreamTime); source1.addChild(processorStreamTime);
@ -1099,9 +1104,19 @@ public class StreamTaskTest {
source1.addChild(processorSystemTime); source1.addChild(processorSystemTime);
source3.addChild(processorSystemTime); source3.addChild(processorSystemTime);
return new StreamTask(taskId00, partitions, topology, consumer, changelogReader, config, return new StreamTask(
streamsMetrics, stateDirectory, null, time, producer) { taskId00,
partitions,
topology,
consumer,
changelogReader,
createConfig(false),
streamsMetrics,
stateDirectory,
null,
time,
producer
) {
@Override @Override
protected void flushState() { protected void flushState() {
throw new RuntimeException("KABOOM!"); throw new RuntimeException("KABOOM!");
@ -1109,8 +1124,19 @@ public class StreamTaskTest {
}; };
} }
private Iterable<ConsumerRecord<byte[], byte[]>> records(final ConsumerRecord<byte[], byte[]>... recs) { private ConsumerRecord<byte[], byte[]> getConsumerRecord(final TopicPartition topicPartition, final long offset) {
return Arrays.asList(recs); return new ConsumerRecord<>(
topicPartition.topic(),
topicPartition.partition(),
offset,
0L,
TimestampType.CREATE_TIME,
0L,
0,
0,
recordKey,
recordValue
);
} }
} }

View File

@ -72,6 +72,10 @@ import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.UUID; 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.hamcrest.CoreMatchers.equalTo;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
@ -93,7 +97,7 @@ public class StreamThreadTest {
private InternalTopologyBuilder internalTopologyBuilder; private InternalTopologyBuilder internalTopologyBuilder;
private final StreamsConfig config = new StreamsConfig(configProps(false)); private final StreamsConfig config = new StreamsConfig(configProps(false));
private final String stateDir = TestUtils.tempDirectory().getPath(); 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 StreamsMetadataState streamsMetadataState;
private final ConsumedInternal<Object, Object> consumed = new ConsumedInternal<>(); 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 task2 = new TaskId(0, 2);
private final TaskId task3 = new TaskId(1, 1); private final TaskId task3 = new TaskId(1, 1);
private Properties configProps(final boolean enableEos) { private Properties configProps(final boolean enableEoS) {
return new Properties() { return mkProperties(mkMap(
{ mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId),
setProperty(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"),
setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"),
setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()),
setProperty(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()),
setProperty(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()); mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, enableEoS ? StreamsConfig.EXACTLY_ONCE : StreamsConfig.AT_LEAST_ONCE)
if (enableEos) { ));
setProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE);
}
}
};
} }
@Test @Test
@ -146,8 +146,8 @@ public class StreamThreadTest {
final ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; final ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener;
thread.setState(StreamThread.State.RUNNING); thread.setState(StreamThread.State.RUNNING);
List<TopicPartition> revokedPartitions; final List<TopicPartition> revokedPartitions;
List<TopicPartition> assignedPartitions; final List<TopicPartition> assignedPartitions;
// revoke nothing // revoke nothing
revokedPartitions = Collections.emptyList(); revokedPartitions = Collections.emptyList();
@ -156,7 +156,7 @@ public class StreamThreadTest {
assertEquals(thread.state(), StreamThread.State.PARTITIONS_REVOKED); assertEquals(thread.state(), StreamThread.State.PARTITIONS_REVOKED);
// assign single partition // assign single partition
assignedPartitions = Collections.singletonList(t1p1); assignedPartitions = singletonList(t1p1);
thread.taskManager().setAssignmentMetadata(Collections.<TaskId, Set<TopicPartition>>emptyMap(), Collections.<TaskId, Set<TopicPartition>>emptyMap()); thread.taskManager().setAssignmentMetadata(Collections.<TaskId, Set<TopicPartition>>emptyMap(), Collections.<TaskId, Set<TopicPartition>>emptyMap());
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer; 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); Assert.assertEquals(StreamThread.State.PARTITIONS_ASSIGNED, stateListener.oldState);
thread.shutdown(); thread.shutdown();
assertTrue(thread.state() == StreamThread.State.PENDING_SHUTDOWN); assertSame(StreamThread.State.PENDING_SHUTDOWN, thread.state());
} }
@Test @Test
@ -199,39 +199,44 @@ public class StreamThreadTest {
assertEquals(thread.state(), StreamThread.State.DEAD); assertEquals(thread.state(), StreamThread.State.DEAD);
} }
private Cluster createCluster(final int numNodes) { private Cluster createCluster() {
HashMap<Integer, Node> nodes = new HashMap<>(); final Node node = new Node(0, "localhost", 8121);
for (int i = 0; i < numNodes; ++i) { return new Cluster(
nodes.put(i, new Node(i, "localhost", 8121 + i)); "mockClusterId",
} singletonList(node),
return new Cluster("mockClusterId", nodes.values(), Collections.<PartitionInfo>emptySet(),
Collections.<PartitionInfo>emptySet(), Collections.<String>emptySet(), Collections.<String>emptySet(),
Collections.<String>emptySet(), nodes.get(0)); 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) { if (eosEnabled) {
clientSupplier.setApplicationIdForProducer(applicationId); clientSupplier.setApplicationIdForProducer(applicationId);
} }
clientSupplier.setClusterForAdminClient(createCluster(1)); clientSupplier.setClusterForAdminClient(createCluster());
return StreamThread.create(internalTopologyBuilder, return StreamThread.create(
config, internalTopologyBuilder,
clientSupplier, config,
clientSupplier.getAdminClient(config.getAdminConfigs(clientId)), clientSupplier,
processId, clientSupplier.getAdminClient(config.getAdminConfigs(clientId)),
clientId, processId,
metrics, clientId,
mockTime, metrics,
streamsMetadataState, mockTime,
0, streamsMetadataState,
stateDirectory, 0,
new MockStateRestoreListener()); stateDirectory,
new MockStateRestoreListener());
} }
@Test @Test
public void testMetrics() { public void testMetricsCreatedAtStartup() {
final StreamThread thread = createStreamThread(clientId, config, false); final StreamThread thread = createStreamThread(clientId, config, false);
final String defaultGroupName = "stream-metrics"; final String defaultGroupName = "stream-metrics";
final String defaultPrefix = "thread." + thread.getName(); 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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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 Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 1); final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 1);
StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap()); final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
final StreamThread thread = new StreamThread(mockTime, final StreamThread thread = new StreamThread(
config, mockTime,
consumer, config,
consumer, consumer,
null, consumer,
taskManager, null,
streamsMetrics, taskManager,
internalTopologyBuilder, streamsMetrics,
clientId, internalTopologyBuilder,
new LogContext("") clientId,
new LogContext("")
); );
thread.maybeCommit(mockTime.milliseconds()); thread.maybeCommit(mockTime.milliseconds());
mockTime.sleep(commitInterval - 10L); mockTime.sleep(commitInterval - 10L);
@ -296,7 +309,7 @@ public class StreamThreadTest {
@SuppressWarnings({"unchecked", "ThrowableNotThrown"}) @SuppressWarnings({"unchecked", "ThrowableNotThrown"})
@Test @Test
public void shouldNotCauseExceptionIfNothingCommited() { public void shouldNotCauseExceptionIfNothingCommitted() {
final long commitInterval = 1000L; final long commitInterval = 1000L;
final Properties props = configProps(false); final Properties props = configProps(false);
props.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); props.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir);
@ -306,17 +319,18 @@ public class StreamThreadTest {
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class); final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 0); final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 0);
StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap()); final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
final StreamThread thread = new StreamThread(mockTime, final StreamThread thread = new StreamThread(
config, mockTime,
consumer, config,
consumer, consumer,
null, consumer,
taskManager, null,
streamsMetrics, taskManager,
internalTopologyBuilder, streamsMetrics,
clientId, internalTopologyBuilder,
new LogContext("")); clientId,
new LogContext(""));
thread.maybeCommit(mockTime.milliseconds()); thread.maybeCommit(mockTime.milliseconds());
mockTime.sleep(commitInterval - 10L); mockTime.sleep(commitInterval - 10L);
thread.maybeCommit(mockTime.milliseconds()); thread.maybeCommit(mockTime.milliseconds());
@ -337,17 +351,18 @@ public class StreamThreadTest {
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class); final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
final TaskManager taskManager = mockTaskManagerCommit(consumer, 2, 1); final TaskManager taskManager = mockTaskManagerCommit(consumer, 2, 1);
StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap()); final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(metrics, "", "", Collections.<String, String>emptyMap());
final StreamThread thread = new StreamThread(mockTime, final StreamThread thread = new StreamThread(
config, mockTime,
consumer, config,
consumer, consumer,
null, consumer,
taskManager, null,
streamsMetrics, taskManager,
internalTopologyBuilder, streamsMetrics,
clientId, internalTopologyBuilder,
new LogContext("")); clientId,
new LogContext(""));
thread.maybeCommit(mockTime.milliseconds()); thread.maybeCommit(mockTime.milliseconds());
mockTime.sleep(commitInterval + 1); mockTime.sleep(commitInterval + 1);
thread.maybeCommit(mockTime.milliseconds()); thread.maybeCommit(mockTime.milliseconds());
@ -385,7 +400,7 @@ public class StreamThreadTest {
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer; final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
mockConsumer.assign(assignedPartitions); mockConsumer.assign(assignedPartitions);
Map<TopicPartition, Long> beginOffsets = new HashMap<>(); final Map<TopicPartition, Long> beginOffsets = new HashMap<>();
beginOffsets.put(t1p1, 0L); beginOffsets.put(t1p1, 0L);
beginOffsets.put(t1p2, 0L); beginOffsets.put(t1p2, 0L);
mockConsumer.updateBeginningOffsets(beginOffsets); mockConsumer.updateBeginningOffsets(beginOffsets);
@ -422,7 +437,7 @@ public class StreamThreadTest {
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer; final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
mockConsumer.assign(assignedPartitions); mockConsumer.assign(assignedPartitions);
Map<TopicPartition, Long> beginOffsets = new HashMap<>(); final Map<TopicPartition, Long> beginOffsets = new HashMap<>();
beginOffsets.put(t1p1, 0L); beginOffsets.put(t1p1, 0L);
beginOffsets.put(t1p2, 0L); beginOffsets.put(t1p2, 0L);
mockConsumer.updateBeginningOffsets(beginOffsets); mockConsumer.updateBeginningOffsets(beginOffsets);
@ -456,7 +471,7 @@ public class StreamThreadTest {
thread.taskManager().setAssignmentMetadata(activeTasks, Collections.<TaskId, Set<TopicPartition>>emptyMap()); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.<TaskId, Set<TopicPartition>>emptyMap());
final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer; final MockConsumer<byte[], byte[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
mockConsumer.assign(assignedPartitions); mockConsumer.assign(assignedPartitions);
Map<TopicPartition, Long> beginOffsets = new HashMap<>(); final Map<TopicPartition, Long> beginOffsets = new HashMap<>();
beginOffsets.put(t1p1, 0L); beginOffsets.put(t1p1, 0L);
beginOffsets.put(t1p2, 0L); beginOffsets.put(t1p2, 0L);
mockConsumer.updateBeginningOffsets(beginOffsets); mockConsumer.updateBeginningOffsets(beginOffsets);
@ -483,30 +498,32 @@ public class StreamThreadTest {
EasyMock.replay(taskManager, consumer); EasyMock.replay(taskManager, consumer);
final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl( final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(
metrics, metrics,
"", "",
"", "",
Collections.<String, String>emptyMap()); Collections.<String, String>emptyMap()
);
final StreamThread thread = new StreamThread( final StreamThread thread = new StreamThread(
mockTime, mockTime,
config, config,
consumer, consumer,
consumer, consumer,
null, null,
taskManager, taskManager,
streamsMetrics, streamsMetrics,
internalTopologyBuilder, internalTopologyBuilder,
clientId, clientId,
new LogContext("")); new LogContext("")
);
thread.setStateListener( thread.setStateListener(
new StreamThread.StateListener() { new StreamThread.StateListener() {
@Override @Override
public void onChange(final Thread t, final ThreadStateTransitionValidator newState, final ThreadStateTransitionValidator oldState) { public void onChange(final Thread t, final ThreadStateTransitionValidator newState, final ThreadStateTransitionValidator oldState) {
if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.RUNNING) { if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.RUNNING) {
thread.shutdown(); thread.shutdown();
}
} }
}); }
});
thread.run(); thread.run();
EasyMock.verify(taskManager); EasyMock.verify(taskManager);
} }
@ -521,21 +538,23 @@ public class StreamThreadTest {
EasyMock.replay(taskManager, consumer); EasyMock.replay(taskManager, consumer);
final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl( final StreamThread.StreamsMetricsThreadImpl streamsMetrics = new StreamThread.StreamsMetricsThreadImpl(
metrics, metrics,
"", "",
"", "",
Collections.<String, String>emptyMap()); Collections.<String, String>emptyMap()
);
final StreamThread thread = new StreamThread( final StreamThread thread = new StreamThread(
mockTime, mockTime,
config, config,
consumer, consumer,
consumer, consumer,
null, null,
taskManager, taskManager,
streamsMetrics, streamsMetrics,
internalTopologyBuilder, internalTopologyBuilder,
clientId, clientId,
new LogContext("")); new LogContext("")
);
thread.shutdown(); thread.shutdown();
EasyMock.verify(taskManager); EasyMock.verify(taskManager);
} }
@ -601,7 +620,7 @@ public class StreamThreadTest {
final MockConsumer<byte[], byte[]> consumer = clientSupplier.consumer; 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.setState(StreamThread.State.RUNNING);
thread.rebalanceListener.onPartitionsRevoked(null); thread.rebalanceListener.onPartitionsRevoked(null);
@ -749,7 +768,7 @@ public class StreamThreadTest {
thread.runOnce(-1); thread.runOnce(-1);
ThreadMetadata threadMetadata = thread.threadMetadata(); final ThreadMetadata threadMetadata = thread.threadMetadata();
assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState()); assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState());
assertTrue(threadMetadata.activeTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1)))); assertTrue(threadMetadata.activeTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1))));
assertTrue(threadMetadata.standbyTasks().isEmpty()); assertTrue(threadMetadata.standbyTasks().isEmpty());
@ -762,12 +781,15 @@ public class StreamThreadTest {
final StreamThread thread = createStreamThread(clientId, config, false); final StreamThread thread = createStreamThread(clientId, config, false);
final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer; final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer;
restoreConsumer.updatePartitions("stream-thread-test-count-one-changelog", restoreConsumer.updatePartitions(
Collections.singletonList(new PartitionInfo("stream-thread-test-count-one-changelog", "stream-thread-test-count-one-changelog",
0, singletonList(new PartitionInfo("stream-thread-test-count-one-changelog",
null, 0,
new Node[0], null,
new Node[0]))); new Node[0],
new Node[0])
)
);
final HashMap<TopicPartition, Long> offsets = new HashMap<>(); final HashMap<TopicPartition, Long> offsets = new HashMap<>();
offsets.put(new TopicPartition("stream-thread-test-count-one-changelog", 1), 0L); offsets.put(new TopicPartition("stream-thread-test-count-one-changelog", 1), 0L);
@ -789,7 +811,7 @@ public class StreamThreadTest {
thread.runOnce(-1); thread.runOnce(-1);
ThreadMetadata threadMetadata = thread.threadMetadata(); final ThreadMetadata threadMetadata = thread.threadMetadata();
assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState()); assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState());
assertTrue(threadMetadata.standbyTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1)))); assertTrue(threadMetadata.standbyTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1))));
assertTrue(threadMetadata.activeTasks().isEmpty()); assertTrue(threadMetadata.activeTasks().isEmpty());
@ -804,17 +826,22 @@ public class StreamThreadTest {
final TopicPartition partition1 = new TopicPartition(changelogName, 1); final TopicPartition partition1 = new TopicPartition(changelogName, 1);
final TopicPartition partition2 = t2p1; final TopicPartition partition2 = t2p1;
internalStreamsBuilder.stream(Collections.singleton(topic1), consumed) 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, "")); internalStreamsBuilder.table(topic2, new ConsumedInternal(), new MaterializedInternal(Materialized.as(storeName2), internalStreamsBuilder, ""));
final StreamThread thread = createStreamThread(clientId, config, false); final StreamThread thread = createStreamThread(clientId, config, false);
final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer; final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer;
restoreConsumer.updatePartitions(changelogName, restoreConsumer.updatePartitions(changelogName,
Collections.singletonList(new PartitionInfo(changelogName, singletonList(
1, new PartitionInfo(
null, changelogName,
new Node[0], 1,
new Node[0]))); null,
new Node[0],
new Node[0]
)
)
);
restoreConsumer.assign(Utils.mkSet(partition1, partition2)); restoreConsumer.assign(Utils.mkSet(partition1, partition2));
restoreConsumer.updateEndOffsets(Collections.singletonMap(partition1, 10L)); restoreConsumer.updateEndOffsets(Collections.singletonMap(partition1, 10L));
@ -867,30 +894,30 @@ public class StreamThreadTest {
public Processor<Object, Object> get() { public Processor<Object, Object> get() {
return new Processor<Object, Object>() { return new Processor<Object, Object>() {
@Override @Override
public void init(ProcessorContext context) { public void init(final ProcessorContext context) {
context.schedule(100L, PunctuationType.STREAM_TIME, new Punctuator() { context.schedule(100L, PunctuationType.STREAM_TIME, new Punctuator() {
@Override @Override
public void punctuate(long timestamp) { public void punctuate(final long timestamp) {
punctuatedStreamTime.add(timestamp); punctuatedStreamTime.add(timestamp);
} }
}); });
context.schedule(100L, PunctuationType.WALL_CLOCK_TIME, new Punctuator() { context.schedule(100L, PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
@Override @Override
public void punctuate(long timestamp) { public void punctuate(final long timestamp) {
punctuatedWallClockTime.add(timestamp); punctuatedWallClockTime.add(timestamp);
} }
}); });
} }
@Override @Override
public void process(Object key, Object value) { } public void process(final Object key, final Object value) {}
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@Override @Override
public void punctuate(long timestamp) { } public void punctuate(final long timestamp) {}
@Override @Override
public void close() { } public void close() {}
}; };
} }
}; };
@ -959,18 +986,18 @@ public class StreamThreadTest {
final StreamThread thread = createStreamThread(clientId, config, false); final StreamThread thread = createStreamThread(clientId, config, false);
final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer; final MockConsumer<byte[], byte[]> restoreConsumer = clientSupplier.restoreConsumer;
restoreConsumer.updatePartitions("stream-thread-test-count-one-changelog", restoreConsumer.updatePartitions("stream-thread-test-count-one-changelog",
Utils.mkList( Utils.mkList(
new PartitionInfo("stream-thread-test-count-one-changelog", new PartitionInfo("stream-thread-test-count-one-changelog",
0, 0,
null, null,
new Node[0], new Node[0],
new Node[0]), new Node[0]),
new PartitionInfo("stream-thread-test-count-one-changelog", new PartitionInfo("stream-thread-test-count-one-changelog",
1, 1,
null, null,
new Node[0], new Node[0],
new Node[0]) new Node[0])
)); ));
final HashMap<TopicPartition, Long> offsets = new HashMap<>(); 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", 0), 0L);
offsets.put(new TopicPartition("stream-thread-test-count-one-changelog", 1), 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) internalStreamsBuilder.stream(Collections.singleton("topic"), consumed)
.groupByKey().count(Materialized.<Object, Long, KeyValueStore<Bytes, byte[]>>as("count")); .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[]> mockConsumer = (MockConsumer<byte[], byte[]>) thread.consumer;
final MockConsumer<byte[], byte[]> mockRestoreConsumer = (MockConsumer<byte[], byte[]>) thread.restoreConsumer; final MockConsumer<byte[], byte[]> mockRestoreConsumer = (MockConsumer<byte[], byte[]>) thread.restoreConsumer;
@ -1017,26 +1044,33 @@ public class StreamThreadTest {
activeTasks.put(new TaskId(0, 0), topicPartitionSet); activeTasks.put(new TaskId(0, 0), topicPartitionSet);
thread.taskManager().setAssignmentMetadata(activeTasks, Collections.<TaskId, Set<TopicPartition>>emptyMap()); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.<TaskId, Set<TopicPartition>>emptyMap());
mockConsumer.updatePartitions("topic", new ArrayList<PartitionInfo>() { mockConsumer.updatePartitions(
{ "topic",
add(new PartitionInfo("topic", singletonList(
new PartitionInfo(
"topic",
0, 0,
null, null,
new Node[0], new Node[0],
new Node[0])); new Node[0]
} )
}); )
);
mockConsumer.updateBeginningOffsets(Collections.singletonMap(topicPartition, 0L)); mockConsumer.updateBeginningOffsets(Collections.singletonMap(topicPartition, 0L));
mockRestoreConsumer.updatePartitions("stream-thread-test-count-changelog", new ArrayList<PartitionInfo>() { mockRestoreConsumer.updatePartitions(
{ "stream-thread-test-count-changelog",
add(new PartitionInfo("stream-thread-test-count-changelog", singletonList(
new PartitionInfo(
"stream-thread-test-count-changelog",
0, 0,
null, null,
new Node[0], new Node[0],
new Node[0])); new Node[0]
} )
}); )
);
final TopicPartition changelogPartition = new TopicPartition("stream-thread-test-count-changelog", 0); final TopicPartition changelogPartition = new TopicPartition("stream-thread-test-count-changelog", 0);
final Set<TopicPartition> changelogPartitionSet = Collections.singleton(changelogPartition); final Set<TopicPartition> changelogPartitionSet = Collections.singleton(changelogPartition);
mockRestoreConsumer.updateBeginningOffsets(Collections.singletonMap(changelogPartition, 0L)); mockRestoreConsumer.updateBeginningOffsets(Collections.singletonMap(changelogPartition, 0L));
@ -1138,7 +1172,8 @@ public class StreamThreadTest {
assertEquals(6.0, metrics.metric(skippedTotalMetric).metricValue()); 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()); assertEquals(state.name(), metadata.threadState());
assertTrue(metadata.activeTasks().isEmpty()); assertTrue(metadata.activeTasks().isEmpty());
assertTrue(metadata.standbyTasks().isEmpty()); assertTrue(metadata.standbyTasks().isEmpty());

View File

@ -32,57 +32,57 @@ public class StreamsMetricsImplTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void testNullMetrics() { public void testNullMetrics() {
String groupName = "doesNotMatter"; final String groupName = "doesNotMatter";
Map<String, String> tags = new HashMap<>(); final Map<String, String> tags = new HashMap<>();
StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(null, groupName, tags); new StreamsMetricsImpl(null, groupName, tags);
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void testRemoveNullSensor() { public void testRemoveNullSensor() {
String groupName = "doesNotMatter"; final String groupName = "doesNotMatter";
Map<String, String> tags = new HashMap<>(); final Map<String, String> tags = new HashMap<>();
StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
streamsMetrics.removeSensor(null); streamsMetrics.removeSensor(null);
} }
@Test @Test
public void testRemoveSensor() { public void testRemoveSensor() {
String groupName = "doesNotMatter"; final String groupName = "doesNotMatter";
String sensorName = "sensor1"; final String sensorName = "sensor1";
String scope = "scope"; final String scope = "scope";
String entity = "entity"; final String entity = "entity";
String operation = "put"; final String operation = "put";
Map<String, String> tags = new HashMap<>(); final Map<String, String> tags = new HashMap<>();
StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags);
Sensor sensor1 = streamsMetrics.addSensor(sensorName, Sensor.RecordingLevel.DEBUG); final Sensor sensor1 = streamsMetrics.addSensor(sensorName, Sensor.RecordingLevel.DEBUG);
streamsMetrics.removeSensor(sensor1); 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); 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); 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); streamsMetrics.removeSensor(sensor3);
} }
@Test @Test
public void testLatencyMetrics() { public void testLatencyMetrics() {
String groupName = "doesNotMatter"; final String groupName = "doesNotMatter";
String scope = "scope"; final String scope = "scope";
String entity = "entity"; final String entity = "entity";
String operation = "put"; final String operation = "put";
Map<String, String> tags = new HashMap<>(); final Map<String, String> tags = new HashMap<>();
StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags); 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(); 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 // 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 final int meterMetricsCount = 2; // Each Meter is a combination of a Rate and a Total
int otherMetricsCount = 4; final int otherMetricsCount = 4;
assertEquals(meterMetricsCount * 2 + otherMetricsCount + 1, metrics.size()); assertEquals(meterMetricsCount * 2 + otherMetricsCount + 1, metrics.size());
streamsMetrics.removeSensor(sensor1); streamsMetrics.removeSensor(sensor1);
@ -92,17 +92,17 @@ public class StreamsMetricsImplTest {
@Test @Test
public void testThroughputMetrics() { public void testThroughputMetrics() {
String groupName = "doesNotMatter"; final String groupName = "doesNotMatter";
String scope = "scope"; final String scope = "scope";
String entity = "entity"; final String entity = "entity";
String operation = "put"; final String operation = "put";
Map<String, String> tags = new HashMap<>(); final Map<String, String> tags = new HashMap<>();
StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags); 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(); 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 // 2 meter metrics plus a common metric that keeps track of total registered metrics in Metrics() constructor
assertEquals(meterMetricsCount * 2 + 1, metrics.size()); assertEquals(meterMetricsCount * 2 + 1, metrics.size());

View File

@ -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. * 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 * 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. * {@link org.apache.kafka.streams.state.internals.MeteredKeyValueStore} to monitor and write its entries to the Kafka topic.
* <p> *
* <h2>Basic usage</h2> * <h2>Basic usage</h2>
* This component can be used to help test a {@link KeyValueStore}'s ability to read and write entries. * 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)); * assertEquals(true, driver.flushedEntryRemoved(5));
* </pre> * </pre>
* *
* <p> *
* <h2>Restoring a store</h2> * <h2>Restoring a store</h2>
* This component can be used to test whether a {@link KeyValueStore} implementation properly * 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 * {@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 * {@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. * 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}, * @param keyClass the class for the keys; must be one of {@code String.class}, {@code Integer.class},
* {@code Long.class}, or {@code byte[].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}, * @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 * @return the test driver; never null
*/ */
public static <K, V> KeyValueStoreTestDriver<K, V> create(final Class<K> keyClass, final Class<V> valueClass) { 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 * deserializers. This can be used when store is created to rely upon the ProcessorContext's default key and value serializers
* and deserializers. * and deserializers.
* *
* @param keySerializer the key 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 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 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 * @param valueDeserializer the value deserializer for the {@link ProcessorContext}; may not be null
* @return the test driver; never null * @return the test driver; never null
*/ */
@ -195,7 +195,7 @@ public class KeyValueStoreTestDriver<K, V> {
final Long timestamp, final Long timestamp,
final Serializer<K1> keySerializer, final Serializer<K1> keySerializer,
final Serializer<V1> valueSerializer) { 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 K keyTest = serdes.keyFrom(keySerializer.serialize(topic, key));
final V valueTest = serdes.valueFrom(valueSerializer.serialize(topic, value)); 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(); stateDir.mkdirs();
stateSerdes = serdes; stateSerdes = serdes;
@ -294,7 +295,7 @@ public class KeyValueStoreTestDriver<K, V> {
* assertEquals(3, driver.sizeOf(store)); * assertEquals(3, driver.sizeOf(store));
* </pre> * </pre>
* *
* @param key the key for the entry * @param key the key for the entry
* @param value the value for the entry * @param value the value for the entry
* @see #checkForRestoredEntries(KeyValueStore) * @see #checkForRestoredEntries(KeyValueStore)
*/ */
@ -346,7 +347,7 @@ public class KeyValueStoreTestDriver<K, V> {
*/ */
public int sizeOf(final KeyValueStore<K, V> store) { public int sizeOf(final KeyValueStore<K, V> store) {
int size = 0; int size = 0;
try (KeyValueIterator<K, V> iterator = store.all()) { try (final KeyValueIterator<K, V> iterator = store.all()) {
while (iterator.hasNext()) { while (iterator.hasNext()) {
iterator.next(); iterator.next();
++size; ++size;
@ -360,7 +361,7 @@ public class KeyValueStoreTestDriver<K, V> {
* *
* @param key the key * @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 * @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) { public V flushedEntryStored(final K key) {
return flushedEntries.get(key); return flushedEntries.get(key);
@ -371,7 +372,7 @@ public class KeyValueStoreTestDriver<K, V> {
* *
* @param key the key * @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 * @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) { public boolean flushedEntryRemoved(final K key) {
return flushedRemovals.contains(key); return flushedRemovals.contains(key);

View File

@ -53,6 +53,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import static java.util.Objects.requireNonNull;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertEquals; 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.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
@SuppressWarnings("PointlessArithmeticExpression")
public class RocksDBWindowStoreTest { public class RocksDBWindowStoreTest {
private static final long DEFAULT_CACHE_SIZE_BYTES = 1024 * 1024L; 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()) { private final RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTestTask", new LogContext("RocksDBWindowStoreTestTask "), new DefaultProductionExceptionHandler()) {
@Override @Override
public <K1, V1> void send(final String topic, public <K1, V1> void send(final String topic,
K1 key, final K1 key,
V1 value, final V1 value,
Integer partition, final Integer partition,
Long timestamp, final Long timestamp,
Serializer<K1> keySerializer, final Serializer<K1> keySerializer,
Serializer<V1> valueSerializer) { final Serializer<V1> valueSerializer) {
changeLog.add(new KeyValue<>( changeLog.add(new KeyValue<>(
keySerializer.serialize(topic, key), keySerializer.serialize(topic, key),
valueSerializer.serialize(topic, value)) valueSerializer.serialize(topic, value))
); );
} }
}; };
@ -99,13 +101,14 @@ public class RocksDBWindowStoreTest {
private WindowStore<Integer, String> createWindowStore(final ProcessorContext context, final boolean retainDuplicates) { private WindowStore<Integer, String> createWindowStore(final ProcessorContext context, final boolean retainDuplicates) {
final WindowStore<Integer, String> store = Stores.windowStoreBuilder( final WindowStore<Integer, String> store = Stores.windowStoreBuilder(
Stores.persistentWindowStore(windowName, Stores.persistentWindowStore(
retentionPeriod, windowName,
numSegments, retentionPeriod,
windowSize, numSegments,
retainDuplicates), windowSize,
Serdes.Integer(), retainDuplicates),
Serdes.String()).build(); Serdes.Integer(),
Serdes.String()).build();
store.init(context, store); store.init(context, store);
return store; return store;
@ -158,7 +161,7 @@ public class RocksDBWindowStoreTest {
@Test @Test
public void testRangeAndSinglePointFetch() { public void testRangeAndSinglePointFetch() {
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
long startTime = segmentSize - 4L; final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context); putFirstBatch(windowStore, startTime, context);
@ -203,7 +206,7 @@ public class RocksDBWindowStoreTest {
// Flush the store and verify all current entries were properly flushed ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); 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("zero@0"), entriesByKey.get(0));
assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1)); assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
@ -217,7 +220,7 @@ public class RocksDBWindowStoreTest {
@Test @Test
public void shouldGetAll() { public void shouldGetAll() {
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
long startTime = segmentSize - 4L; final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context); putFirstBatch(windowStore, startTime, context);
@ -228,15 +231,15 @@ public class RocksDBWindowStoreTest {
final KeyValue<Windowed<Integer>, String> five = windowedPair(5, "five", startTime + 5); final KeyValue<Windowed<Integer>, String> five = windowedPair(5, "five", startTime + 5);
assertEquals( assertEquals(
Utils.mkList(zero, one, two, four, five), Utils.mkList(zero, one, two, four, five),
StreamsTestUtils.toList(windowStore.all()) StreamsTestUtils.toList(windowStore.all())
); );
} }
@Test @Test
public void shouldFetchAllInTimeRange() { public void shouldFetchAllInTimeRange() {
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
long startTime = segmentSize - 4L; final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context); putFirstBatch(windowStore, startTime, context);
@ -247,25 +250,25 @@ public class RocksDBWindowStoreTest {
final KeyValue<Windowed<Integer>, String> five = windowedPair(5, "five", startTime + 5); final KeyValue<Windowed<Integer>, String> five = windowedPair(5, "five", startTime + 5);
assertEquals( assertEquals(
Utils.mkList(one, two, four), Utils.mkList(one, two, four),
StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 4)) StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 4))
); );
assertEquals( assertEquals(
Utils.mkList(zero, one, two), Utils.mkList(zero, one, two),
StreamsTestUtils.toList(windowStore.fetchAll(startTime + 0, startTime + 3)) StreamsTestUtils.toList(windowStore.fetchAll(startTime + 0, startTime + 3))
); );
assertEquals( assertEquals(
Utils.mkList(one, two, four, five), Utils.mkList(one, two, four, five),
StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 5)) StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 5))
); );
} }
@Test @Test
public void testFetchRange() { public void testFetchRange() {
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
long startTime = segmentSize - 4L; final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context); putFirstBatch(windowStore, startTime, context);
@ -293,7 +296,7 @@ public class RocksDBWindowStoreTest {
); );
assertEquals( assertEquals(
Utils.mkList(zero, one, two, Utils.mkList(zero, one, two,
four, five), four, five),
StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 0L - windowSize, startTime + 0L + windowSize + 5L)) StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 0L - windowSize, startTime + 0L + windowSize + 5L))
); );
assertEquals( assertEquals(
@ -313,7 +316,7 @@ public class RocksDBWindowStoreTest {
@Test @Test
public void testPutAndFetchBefore() { public void testPutAndFetchBefore() {
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
long startTime = segmentSize - 4L; final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context); putFirstBatch(windowStore, startTime, context);
@ -345,7 +348,7 @@ public class RocksDBWindowStoreTest {
// Flush the store and verify all current entries were properly flushed ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); 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("zero@0"), entriesByKey.get(0));
assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1)); assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
@ -359,7 +362,7 @@ public class RocksDBWindowStoreTest {
@Test @Test
public void testPutAndFetchAfter() { public void testPutAndFetchAfter() {
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
long startTime = segmentSize - 4L; final long startTime = segmentSize - 4L;
putFirstBatch(windowStore, startTime, context); putFirstBatch(windowStore, startTime, context);
@ -391,7 +394,7 @@ public class RocksDBWindowStoreTest {
// Flush the store and verify all current entries were properly flushed ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); 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("zero@0"), entriesByKey.get(0));
assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1)); assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
@ -405,7 +408,7 @@ public class RocksDBWindowStoreTest {
@Test @Test
public void testPutSameKeyTimestamp() { public void testPutSameKeyTimestamp() {
windowStore = createWindowStore(context, true); windowStore = createWindowStore(context, true);
long startTime = segmentSize - 4L; final long startTime = segmentSize - 4L;
context.setRecordContext(createRecordContext(startTime)); context.setRecordContext(createRecordContext(startTime));
windowStore.put(0, "zero"); windowStore.put(0, "zero");
@ -425,7 +428,7 @@ public class RocksDBWindowStoreTest {
// Flush the store and verify all current entries were properly flushed ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); 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)); assertEquals(Utils.mkSet("zero@0", "zero@0", "zero+@0", "zero++@0"), entriesByKey.get(0));
} }
@ -436,121 +439,156 @@ public class RocksDBWindowStoreTest {
// to validate segments // to validate segments
final Segments segments = new Segments(windowName, retentionPeriod, numSegments); final Segments segments = new Segments(windowName, retentionPeriod, numSegments);
long startTime = segmentSize * 2; final long startTime = segmentSize * 2;
long incr = segmentSize / 2; final long increment = segmentSize / 2;
context.setRecordContext(createRecordContext(startTime)); context.setRecordContext(createRecordContext(startTime));
windowStore.put(0, "zero"); windowStore.put(0, "zero");
assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir)); assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir));
context.setRecordContext(createRecordContext(startTime + incr)); context.setRecordContext(createRecordContext(startTime + increment));
windowStore.put(1, "one"); windowStore.put(1, "one");
assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir)); assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir));
context.setRecordContext(createRecordContext(startTime + incr * 2)); context.setRecordContext(createRecordContext(startTime + increment * 2));
windowStore.put(2, "two"); windowStore.put(2, "two");
assertEquals(Utils.mkSet(segments.segmentName(2), assertEquals(
segments.segmentName(3)), segmentDirs(baseDir)); 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"); windowStore.put(4, "four");
assertEquals(Utils.mkSet(segments.segmentName(2), assertEquals(
segments.segmentName(3), Utils.mkSet(
segments.segmentName(4)), segmentDirs(baseDir)); 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"); windowStore.put(5, "five");
assertEquals(Utils.mkSet(segments.segmentName(2), assertEquals(
segments.segmentName(3), Utils.mkSet(
segments.segmentName(4)), segmentDirs(baseDir)); 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("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("one"), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + 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 + incr * 3 - windowSize, startTime + incr * 3 + 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 + incr * 4 - windowSize, startTime + incr * 4 + 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 + incr * 5 - windowSize, startTime + incr * 5 + 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"); windowStore.put(6, "six");
assertEquals(Utils.mkSet(segments.segmentName(3), assertEquals(
segments.segmentName(4), Utils.mkSet(
segments.segmentName(5)), segmentDirs(baseDir)); 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(0, startTime - windowSize, startTime + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + 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 + incr * 3 - windowSize, startTime + incr * 3 + 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 + incr * 4 - windowSize, startTime + incr * 4 + 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 + incr * 5 - windowSize, startTime + incr * 5 + 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 + incr * 6 - windowSize, startTime + incr * 6 + 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"); windowStore.put(7, "seven");
assertEquals(Utils.mkSet(segments.segmentName(3), assertEquals(
segments.segmentName(4), Utils.mkSet(
segments.segmentName(5)), segmentDirs(baseDir)); 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(0, startTime - windowSize, startTime + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + 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 + incr * 3 - windowSize, startTime + incr * 3 + 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 + incr * 4 - windowSize, startTime + incr * 4 + 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 + incr * 5 - windowSize, startTime + incr * 5 + 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 + incr * 6 - windowSize, startTime + incr * 6 + 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 + incr * 7 - windowSize, startTime + incr * 7 + windowSize))); assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
context.setRecordContext(createRecordContext(startTime + incr * 8)); context.setRecordContext(createRecordContext(startTime + increment * 8));
windowStore.put(8, "eight"); windowStore.put(8, "eight");
assertEquals(Utils.mkSet(segments.segmentName(4), assertEquals(
segments.segmentName(5), Utils.mkSet(
segments.segmentName(6)), segmentDirs(baseDir)); 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(0, startTime - windowSize, startTime + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + 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 + incr * 4 - windowSize, startTime + incr * 4 + 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 + incr * 5 - windowSize, startTime + incr * 5 + 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 + incr * 6 - windowSize, startTime + incr * 6 + 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 + incr * 7 - windowSize, startTime + incr * 7 + 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 + incr * 8 - windowSize, startTime + incr * 8 + windowSize))); assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
// check segment directories // check segment directories
windowStore.flush(); windowStore.flush();
assertEquals(Utils.mkSet(segments.segmentName(4), assertEquals(
segments.segmentName(5), Utils.mkSet(
segments.segmentName(6)), segmentDirs(baseDir)); segments.segmentName(4),
segments.segmentName(5),
segments.segmentName(6)
),
segmentDirs(baseDir)
);
} }
@Test @Test
public void testRestore() throws IOException { public void testRestore() throws IOException {
long startTime = segmentSize * 2; final long startTime = segmentSize * 2;
long incr = segmentSize / 2; final long increment = segmentSize / 2;
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
context.setRecordContext(createRecordContext(startTime)); context.setRecordContext(createRecordContext(startTime));
windowStore.put(0, "zero"); windowStore.put(0, "zero");
context.setRecordContext(createRecordContext(startTime + incr)); context.setRecordContext(createRecordContext(startTime + increment));
windowStore.put(1, "one"); windowStore.put(1, "one");
context.setRecordContext(createRecordContext(startTime + incr * 2)); context.setRecordContext(createRecordContext(startTime + increment * 2));
windowStore.put(2, "two"); windowStore.put(2, "two");
context.setRecordContext(createRecordContext(startTime + incr * 3)); context.setRecordContext(createRecordContext(startTime + increment * 3));
windowStore.put(3, "three"); windowStore.put(3, "three");
context.setRecordContext(createRecordContext(startTime + incr * 4)); context.setRecordContext(createRecordContext(startTime + increment * 4));
windowStore.put(4, "four"); windowStore.put(4, "four");
context.setRecordContext(createRecordContext(startTime + incr * 5)); context.setRecordContext(createRecordContext(startTime + increment * 5));
windowStore.put(5, "five"); windowStore.put(5, "five");
context.setRecordContext(createRecordContext(startTime + incr * 6)); context.setRecordContext(createRecordContext(startTime + increment * 6));
windowStore.put(6, "six"); windowStore.put(6, "six");
context.setRecordContext(createRecordContext(startTime + incr * 7)); context.setRecordContext(createRecordContext(startTime + increment * 7));
windowStore.put(7, "seven"); windowStore.put(7, "seven");
context.setRecordContext(createRecordContext(startTime + incr * 8)); context.setRecordContext(createRecordContext(startTime + increment * 8));
windowStore.put(8, "eight"); windowStore.put(8, "eight");
windowStore.flush(); windowStore.flush();
@ -561,32 +599,32 @@ public class RocksDBWindowStoreTest {
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); 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(1, startTime + increment - windowSize, startTime + increment + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
context.restore(windowName, changeLog); context.restore(windowName, changeLog);
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); 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(1, startTime + increment - windowSize, startTime + increment + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + 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 + incr * 4 - windowSize, startTime + incr * 4 + 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 + incr * 5 - windowSize, startTime + incr * 5 + 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 + incr * 6 - windowSize, startTime + incr * 6 + 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 + incr * 7 - windowSize, startTime + incr * 7 + 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 + incr * 8 - windowSize, startTime + incr * 8 + windowSize))); assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
// check segment directories // check segment directories
windowStore.flush(); windowStore.flush();
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)), Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
} }
@ -597,23 +635,23 @@ public class RocksDBWindowStoreTest {
context.setRecordContext(createRecordContext(0)); context.setRecordContext(createRecordContext(0));
windowStore.put(0, "v"); windowStore.put(0, "v");
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(0L)), Utils.mkSet(segments.segmentName(0L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
context.setRecordContext(createRecordContext(59999)); context.setRecordContext(createRecordContext(59999));
windowStore.put(0, "v"); windowStore.put(0, "v");
windowStore.put(0, "v"); windowStore.put(0, "v");
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(0L)), Utils.mkSet(segments.segmentName(0L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
context.setRecordContext(createRecordContext(60000)); context.setRecordContext(createRecordContext(60000));
windowStore.put(0, "v"); windowStore.put(0, "v");
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)), Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
WindowStoreIterator iter; WindowStoreIterator iter;
@ -628,8 +666,8 @@ public class RocksDBWindowStoreTest {
assertEquals(4, fetchedCount); assertEquals(4, fetchedCount);
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)), Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
context.setRecordContext(createRecordContext(180000)); context.setRecordContext(createRecordContext(180000));
@ -644,8 +682,8 @@ public class RocksDBWindowStoreTest {
assertEquals(2, fetchedCount); assertEquals(2, fetchedCount);
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(1L), segments.segmentName(3L)), Utils.mkSet(segments.segmentName(1L), segments.segmentName(3L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
context.setRecordContext(createRecordContext(300000)); context.setRecordContext(createRecordContext(300000));
@ -660,15 +698,16 @@ public class RocksDBWindowStoreTest {
assertEquals(1, fetchedCount); assertEquals(1, fetchedCount);
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(3L), segments.segmentName(5L)), Utils.mkSet(segments.segmentName(3L), segments.segmentName(5L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
} }
@SuppressWarnings("ResultOfMethodCallIgnored")
@Test @Test
public void testInitialLoading() { public void testInitialLoading() {
File storeDir = new File(baseDir, windowName); final File storeDir = new File(baseDir, windowName);
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
@ -684,19 +723,19 @@ public class RocksDBWindowStoreTest {
windowStore = createWindowStore(context); windowStore = createWindowStore(context);
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)), Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
try (WindowStoreIterator iter = windowStore.fetch(0, 0L, 1000000L)) { try (final WindowStoreIterator iter = windowStore.fetch(0, 0L, 1000000L)) {
while (iter.hasNext()) { while (iter.hasNext()) {
iter.next(); iter.next();
} }
} }
assertEquals( assertEquals(
Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)), Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
segmentDirs(baseDir) segmentDirs(baseDir)
); );
} }
@ -712,16 +751,17 @@ public class RocksDBWindowStoreTest {
assertTrue(iterator.hasNext()); assertTrue(iterator.hasNext());
windowStore.close(); windowStore.close();
try { try {
//noinspection ResultOfMethodCallIgnored
iterator.hasNext(); iterator.hasNext();
fail("should have thrown InvalidStateStoreException on closed store"); fail("should have thrown InvalidStateStoreException on closed store");
} catch (InvalidStateStoreException e) { } catch (final InvalidStateStoreException e) {
// ok // ok
} }
try { try {
iterator.next(); iterator.next();
fail("should have thrown InvalidStateStoreException on closed store"); fail("should have thrown InvalidStateStoreException on closed store");
} catch (InvalidStateStoreException e) { } catch (final InvalidStateStoreException e) {
// ok // ok
} }
} }
@ -731,13 +771,14 @@ public class RocksDBWindowStoreTest {
final long windowSize = 0x7a00000000000000L; final long windowSize = 0x7a00000000000000L;
final long retentionPeriod = 0x7a00000000000000L; final long retentionPeriod = 0x7a00000000000000L;
final WindowStore<String, String> windowStore = Stores.windowStoreBuilder( final WindowStore<String, String> windowStore = Stores.windowStoreBuilder(
Stores.persistentWindowStore(windowName, Stores.persistentWindowStore(
retentionPeriod, windowName,
2, retentionPeriod,
windowSize, 2,
true), windowSize,
Serdes.String(), true),
Serdes.String()).build(); Serdes.String(),
Serdes.String()).build();
windowStore.init(context, windowStore); windowStore.init(context, windowStore);
@ -796,13 +837,13 @@ public class RocksDBWindowStoreTest {
} }
@Test @Test
public void shouldNoNullPointerWhenSerdeDoesntHandleNull() { public void shouldNoNullPointerWhenSerdeDoesNotHandleNull() {
windowStore = new RocksDBWindowStore<>( windowStore = new RocksDBWindowStore<>(
new RocksDBSegmentedBytesStore(windowName, retentionPeriod, numSegments, new WindowKeySchema()), new RocksDBSegmentedBytesStore(windowName, retentionPeriod, numSegments, new WindowKeySchema()),
Serdes.Integer(), Serdes.Integer(),
new SerdeThatDoesntHandleNull(), new SerdeThatDoesntHandleNull(),
false, false,
windowSize); windowSize);
windowStore.init(context, windowStore); windowStore.init(context, windowStore);
assertNull(windowStore.fetch(1, 0)); assertNull(windowStore.fetch(1, 0));
@ -811,13 +852,14 @@ public class RocksDBWindowStoreTest {
@Test @Test
public void shouldFetchAndIterateOverExactBinaryKeys() { public void shouldFetchAndIterateOverExactBinaryKeys() {
final WindowStore<Bytes, String> windowStore = Stores.windowStoreBuilder( final WindowStore<Bytes, String> windowStore = Stores.windowStoreBuilder(
Stores.persistentWindowStore(windowName, Stores.persistentWindowStore(
60000, windowName,
2, 60000,
60000, 2,
true), 60000,
Serdes.Bytes(), true),
Serdes.String()).build(); Serdes.Bytes(),
Serdes.String()).build();
windowStore.init(context, windowStore); windowStore.init(context, windowStore);
@ -842,7 +884,9 @@ public class RocksDBWindowStoreTest {
assertThat(toList(windowStore.fetch(key3, 0, Long.MAX_VALUE)), equalTo(expectedKey3)); 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)); context.setRecordContext(createRecordContext(startTime));
store.put(0, "zero"); store.put(0, "zero");
context.setRecordContext(createRecordContext(startTime + 1L)); context.setRecordContext(createRecordContext(startTime + 1L));
@ -855,7 +899,9 @@ public class RocksDBWindowStoreTest {
store.put(5, "five"); 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)); context.setRecordContext(createRecordContext(startTime + 3L));
store.put(2, "two+1"); store.put(2, "two+1");
context.setRecordContext(createRecordContext(startTime + 4L)); context.setRecordContext(createRecordContext(startTime + 4L));
@ -870,24 +916,25 @@ public class RocksDBWindowStoreTest {
store.put(2, "two+6"); store.put(2, "two+6");
} }
private <E> List<E> toList(WindowStoreIterator<E> iterator) { private <E> List<E> toList(final WindowStoreIterator<E> iterator) {
ArrayList<E> list = new ArrayList<>(); final ArrayList<E> list = new ArrayList<>();
while (iterator.hasNext()) { while (iterator.hasNext()) {
list.add(iterator.next().value); list.add(iterator.next().value);
} }
return list; return list;
} }
private Set<String> segmentDirs(File baseDir) { private Set<String> segmentDirs(final File baseDir) {
File windowDir = new File(baseDir, windowName); 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) { private Map<Integer, Set<String>> entriesByKey(final List<KeyValue<byte[], byte[]>> changeLog,
HashMap<Integer, Set<String>> entriesByKey = new HashMap<>(); @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 long timestamp = WindowKeySchema.extractStoreTimestamp(entry.key);
final Integer key = WindowKeySchema.extractStoreKey(entry.key, serdes); final Integer key = WindowKeySchema.extractStoreKey(entry.key, serdes);
@ -904,11 +951,11 @@ public class RocksDBWindowStoreTest {
return entriesByKey; 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); 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); return KeyValue.pair(new Windowed<>(key, WindowKeySchema.timeWindowForSize(timestamp, windowSize)), value);
} }
} }

View File

@ -40,29 +40,29 @@ public class StoreChangeLoggerTest {
private final Map<Integer, String> logged = new HashMap<>(); private final Map<Integer, String> logged = new HashMap<>();
private final InternalMockProcessorContext context = new InternalMockProcessorContext(StateSerdes.withBuiltinTypes(topic, Integer.class, String.class), private final InternalMockProcessorContext context = new InternalMockProcessorContext(StateSerdes.withBuiltinTypes(topic, Integer.class, String.class),
new RecordCollectorImpl(null, "StoreChangeLoggerTest", new LogContext("StoreChangeLoggerTest "), new DefaultProductionExceptionHandler()) { new RecordCollectorImpl(null, "StoreChangeLoggerTest", new LogContext("StoreChangeLoggerTest "), new DefaultProductionExceptionHandler()) {
@Override @Override
public <K1, V1> void send(final String topic, public <K1, V1> void send(final String topic,
final K1 key, final K1 key,
final V1 value, final V1 value,
final Integer partition, final Integer partition,
final Long timestamp, final Long timestamp,
final Serializer<K1> keySerializer, final Serializer<K1> keySerializer,
final Serializer<V1> valueSerializer) { final Serializer<V1> valueSerializer) {
logged.put((Integer) key, (String) value); 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)); private final StoreChangeLogger<Integer, String> changeLogger = new StoreChangeLogger<>(topic, context, StateSerdes.withBuiltinTypes(topic, Integer.class, String.class));

View File

@ -61,7 +61,6 @@ import static org.junit.Assert.assertEquals;
public class StreamThreadStateStoreProviderTest { public class StreamThreadStateStoreProviderTest {
private StreamTask taskOne; private StreamTask taskOne;
private StreamTask taskTwo;
private StreamThreadStateStoreProvider provider; private StreamThreadStateStoreProvider provider;
private StateDirectory stateDirectory; private StateDirectory stateDirectory;
private File stateDir; private File stateDir;
@ -71,19 +70,19 @@ public class StreamThreadStateStoreProviderTest {
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@Before @Before
public void before() throws IOException { public void before() {
final TopologyBuilder builder = new TopologyBuilder(); final TopologyBuilder builder = new TopologyBuilder();
builder.addSource("the-source", topicName); builder.addSource("the-source", topicName);
builder.addProcessor("the-processor", new MockProcessorSupplier(), "the-source"); builder.addProcessor("the-processor", new MockProcessorSupplier(), "the-source");
builder.addStateStore(Stores.create("kv-store") builder.addStateStore(Stores.create("kv-store")
.withStringKeys() .withStringKeys()
.withStringValues().inMemory().build(), "the-processor"); .withStringValues().inMemory().build(), "the-processor");
builder.addStateStore(Stores.create("window-store") builder.addStateStore(Stores.create("window-store")
.withStringKeys() .withStringKeys()
.withStringValues() .withStringValues()
.persistent() .persistent()
.windowed(10, 10, 2, false).build(), "the-processor"); .windowed(10, 10, 2, false).build(), "the-processor");
final Properties properties = new Properties(); final Properties properties = new Properties();
final String applicationId = "applicationId"; final String applicationId = "applicationId";
@ -99,18 +98,17 @@ public class StreamThreadStateStoreProviderTest {
builder.setApplicationId(applicationId); builder.setApplicationId(applicationId);
final ProcessorTopology topology = builder.build(null); final ProcessorTopology topology = builder.build(null);
tasks = new HashMap<>(); tasks = new HashMap<>();
stateDirectory = new StateDirectory(streamsConfig, new MockTime()); 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(); taskOne.initializeStateStores();
tasks.put(new TaskId(0, 0), tasks.put(new TaskId(0, 0), taskOne);
taskOne);
taskTwo = createStreamsTask(applicationId, streamsConfig, clientSupplier, topology, final StreamTask taskTwo = createStreamsTask(streamsConfig, clientSupplier, topology, new TaskId(0, 1));
new TaskId(0, 1));
taskTwo.initializeStateStores(); taskTwo.initializeStateStores();
tasks.put(new TaskId(0, 1), tasks.put(new TaskId(0, 1), taskTwo);
taskTwo);
threadMock = EasyMock.createNiceMock(StreamThread.class); threadMock = EasyMock.createNiceMock(StreamThread.class);
provider = new StreamThreadStateStoreProvider(threadMock); provider = new StreamThreadStateStoreProvider(threadMock);
@ -121,7 +119,7 @@ public class StreamThreadStateStoreProviderTest {
public void cleanUp() throws IOException { public void cleanUp() throws IOException {
Utils.delete(stateDir); Utils.delete(stateDir);
} }
@Test @Test
public void shouldFindKeyValueStores() { public void shouldFindKeyValueStores() {
mockThread(true); mockThread(true);
@ -164,8 +162,10 @@ public class StreamThreadStateStoreProviderTest {
@Test @Test
public void shouldReturnEmptyListIfStoreExistsButIsNotOfTypeValueStore() { public void shouldReturnEmptyListIfStoreExistsButIsNotOfTypeValueStore() {
mockThread(true); mockThread(true);
assertEquals(Collections.emptyList(), provider.stores("window-store", assertEquals(
QueryableStoreTypes.keyValueStore())); Collections.emptyList(),
provider.stores("window-store", QueryableStoreTypes.keyValueStore())
);
} }
@Test(expected = InvalidStateStoreException.class) @Test(expected = InvalidStateStoreException.class)
@ -174,8 +174,7 @@ public class StreamThreadStateStoreProviderTest {
provider.stores("kv-store", QueryableStoreTypes.keyValueStore()); provider.stores("kv-store", QueryableStoreTypes.keyValueStore());
} }
private StreamTask createStreamsTask(final String applicationId, private StreamTask createStreamsTask(final StreamsConfig streamsConfig,
final StreamsConfig streamsConfig,
final MockClientSupplier clientSupplier, final MockClientSupplier clientSupplier,
final ProcessorTopology topology, final ProcessorTopology topology,
final TaskId taskId) { final TaskId taskId) {
@ -190,8 +189,8 @@ public class StreamThreadStateStoreProviderTest {
stateDirectory, stateDirectory,
null, null,
new MockTime(), new MockTime(),
clientSupplier.getProducer(new HashMap<String, Object>())) { clientSupplier.getProducer(new HashMap<String, Object>())
) {
@Override @Override
protected void updateOffsetLimits() {} protected void updateOffsetLimits() {}
}; };
@ -205,28 +204,21 @@ public class StreamThreadStateStoreProviderTest {
private void configureRestoreConsumer(final MockClientSupplier clientSupplier, private void configureRestoreConsumer(final MockClientSupplier clientSupplier,
final String topic) { final String topic) {
clientSupplier.restoreConsumer final List<PartitionInfo> partitions = Arrays.asList(
.updatePartitions(topic, new PartitionInfo(topic, 0, null, null, null),
Arrays.asList( new PartitionInfo(topic, 1, null, null, null)
new PartitionInfo(topic, 0, null, );
null, null), clientSupplier.restoreConsumer.updatePartitions(topic, partitions);
new PartitionInfo(topic, 1, null,
null, null)));
final TopicPartition tp1 = new TopicPartition(topic, 0); final TopicPartition tp1 = new TopicPartition(topic, 0);
final TopicPartition tp2 = new TopicPartition(topic, 1); final TopicPartition tp2 = new TopicPartition(topic, 1);
clientSupplier.restoreConsumer clientSupplier.restoreConsumer.assign(Arrays.asList(tp1, tp2));
.assign(Arrays.asList(
tp1,
tp2));
final Map<TopicPartition, Long> offsets = new HashMap<>(); final Map<TopicPartition, Long> offsets = new HashMap<>();
offsets.put(tp1, 0L); offsets.put(tp1, 0L);
offsets.put(tp2, 0L); offsets.put(tp2, 0L);
clientSupplier.restoreConsumer clientSupplier.restoreConsumer.updateBeginningOffsets(offsets);
.updateBeginningOffsets(offsets); clientSupplier.restoreConsumer.updateEndOffsets(offsets);
clientSupplier.restoreConsumer
.updateEndOffsets(offsets);
} }
} }

View File

@ -45,7 +45,7 @@ import java.util.Set;
public class KStreamTestDriver extends ExternalResource { 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 ProcessorTopology topology;
private InternalMockProcessorContext context; private InternalMockProcessorContext context;
@ -231,7 +231,7 @@ public class KStreamTestDriver extends ExternalResource {
final List<ProcessorNode> nodes = topology.processors(); final List<ProcessorNode> nodes = topology.processors();
for (final ProcessorNode node: nodes) { for (final ProcessorNode node : nodes) {
names.add(node.name()); names.add(node.name());
} }
@ -241,7 +241,7 @@ public class KStreamTestDriver extends ExternalResource {
public ProcessorNode processor(final String name) { public ProcessorNode processor(final String name) {
final List<ProcessorNode> nodes = topology.processors(); final List<ProcessorNode> nodes = topology.processors();
for (final ProcessorNode node: nodes) { for (final ProcessorNode node : nodes) {
if (node.name().equals(name)) { if (node.name().equals(name)) {
return node; return node;
} }

View File

@ -32,7 +32,6 @@ import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.InternalTopologyAccessor;
import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.StreamsMetrics;
import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.Topology;
@ -157,26 +156,16 @@ public class ProcessorTopologyTestDriver {
private StreamTask task; private StreamTask task;
private GlobalStateUpdateTask globalStateTask; 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. * 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 * @param builder the topology builder that will be used to create the topology instance
*/ */
public ProcessorTopologyTestDriver(final StreamsConfig config, public ProcessorTopologyTestDriver(final StreamsConfig config,
final InternalTopologyBuilder builder) { final InternalTopologyBuilder builder) {
topology = builder.setApplicationId(APPLICATION_ID).build(null); topology = builder.setApplicationId(APPLICATION_ID).build(null);
final ProcessorTopology globalTopology = builder.buildGlobalStateTopology(); final ProcessorTopology globalTopology = builder.buildGlobalStateTopology();
// Set up the consumer and producer ... // Set up the consumer and producer ...
final Consumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); final Consumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
@ -218,36 +207,42 @@ public class ProcessorTopologyTestDriver {
globalPartitionsByTopic.put(topicName, partition); globalPartitionsByTopic.put(topicName, partition);
offsetsByTopicPartition.put(partition, new AtomicLong()); offsetsByTopicPartition.put(partition, new AtomicLong());
} }
final GlobalStateManagerImpl stateManager = new GlobalStateManagerImpl(new LogContext("mock "), final GlobalStateManagerImpl stateManager = new GlobalStateManagerImpl(
globalTopology, new LogContext("mock "),
globalConsumer, globalTopology,
stateDirectory, globalConsumer,
stateRestoreListener, stateDirectory,
config); stateRestoreListener,
config);
final GlobalProcessorContextImpl globalProcessorContext = new GlobalProcessorContextImpl(config, stateManager, streamsMetrics, cache); final GlobalProcessorContextImpl globalProcessorContext = new GlobalProcessorContextImpl(config, stateManager, streamsMetrics, cache);
stateManager.setGlobalProcessorContext(globalProcessorContext); stateManager.setGlobalProcessorContext(globalProcessorContext);
globalStateTask = new GlobalStateUpdateTask(globalTopology, globalStateTask = new GlobalStateUpdateTask(
globalProcessorContext, globalTopology,
stateManager, globalProcessorContext,
new LogAndContinueExceptionHandler(), stateManager,
new LogContext()); new LogAndContinueExceptionHandler(),
new LogContext());
globalStateTask.initialize(); globalStateTask.initialize();
} }
if (!partitionsByTopic.isEmpty()) { if (!partitionsByTopic.isEmpty()) {
task = new StreamTask(TASK_ID, task = new StreamTask(
partitionsByTopic.values(), TASK_ID,
topology, partitionsByTopic.values(),
consumer, topology,
new StoreChangelogReader( consumer,
createRestoreConsumer(topology.storeToChangelogTopic()), new StoreChangelogReader(
new MockStateRestoreListener(), createRestoreConsumer(topology.storeToChangelogTopic()),
new LogContext("topology-test-driver ")), new MockStateRestoreListener(),
config, new LogContext("topology-test-driver ")
streamsMetrics, stateDirectory, ),
cache, config,
new MockTime(), streamsMetrics,
producer); stateDirectory,
cache,
new MockTime(),
producer
);
task.initializeStateStores(); task.initializeStateStores();
task.initializeTopology(); 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. * 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 topicName the name of the topic on which the message is to be sent
* @param key the raw message key * @param key the raw message key
* @param value the raw message value * @param value the raw message value
* @param timestamp the raw message timestamp * @param timestamp the raw message timestamp
*/ */
public void process(final String topicName, 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. * 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 topicName the name of the topic on which the message is to be sent
* @param key the raw message key * @param key the raw message key
* @param value the raw message value * @param value the raw message value
*/ */
public void process(final String topicName, public void process(final String topicName,
final byte[] key, 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. * 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 topicName the name of the topic on which the message is to be sent
* @param key the raw message key * @param key the raw message key
* @param value the raw message value * @param value the raw message value
* @param keySerializer the serializer for the key * @param keySerializer the serializer for the key
* @param valueSerializer the serializer for the value * @param valueSerializer the serializer for the value
*/ */
public <K, V> void process(final String topicName, 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. * 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 topicName the name of the topic on which the message is to be sent
* @param key the raw message key * @param key the raw message key
* @param value the raw message value * @param value the raw message value
* @param keySerializer the serializer for the key * @param keySerializer the serializer for the key
* @param valueSerializer the serializer for the value * @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, public <K, V> void process(final String topicName,
final K key, 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 * 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[])}. * {@link #process(String, byte[], byte[])}.
* *
* @param topic the name of the topic * @param topic the name of the topic
* @param keyDeserializer the deserializer for the key type * @param keyDeserializer the deserializer for the key type
* @param valueDeserializer the deserializer for the value type * @param valueDeserializer the deserializer for the value type
* @return the next record on that topic, or null if there is no record available * @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 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(); final String topicName = storeAndTopic.getValue();
// Set up the restore-state topic ... // Set up the restore-state topic ...
// consumer.subscribe(new TopicPartition(topicName, 1)); // consumer.subscribe(new TopicPartition(topicName, 1));

View File

@ -39,7 +39,7 @@ public class StreamsTestUtils {
final String valueSerdeClassName, final String valueSerdeClassName,
final Properties additional) { final Properties additional) {
Properties streamsConfiguration = new Properties(); final Properties streamsConfiguration = new Properties();
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
streamsConfiguration.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000"); streamsConfiguration.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");

View File

@ -192,19 +192,22 @@ public class TopologyTestDriver implements Closeable {
* Initialized the internally mocked wall-clock time with {@link System#currentTimeMillis() current system time}. * Initialized the internally mocked wall-clock time with {@link System#currentTimeMillis() current system time}.
* *
* @param topology the topology to be tested * @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, public TopologyTestDriver(final Topology topology,
final Properties config) { final Properties config) {
this(topology, config, System.currentTimeMillis()); this(topology, config, System.currentTimeMillis());
} }
/** /**
* Create a new test diver instance. * Create a new test diver instance.
* *
* @param topology the topology to be tested * @param topology the topology to be tested
* @param config the configuration for the topology * @param config the configuration for the topology
* @param initialWallClockTimeMs the initial value of internally mocked wall-clock time * @param initialWallClockTimeMs the initial value of internally mocked wall-clock time
*/ */
@SuppressWarnings("WeakerAccess")
public TopologyTestDriver(final Topology topology, public TopologyTestDriver(final Topology topology,
final Properties config, final Properties config,
final long initialWallClockTimeMs) { final long initialWallClockTimeMs) {
@ -215,7 +218,7 @@ public class TopologyTestDriver implements Closeable {
internalTopologyBuilder.setApplicationId(streamsConfig.getString(StreamsConfig.APPLICATION_ID_CONFIG)); internalTopologyBuilder.setApplicationId(streamsConfig.getString(StreamsConfig.APPLICATION_ID_CONFIG));
processorTopology = internalTopologyBuilder.build(null); processorTopology = internalTopologyBuilder.build(null);
final ProcessorTopology globalTopology = internalTopologyBuilder.buildGlobalStateTopology(); final ProcessorTopology globalTopology = internalTopologyBuilder.buildGlobalStateTopology();
final Serializer<byte[]> bytesSerializer = new ByteArraySerializer(); final Serializer<byte[]> bytesSerializer = new ByteArraySerializer();
producer = new MockProducer<byte[], byte[]>(true, bytesSerializer, bytesSerializer) { producer = new MockProducer<byte[], byte[]>(true, bytesSerializer, bytesSerializer) {
@ -237,13 +240,13 @@ public class TopologyTestDriver implements Closeable {
streamsMetrics); streamsMetrics);
final StateRestoreListener stateRestoreListener = new StateRestoreListener() { final StateRestoreListener stateRestoreListener = new StateRestoreListener() {
@Override @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 @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 @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()) { for (final InternalTopologyBuilder.TopicsInfo topicsInfo : internalTopologyBuilder.topicGroups().values()) {
@ -321,6 +324,7 @@ public class TopologyTestDriver implements Closeable {
* *
* @param consumerRecord the record to be processed * @param consumerRecord the record to be processed
*/ */
@SuppressWarnings("WeakerAccess")
public void pipeInput(final ConsumerRecord<byte[], byte[]> consumerRecord) { public void pipeInput(final ConsumerRecord<byte[], byte[]> consumerRecord) {
final String topicName = consumerRecord.topic(); 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. * Send input messages to the topology and then commit each message individually.
* *
* @param records a list of records to be processed * @param records a list of records to be processed
*/ */
@SuppressWarnings("WeakerAccess")
public void pipeInput(final List<ConsumerRecord<byte[], byte[]>> records) { public void pipeInput(final List<ConsumerRecord<byte[], byte[]>> records) {
for (final ConsumerRecord<byte[], byte[]> record : records) { for (final ConsumerRecord<byte[], byte[]> record : records) {
pipeInput(record); pipeInput(record);
@ -418,6 +424,7 @@ public class TopologyTestDriver implements Closeable {
* *
* @param advanceMs the amount of time to advance wall-clock time in milliseconds * @param advanceMs the amount of time to advance wall-clock time in milliseconds
*/ */
@SuppressWarnings("WeakerAccess")
public void advanceWallClockTime(final long advanceMs) { public void advanceWallClockTime(final long advanceMs) {
mockTime.sleep(advanceMs); mockTime.sleep(advanceMs);
if (task != null) { if (task != null) {
@ -434,6 +441,7 @@ public class TopologyTestDriver implements Closeable {
* @param topic the name of the topic * @param topic the name of the topic
* @return the next record on that topic, or {@code null} if there is no record available * @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) { public ProducerRecord<byte[], byte[]> readOutput(final String topic) {
final Queue<ProducerRecord<byte[], byte[]>> outputRecords = outputRecordsByTopic.get(topic); final Queue<ProducerRecord<byte[], byte[]>> outputRecords = outputRecordsByTopic.get(topic);
if (outputRecords == null) { if (outputRecords == null) {
@ -446,11 +454,12 @@ public class TopologyTestDriver implements Closeable {
* Read the next record from the given topic. * Read the next record from the given topic.
* These records were output by the topology during the previous calls to {@link #pipeInput(ConsumerRecord)}. * These records were output by the topology during the previous calls to {@link #pipeInput(ConsumerRecord)}.
* *
* @param topic the name of the topic * @param topic the name of the topic
* @param keyDeserializer the deserializer for the key type * @param keyDeserializer the deserializer for the key type
* @param valueDeserializer the deserializer for the value 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 * @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, public <K, V> ProducerRecord<K, V> readOutput(final String topic,
final Deserializer<K> keyDeserializer, final Deserializer<K> keyDeserializer,
final Deserializer<V> valueDeserializer) { final Deserializer<V> valueDeserializer) {
@ -476,6 +485,7 @@ public class TopologyTestDriver implements Closeable {
* @see #getWindowStore(String) * @see #getWindowStore(String)
* @see #getSessionStore(String) * @see #getSessionStore(String)
*/ */
@SuppressWarnings("WeakerAccess")
public Map<String, StateStore> getAllStateStores() { public Map<String, StateStore> getAllStateStores() {
final Map<String, StateStore> allStores = new HashMap<>(); final Map<String, StateStore> allStores = new HashMap<>();
for (final String storeName : internalTopologyBuilder.allStateStoreName()) { for (final String storeName : internalTopologyBuilder.allStateStoreName()) {
@ -521,7 +531,7 @@ public class TopologyTestDriver implements Closeable {
* @see #getWindowStore(String) * @see #getWindowStore(String)
* @see #getSessionStore(String) * @see #getSessionStore(String)
*/ */
@SuppressWarnings("unchecked") @SuppressWarnings({"unchecked", "WeakerAccess"})
public <K, V> KeyValueStore<K, V> getKeyValueStore(final String name) { public <K, V> KeyValueStore<K, V> getKeyValueStore(final String name) {
final StateStore store = getStateStore(name); final StateStore store = getStateStore(name);
return store instanceof KeyValueStore ? (KeyValueStore<K, V>) store : null; return store instanceof KeyValueStore ? (KeyValueStore<K, V>) store : null;
@ -541,7 +551,7 @@ public class TopologyTestDriver implements Closeable {
* @see #getKeyValueStore(String) * @see #getKeyValueStore(String)
* @see #getSessionStore(String) (String) * @see #getSessionStore(String) (String)
*/ */
@SuppressWarnings("unchecked") @SuppressWarnings({"unchecked", "WeakerAccess", "unused"})
public <K, V> WindowStore<K, V> getWindowStore(final String name) { public <K, V> WindowStore<K, V> getWindowStore(final String name) {
final StateStore store = getStateStore(name); final StateStore store = getStateStore(name);
return store instanceof WindowStore ? (WindowStore<K, V>) store : null; return store instanceof WindowStore ? (WindowStore<K, V>) store : null;
@ -561,7 +571,7 @@ public class TopologyTestDriver implements Closeable {
* @see #getKeyValueStore(String) * @see #getKeyValueStore(String)
* @see #getWindowStore(String) * @see #getWindowStore(String)
*/ */
@SuppressWarnings("unchecked") @SuppressWarnings({"unchecked", "WeakerAccess", "unused"})
public <K, V> SessionStore<K, V> getSessionStore(final String name) { public <K, V> SessionStore<K, V> getSessionStore(final String name) {
final StateStore store = getStateStore(name); final StateStore store = getStateStore(name);
return store instanceof SessionStore ? (SessionStore<K, V>) store : null; return store instanceof SessionStore ? (SessionStore<K, V>) store : null;
@ -634,7 +644,7 @@ public class TopologyTestDriver implements Closeable {
}; };
// for each store // 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(); final String topicName = storeAndTopic.getValue();
// Set up the restore-state topic ... // Set up the restore-state topic ...
// consumer.subscribe(new TopicPartition(topicName, 0)); // consumer.subscribe(new TopicPartition(topicName, 0));

View File

@ -31,6 +31,7 @@ import org.apache.kafka.streams.processor.internals.StreamsMetricsImpl;
import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore;
import java.io.File; import java.io.File;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
@ -45,7 +46,7 @@ import java.util.Properties;
* tests that serve as example usage. * tests that serve as example usage.
* <p> * <p>
* Note that this class does not take any automated actions (such as firing scheduled punctuators). * 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 * 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}. * {@link Topology} and using the {@link TopologyTestDriver}.
*/ */
@ -96,8 +97,9 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
return punctuator; return punctuator;
} }
@SuppressWarnings("WeakerAccess")
public void cancel() { public void cancel() {
this.cancelled = true; cancelled = true;
} }
public boolean cancelled() { public boolean cancelled() {
@ -122,7 +124,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
/** /**
* The child this data was forwarded to. * 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() { public String childName() {
return 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}. * 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.taskId = taskId;
this.config = streamsConfig; this.config = streamsConfig;
this.stateDir = stateDir; 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 @Override
@ -379,15 +381,15 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
return capturedPunctuators; return capturedPunctuators;
} }
@SuppressWarnings("unchecked")
@Override @Override
public <K, V> void forward(final K key, final V value) { public <K, V> void forward(final K key, final V value) {
//noinspection unchecked
capturedForwards.add(new CapturedForward(To.all(), new KeyValue(key, value))); capturedForwards.add(new CapturedForward(To.all(), new KeyValue(key, value)));
} }
@SuppressWarnings("unchecked")
@Override @Override
public <K, V> void forward(final K key, final V value, final To to) { 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))); capturedForwards.add(new CapturedForward(to, new KeyValue(key, value)));
} }