mirror of https://github.com/apache/kafka.git
MINOR: revert KIP-770 (#12383)
KIP-770 introduced a performance regression and needs some re-design. Needed to resolve some conflict while reverting. This reverts commits1317f3f77a
and0924fd3f9f
. Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Guozhang Wang <guozhang@confluent.io>
This commit is contained in:
parent
a521bbd755
commit
38b08dfd33
|
@ -188,7 +188,7 @@
|
||||||
files="StreamsMetricsImpl.java"/>
|
files="StreamsMetricsImpl.java"/>
|
||||||
|
|
||||||
<suppress checks="NPathComplexity"
|
<suppress checks="NPathComplexity"
|
||||||
files="(GlobalStateManagerImpl|KafkaStreams|KStreamImplJoin|StreamsPartitionAssignor|StreamThread|TaskManager|TopologyConfig).java"/>
|
files="(KafkaStreams|StreamsPartitionAssignor|StreamThread|TaskManager|GlobalStateManagerImpl|KStreamImplJoin).java"/>
|
||||||
|
|
||||||
<suppress checks="(FinalLocalVariable|UnnecessaryParentheses|BooleanExpressionComplexity|CyclomaticComplexity|WhitespaceAfter|LocalVariableName)"
|
<suppress checks="(FinalLocalVariable|UnnecessaryParentheses|BooleanExpressionComplexity|CyclomaticComplexity|WhitespaceAfter|LocalVariableName)"
|
||||||
files="Murmur3.java"/>
|
files="Murmur3.java"/>
|
||||||
|
|
|
@ -179,7 +179,7 @@ public class PageViewTypedDemo {
|
||||||
props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class);
|
props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class);
|
||||||
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, JSONSerde.class);
|
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, JSONSerde.class);
|
||||||
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, JSONSerde.class);
|
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, JSONSerde.class);
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
|
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
|
||||||
|
|
||||||
// setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data
|
// setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data
|
||||||
|
|
|
@ -60,7 +60,7 @@ public class PageViewUntypedDemo {
|
||||||
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped");
|
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped");
|
||||||
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
||||||
props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class);
|
props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class);
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
|
|
||||||
// setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data
|
// setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data
|
||||||
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
|
|
|
@ -77,7 +77,7 @@ public class TemperatureDemo {
|
||||||
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
|
|
||||||
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
|
|
||||||
final Duration duration24Hours = Duration.ofHours(24);
|
final Duration duration24Hours = Duration.ofHours(24);
|
||||||
|
|
||||||
|
|
|
@ -61,7 +61,7 @@ public final class WordCountDemo {
|
||||||
}
|
}
|
||||||
props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount");
|
props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount");
|
||||||
props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
||||||
props.putIfAbsent(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.putIfAbsent(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
|
|
||||||
|
|
|
@ -108,7 +108,7 @@ public final class WordCountProcessorDemo {
|
||||||
|
|
||||||
props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-processor");
|
props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-processor");
|
||||||
props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
||||||
props.putIfAbsent(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.putIfAbsent(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
|
|
||||||
|
|
|
@ -131,7 +131,7 @@ public final class WordCountTransformerDemo {
|
||||||
}
|
}
|
||||||
props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-transformer");
|
props.putIfAbsent(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-transformer");
|
||||||
props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
props.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
||||||
props.putIfAbsent(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.putIfAbsent(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
props.putIfAbsent(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
props.putIfAbsent(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
|
|
||||||
|
|
|
@ -109,7 +109,6 @@ import static org.apache.kafka.streams.StreamsConfig.METRICS_RECORDING_LEVEL_CON
|
||||||
import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
|
import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
|
||||||
import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix;
|
import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix;
|
||||||
import static org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration;
|
import static org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration;
|
||||||
import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
|
|
||||||
import static org.apache.kafka.streams.processor.internals.ClientUtils.fetchEndOffsets;
|
import static org.apache.kafka.streams.processor.internals.ClientUtils.fetchEndOffsets;
|
||||||
import static org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
|
import static org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
|
||||||
|
|
||||||
|
@ -176,7 +175,6 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
protected final Admin adminClient;
|
protected final Admin adminClient;
|
||||||
private final StreamsMetricsImpl streamsMetrics;
|
private final StreamsMetricsImpl streamsMetrics;
|
||||||
private final long totalCacheSize;
|
private final long totalCacheSize;
|
||||||
private final long inputBufferMaxBytes;
|
|
||||||
private final StreamStateListener streamStateListener;
|
private final StreamStateListener streamStateListener;
|
||||||
private final StateRestoreListener delegatingStateRestoreListener;
|
private final StateRestoreListener delegatingStateRestoreListener;
|
||||||
private final Map<Long, StreamThread.State> threadState;
|
private final Map<Long, StreamThread.State> threadState;
|
||||||
|
@ -941,9 +939,9 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
streamsUncaughtExceptionHandler = this::defaultStreamsUncaughtExceptionHandler;
|
streamsUncaughtExceptionHandler = this::defaultStreamsUncaughtExceptionHandler;
|
||||||
delegatingStateRestoreListener = new DelegatingStateRestoreListener();
|
delegatingStateRestoreListener = new DelegatingStateRestoreListener();
|
||||||
|
|
||||||
totalCacheSize = getTotalCacheSize(applicationConfigs);
|
totalCacheSize = applicationConfigs.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG);
|
||||||
inputBufferMaxBytes = applicationConfigs.getLong(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG);
|
|
||||||
final int numStreamThreads = topologyMetadata.getNumStreamThreads(applicationConfigs);
|
final int numStreamThreads = topologyMetadata.getNumStreamThreads(applicationConfigs);
|
||||||
|
final long cacheSizePerThread = getCacheSizePerThread(numStreamThreads);
|
||||||
|
|
||||||
GlobalStreamThread.State globalThreadState = null;
|
GlobalStreamThread.State globalThreadState = null;
|
||||||
if (hasGlobalTopology) {
|
if (hasGlobalTopology) {
|
||||||
|
@ -953,7 +951,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
applicationConfigs,
|
applicationConfigs,
|
||||||
clientSupplier.getGlobalConsumer(applicationConfigs.getGlobalConsumerConfigs(clientId)),
|
clientSupplier.getGlobalConsumer(applicationConfigs.getGlobalConsumerConfigs(clientId)),
|
||||||
stateDirectory,
|
stateDirectory,
|
||||||
0L,
|
cacheSizePerThread,
|
||||||
streamsMetrics,
|
streamsMetrics,
|
||||||
time,
|
time,
|
||||||
globalThreadId,
|
globalThreadId,
|
||||||
|
@ -974,20 +972,14 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
|
|
||||||
queryableStoreProvider = new QueryableStoreProvider(globalStateStoreProvider);
|
queryableStoreProvider = new QueryableStoreProvider(globalStateStoreProvider);
|
||||||
for (int i = 1; i <= numStreamThreads; i++) {
|
for (int i = 1; i <= numStreamThreads; i++) {
|
||||||
createAndAddStreamThread(0L, 0L, i);
|
createAndAddStreamThread(cacheSizePerThread, i);
|
||||||
}
|
|
||||||
// Initially, all Stream Threads are created with 0 cache size and max buffer size and then resized here.
|
|
||||||
resizeThreadCacheAndBufferMemory(numStreamThreads);
|
|
||||||
if (numStreamThreads > 0) {
|
|
||||||
log.info("Initializing {} StreamThread with cache size/max buffer size values as {} per thread.",
|
|
||||||
numStreamThreads, getThreadCacheAndBufferMemoryString());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
stateDirCleaner = setupStateDirCleaner();
|
stateDirCleaner = setupStateDirCleaner();
|
||||||
rocksDBMetricsRecordingService = maybeCreateRocksDBMetricsRecordingService(clientId, applicationConfigs);
|
rocksDBMetricsRecordingService = maybeCreateRocksDBMetricsRecordingService(clientId, applicationConfigs);
|
||||||
}
|
}
|
||||||
|
|
||||||
private StreamThread createAndAddStreamThread(final long cacheSizePerThread, final long maxBufferSizePerThread, final int threadIdx) {
|
private StreamThread createAndAddStreamThread(final long cacheSizePerThread, final int threadIdx) {
|
||||||
final StreamThread streamThread = StreamThread.create(
|
final StreamThread streamThread = StreamThread.create(
|
||||||
topologyMetadata,
|
topologyMetadata,
|
||||||
applicationConfigs,
|
applicationConfigs,
|
||||||
|
@ -999,7 +991,6 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
time,
|
time,
|
||||||
streamsMetadataState,
|
streamsMetadataState,
|
||||||
cacheSizePerThread,
|
cacheSizePerThread,
|
||||||
maxBufferSizePerThread,
|
|
||||||
stateDirectory,
|
stateDirectory,
|
||||||
delegatingStateRestoreListener,
|
delegatingStateRestoreListener,
|
||||||
threadIdx,
|
threadIdx,
|
||||||
|
@ -1036,7 +1027,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
* Since the number of stream threads increases, the sizes of the caches in the new stream thread
|
* Since the number of stream threads increases, the sizes of the caches in the new stream thread
|
||||||
* and the existing stream threads are adapted so that the sum of the cache sizes over all stream
|
* and the existing stream threads are adapted so that the sum of the cache sizes over all stream
|
||||||
* threads does not exceed the total cache size specified in configuration
|
* threads does not exceed the total cache size specified in configuration
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG}.
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
|
||||||
* <p>
|
* <p>
|
||||||
* Stream threads can only be added if this Kafka Streams client is in state RUNNING or REBALANCING.
|
* Stream threads can only be added if this Kafka Streams client is in state RUNNING or REBALANCING.
|
||||||
*
|
*
|
||||||
|
@ -1047,15 +1038,14 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
final StreamThread streamThread;
|
final StreamThread streamThread;
|
||||||
synchronized (changeThreadCount) {
|
synchronized (changeThreadCount) {
|
||||||
final int threadIdx = getNextThreadIndex();
|
final int threadIdx = getNextThreadIndex();
|
||||||
|
final int numLiveThreads = getNumLiveStreamThreads();
|
||||||
|
final long cacheSizePerThread = getCacheSizePerThread(numLiveThreads + 1);
|
||||||
|
log.info("Adding StreamThread-{}, there will now be {} live threads and the new cache size per thread is {}",
|
||||||
|
threadIdx, numLiveThreads + 1, cacheSizePerThread);
|
||||||
|
resizeThreadCache(cacheSizePerThread);
|
||||||
// Creating thread should hold the lock in order to avoid duplicate thread index.
|
// Creating thread should hold the lock in order to avoid duplicate thread index.
|
||||||
// If the duplicate index happen, the metadata of thread may be duplicate too.
|
// If the duplicate index happen, the metadata of thread may be duplicate too.
|
||||||
// Also, we create the new thread with initial values of cache size and max buffer size as 0
|
streamThread = createAndAddStreamThread(cacheSizePerThread, threadIdx);
|
||||||
// and then resize them later
|
|
||||||
streamThread = createAndAddStreamThread(0L, 0L, threadIdx);
|
|
||||||
final int numLiveThreads = getNumLiveStreamThreads();
|
|
||||||
resizeThreadCacheAndBufferMemory(numLiveThreads);
|
|
||||||
log.info("Adding StreamThread-{}, there are now {} threads with cache size/max buffer size values as {} per thread.",
|
|
||||||
threadIdx, numLiveThreads, getThreadCacheAndBufferMemoryString());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized (stateLock) {
|
synchronized (stateLock) {
|
||||||
|
@ -1066,9 +1056,9 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
log.warn("Terminating the new thread because the Kafka Streams client is in state {}", state);
|
log.warn("Terminating the new thread because the Kafka Streams client is in state {}", state);
|
||||||
streamThread.shutdown();
|
streamThread.shutdown();
|
||||||
threads.remove(streamThread);
|
threads.remove(streamThread);
|
||||||
resizeThreadCacheAndBufferMemory(getNumLiveStreamThreads());
|
final long cacheSizePerThread = getCacheSizePerThread(getNumLiveStreamThreads());
|
||||||
log.info("Resizing thread cache and max buffer size per thread since new thread can not be " +
|
log.info("Resizing thread cache due to terminating added thread, new cache size per thread is {}", cacheSizePerThread);
|
||||||
"started, cache size/max buffer size per thread is {}", getThreadCacheAndBufferMemoryString());
|
resizeThreadCache(cacheSizePerThread);
|
||||||
return Optional.empty();
|
return Optional.empty();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1086,7 +1076,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
* <p>
|
* <p>
|
||||||
* Since the number of stream threads decreases, the sizes of the caches in the remaining stream
|
* Since the number of stream threads decreases, the sizes of the caches in the remaining stream
|
||||||
* threads are adapted so that the sum of the cache sizes over all stream threads equals the total
|
* threads are adapted so that the sum of the cache sizes over all stream threads equals the total
|
||||||
* cache size specified in configuration {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG}.
|
* cache size specified in configuration {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
|
||||||
*
|
*
|
||||||
* @return name of the removed stream thread or empty if a stream thread could not be removed because
|
* @return name of the removed stream thread or empty if a stream thread could not be removed because
|
||||||
* no stream threads are alive
|
* no stream threads are alive
|
||||||
|
@ -1101,10 +1091,9 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
* The removed stream thread is gracefully shut down. This method does not specify which stream
|
* The removed stream thread is gracefully shut down. This method does not specify which stream
|
||||||
* thread is shut down.
|
* thread is shut down.
|
||||||
* <p>
|
* <p>
|
||||||
* Since the number of stream threads decreases, the sizes of the caches and buffer bytes in the remaining stream
|
* Since the number of stream threads decreases, the sizes of the caches in the remaining stream
|
||||||
* threads are adapted so that the sum of the cache sizes and buffer bytes over all stream threads equals the total
|
* threads are adapted so that the sum of the cache sizes over all stream threads equals the total
|
||||||
* cache size specified in configuration {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG} and
|
* cache size specified in configuration {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
|
||||||
* {@link StreamsConfig#INPUT_BUFFER_MAX_BYTES_CONFIG} respectively.
|
|
||||||
*
|
*
|
||||||
* @param timeout The length of time to wait for the thread to shutdown
|
* @param timeout The length of time to wait for the thread to shutdown
|
||||||
* @throws org.apache.kafka.common.errors.TimeoutException if the thread does not stop in time
|
* @throws org.apache.kafka.common.errors.TimeoutException if the thread does not stop in time
|
||||||
|
@ -1144,15 +1133,16 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
log.info("{} is the last remaining thread and must remove itself, therefore we cannot wait "
|
log.info("{} is the last remaining thread and must remove itself, therefore we cannot wait "
|
||||||
+ "for it to complete shutdown as this will result in deadlock.", streamThread.getName());
|
+ "for it to complete shutdown as this will result in deadlock.", streamThread.getName());
|
||||||
}
|
}
|
||||||
resizeThreadCacheAndBufferMemory(getNumLiveStreamThreads());
|
|
||||||
log.info("Resizing thread cache/max buffer size due to removal of thread {}, " +
|
final long cacheSizePerThread = getCacheSizePerThread(getNumLiveStreamThreads());
|
||||||
"new cache size/max buffer size per thread is {}", streamThread.getName(), getThreadCacheAndBufferMemoryString());
|
log.info("Resizing thread cache due to thread removal, new cache size per thread is {}", cacheSizePerThread);
|
||||||
|
resizeThreadCache(cacheSizePerThread);
|
||||||
if (groupInstanceID.isPresent() && callingThreadIsNotCurrentStreamThread) {
|
if (groupInstanceID.isPresent() && callingThreadIsNotCurrentStreamThread) {
|
||||||
final MemberToRemove memberToRemove = new MemberToRemove(groupInstanceID.get());
|
final MemberToRemove memberToRemove = new MemberToRemove(groupInstanceID.get());
|
||||||
final Collection<MemberToRemove> membersToRemove = Collections.singletonList(memberToRemove);
|
final Collection<MemberToRemove> membersToRemove = Collections.singletonList(memberToRemove);
|
||||||
final RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroupResult =
|
final RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroupResult =
|
||||||
adminClient.removeMembersFromConsumerGroup(
|
adminClient.removeMembersFromConsumerGroup(
|
||||||
applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG),
|
applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG),
|
||||||
new RemoveMembersFromConsumerGroupOptions(membersToRemove)
|
new RemoveMembersFromConsumerGroupOptions(membersToRemove)
|
||||||
|
@ -1251,22 +1241,15 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getThreadCacheAndBufferMemoryString() {
|
private long getCacheSizePerThread(final int numStreamThreads) {
|
||||||
final StreamThread streamThread = threads.get(0);
|
if (numStreamThreads == 0) {
|
||||||
return streamThread.getCacheSize() + "/" + streamThread.getMaxBufferSize();
|
return totalCacheSize;
|
||||||
|
}
|
||||||
|
return totalCacheSize / (numStreamThreads + (topologyMetadata.hasGlobalTopology() ? 1 : 0));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void resizeThreadCacheAndBufferMemory(final int numStreamThreads) {
|
private void resizeThreadCache(final long cacheSizePerThread) {
|
||||||
final long cacheSizePerThread;
|
processStreamThread(thread -> thread.resizeCache(cacheSizePerThread));
|
||||||
final long inputBufferMaxBytesPerThread;
|
|
||||||
if (numStreamThreads == 0) {
|
|
||||||
cacheSizePerThread = totalCacheSize;
|
|
||||||
inputBufferMaxBytesPerThread = inputBufferMaxBytes;
|
|
||||||
} else {
|
|
||||||
cacheSizePerThread = totalCacheSize / (numStreamThreads + (topologyMetadata.hasGlobalTopology() ? 1 : 0));
|
|
||||||
inputBufferMaxBytesPerThread = inputBufferMaxBytes / (numStreamThreads + (topologyMetadata.hasGlobalTopology() ? 1 : 0));
|
|
||||||
}
|
|
||||||
processStreamThread(thread -> thread.resizeCacheAndBufferMemory(cacheSizePerThread, inputBufferMaxBytesPerThread));
|
|
||||||
if (globalStreamThread != null) {
|
if (globalStreamThread != null) {
|
||||||
globalStreamThread.resize(cacheSizePerThread);
|
globalStreamThread.resize(cacheSizePerThread);
|
||||||
}
|
}
|
||||||
|
|
|
@ -419,30 +419,18 @@ public class StreamsConfig extends AbstractConfig {
|
||||||
|
|
||||||
/** {@code buffered.records.per.partition} */
|
/** {@code buffered.records.per.partition} */
|
||||||
@SuppressWarnings("WeakerAccess")
|
@SuppressWarnings("WeakerAccess")
|
||||||
@Deprecated
|
|
||||||
public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition";
|
public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition";
|
||||||
public static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "Maximum number of records to buffer per partition.";
|
public static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "Maximum number of records to buffer per partition.";
|
||||||
|
|
||||||
/** {@code input.buffer.max.bytes} */
|
|
||||||
@SuppressWarnings("WeakerAccess")
|
|
||||||
public static final String INPUT_BUFFER_MAX_BYTES_CONFIG = "input.buffer.max.bytes";
|
|
||||||
public static final String INPUT_BUFFER_MAX_BYTES_DOC = "Maximum bytes of records to buffer across all threads";
|
|
||||||
|
|
||||||
/** {@code built.in.metrics.version} */
|
/** {@code built.in.metrics.version} */
|
||||||
public static final String BUILT_IN_METRICS_VERSION_CONFIG = "built.in.metrics.version";
|
public static final String BUILT_IN_METRICS_VERSION_CONFIG = "built.in.metrics.version";
|
||||||
private static final String BUILT_IN_METRICS_VERSION_DOC = "Version of the built-in metrics to use.";
|
private static final String BUILT_IN_METRICS_VERSION_DOC = "Version of the built-in metrics to use.";
|
||||||
|
|
||||||
/** {@code cache.max.bytes.buffering} */
|
/** {@code cache.max.bytes.buffering} */
|
||||||
@SuppressWarnings("WeakerAccess")
|
@SuppressWarnings("WeakerAccess")
|
||||||
@Deprecated
|
|
||||||
public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering";
|
public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering";
|
||||||
public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads";
|
public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads";
|
||||||
|
|
||||||
/** {@statestore.cache.max.bytes} */
|
|
||||||
@SuppressWarnings("WeakerAccess")
|
|
||||||
public static final String STATESTORE_CACHE_MAX_BYTES_CONFIG = "statestore.cache.max.bytes";
|
|
||||||
public static final String STATESTORE_CACHE_MAX_BYTES_DOC = "Maximum number of memory bytes to be used for statestore cache across all threads";
|
|
||||||
|
|
||||||
/** {@code client.id} */
|
/** {@code client.id} */
|
||||||
@SuppressWarnings("WeakerAccess")
|
@SuppressWarnings("WeakerAccess")
|
||||||
public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
|
public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
|
||||||
|
@ -755,12 +743,6 @@ public class StreamsConfig extends AbstractConfig {
|
||||||
atLeast(0),
|
atLeast(0),
|
||||||
Importance.MEDIUM,
|
Importance.MEDIUM,
|
||||||
CACHE_MAX_BYTES_BUFFERING_DOC)
|
CACHE_MAX_BYTES_BUFFERING_DOC)
|
||||||
.define(STATESTORE_CACHE_MAX_BYTES_CONFIG,
|
|
||||||
Type.LONG,
|
|
||||||
10 * 1024 * 1024L,
|
|
||||||
atLeast(0),
|
|
||||||
Importance.MEDIUM,
|
|
||||||
STATESTORE_CACHE_MAX_BYTES_DOC)
|
|
||||||
.define(CLIENT_ID_CONFIG,
|
.define(CLIENT_ID_CONFIG,
|
||||||
Type.STRING,
|
Type.STRING,
|
||||||
"",
|
"",
|
||||||
|
@ -862,11 +844,6 @@ public class StreamsConfig extends AbstractConfig {
|
||||||
in(NO_OPTIMIZATION, OPTIMIZE),
|
in(NO_OPTIMIZATION, OPTIMIZE),
|
||||||
Importance.MEDIUM,
|
Importance.MEDIUM,
|
||||||
TOPOLOGY_OPTIMIZATION_DOC)
|
TOPOLOGY_OPTIMIZATION_DOC)
|
||||||
.define(INPUT_BUFFER_MAX_BYTES_CONFIG,
|
|
||||||
Type.LONG,
|
|
||||||
512 * 1024 * 1024,
|
|
||||||
Importance.MEDIUM,
|
|
||||||
INPUT_BUFFER_MAX_BYTES_DOC)
|
|
||||||
|
|
||||||
// LOW
|
// LOW
|
||||||
|
|
||||||
|
|
|
@ -32,10 +32,9 @@ import java.util.Properties;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
|
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG;
|
|
||||||
import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG;
|
import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_DOC;
|
import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_DOC;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG;
|
import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_DOC;
|
import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_DOC;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG;
|
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC;
|
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC;
|
||||||
|
@ -49,7 +48,6 @@ import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_CONFIG;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC;
|
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB;
|
import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY;
|
import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY;
|
||||||
import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Streams configs that apply at the topology level. The values in the {@link StreamsConfig} parameter of the
|
* Streams configs that apply at the topology level. The values in the {@link StreamsConfig} parameter of the
|
||||||
|
@ -57,21 +55,15 @@ import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCach
|
||||||
* determine the defaults, which can then be overridden for specific topologies by passing them in when creating the
|
* determine the defaults, which can then be overridden for specific topologies by passing them in when creating the
|
||||||
* topology builders via the {@link org.apache.kafka.streams.StreamsBuilder()} method.
|
* topology builders via the {@link org.apache.kafka.streams.StreamsBuilder()} method.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
public class TopologyConfig extends AbstractConfig {
|
public class TopologyConfig extends AbstractConfig {
|
||||||
private static final ConfigDef CONFIG;
|
private static final ConfigDef CONFIG;
|
||||||
static {
|
static {
|
||||||
CONFIG = new ConfigDef()
|
CONFIG = new ConfigDef()
|
||||||
.define(BUFFERED_RECORDS_PER_PARTITION_CONFIG,
|
.define(BUFFERED_RECORDS_PER_PARTITION_CONFIG,
|
||||||
Type.INT,
|
Type.INT,
|
||||||
null,
|
|
||||||
Importance.LOW,
|
|
||||||
BUFFERED_RECORDS_PER_PARTITION_DOC)
|
|
||||||
.define(STATESTORE_CACHE_MAX_BYTES_CONFIG,
|
|
||||||
Type.LONG,
|
|
||||||
null,
|
null,
|
||||||
Importance.MEDIUM,
|
Importance.LOW,
|
||||||
CACHE_MAX_BYTES_BUFFERING_DOC)
|
BUFFERED_RECORDS_PER_PARTITION_DOC)
|
||||||
.define(CACHE_MAX_BYTES_BUFFERING_CONFIG,
|
.define(CACHE_MAX_BYTES_BUFFERING_CONFIG,
|
||||||
Type.LONG,
|
Type.LONG,
|
||||||
null,
|
null,
|
||||||
|
@ -137,49 +129,14 @@ public class TopologyConfig extends AbstractConfig {
|
||||||
maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG);
|
maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG);
|
||||||
log.info("Topology {} is overriding {} to {}", topologyName, BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize);
|
log.info("Topology {} is overriding {} to {}", topologyName, BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize);
|
||||||
} else {
|
} else {
|
||||||
// If the user hasn't explicitly set the buffered.records.per.partition config, then leave it unbounded
|
maxBufferedSize = globalAppConfigs.getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG);
|
||||||
// and rely on the input.buffer.max.bytes instead to keep the memory usage under control
|
|
||||||
maxBufferedSize = globalAppConfigs.originals().containsKey(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)
|
|
||||||
? globalAppConfigs.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG) : -1;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final boolean stateStoreCacheMaxBytesOverridden = isTopologyOverride(STATESTORE_CACHE_MAX_BYTES_CONFIG, topologyOverrides);
|
if (isTopologyOverride(CACHE_MAX_BYTES_BUFFERING_CONFIG, topologyOverrides)) {
|
||||||
final boolean cacheMaxBytesBufferingOverridden = isTopologyOverride(CACHE_MAX_BYTES_BUFFERING_CONFIG, topologyOverrides);
|
cacheSize = getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG);
|
||||||
|
log.info("Topology {} is overriding {} to {}", topologyName, CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSize);
|
||||||
if (!stateStoreCacheMaxBytesOverridden && !cacheMaxBytesBufferingOverridden) {
|
|
||||||
cacheSize = getTotalCacheSize(globalAppConfigs);
|
|
||||||
} else {
|
} else {
|
||||||
if (stateStoreCacheMaxBytesOverridden && cacheMaxBytesBufferingOverridden) {
|
cacheSize = globalAppConfigs.getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG);
|
||||||
cacheSize = getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
|
|
||||||
log.info("Topology {} is using both deprecated config {} and new config {}, hence {} is ignored and the new config {} (value {}) is used",
|
|
||||||
topologyName,
|
|
||||||
CACHE_MAX_BYTES_BUFFERING_CONFIG,
|
|
||||||
STATESTORE_CACHE_MAX_BYTES_CONFIG,
|
|
||||||
CACHE_MAX_BYTES_BUFFERING_CONFIG,
|
|
||||||
STATESTORE_CACHE_MAX_BYTES_CONFIG,
|
|
||||||
cacheSize);
|
|
||||||
} else if (cacheMaxBytesBufferingOverridden) {
|
|
||||||
cacheSize = getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG);
|
|
||||||
log.info("Topology {} is using only deprecated config {}, and will be used to set cache size to {}; " +
|
|
||||||
"we suggest setting the new config {} instead as deprecated {} would be removed in the future.",
|
|
||||||
topologyName,
|
|
||||||
CACHE_MAX_BYTES_BUFFERING_CONFIG,
|
|
||||||
cacheSize,
|
|
||||||
STATESTORE_CACHE_MAX_BYTES_CONFIG,
|
|
||||||
CACHE_MAX_BYTES_BUFFERING_CONFIG);
|
|
||||||
} else {
|
|
||||||
cacheSize = getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cacheSize != 0) {
|
|
||||||
log.warn("Topology {} is overriding cache size to {} but this will not have any effect as the "
|
|
||||||
+ "topology-level cache size config only controls whether record buffering is enabled "
|
|
||||||
+ "or disabled, thus the only valid override value is 0",
|
|
||||||
topologyName, cacheSize);
|
|
||||||
} else {
|
|
||||||
log.info("Topology {} is overriding cache size to {}, record buffering will be disabled",
|
|
||||||
topologyName, cacheSize);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (isTopologyOverride(MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) {
|
if (isTopologyOverride(MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) {
|
||||||
|
|
|
@ -17,16 +17,9 @@
|
||||||
package org.apache.kafka.streams.internals;
|
package org.apache.kafka.streams.internals;
|
||||||
|
|
||||||
import org.apache.kafka.streams.StreamsConfig;
|
import org.apache.kafka.streams.StreamsConfig;
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG;
|
|
||||||
import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG;
|
|
||||||
|
|
||||||
public class StreamsConfigUtils {
|
public class StreamsConfigUtils {
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(StreamsConfigUtils.class);
|
|
||||||
|
|
||||||
public enum ProcessingMode {
|
public enum ProcessingMode {
|
||||||
AT_LEAST_ONCE("AT_LEAST_ONCE"),
|
AT_LEAST_ONCE("AT_LEAST_ONCE"),
|
||||||
|
|
||||||
|
@ -73,28 +66,4 @@ public class StreamsConfigUtils {
|
||||||
return processingMode == ProcessingMode.EXACTLY_ONCE_ALPHA ||
|
return processingMode == ProcessingMode.EXACTLY_ONCE_ALPHA ||
|
||||||
processingMode == ProcessingMode.EXACTLY_ONCE_V2;
|
processingMode == ProcessingMode.EXACTLY_ONCE_V2;
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
public static long getTotalCacheSize(final StreamsConfig config) {
|
|
||||||
// both deprecated and new config set. Warn and use the new one.
|
|
||||||
if (config.originals().containsKey(CACHE_MAX_BYTES_BUFFERING_CONFIG) && config.originals().containsKey(STATESTORE_CACHE_MAX_BYTES_CONFIG)) {
|
|
||||||
if (!config.getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG).equals(config.getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG))) {
|
|
||||||
LOG.warn("Both deprecated config {} and the new config {} are set, hence {} is ignored and {} is used instead.",
|
|
||||||
CACHE_MAX_BYTES_BUFFERING_CONFIG,
|
|
||||||
STATESTORE_CACHE_MAX_BYTES_CONFIG,
|
|
||||||
CACHE_MAX_BYTES_BUFFERING_CONFIG,
|
|
||||||
STATESTORE_CACHE_MAX_BYTES_CONFIG);
|
|
||||||
}
|
|
||||||
return config.getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
|
|
||||||
} else if (config.originals().containsKey(CACHE_MAX_BYTES_BUFFERING_CONFIG)) {
|
|
||||||
// only deprecated config set.
|
|
||||||
LOG.warn("Deprecated config {} is set, and will be used; we suggest setting the new config {} instead as deprecated {} would be removed in the future.",
|
|
||||||
CACHE_MAX_BYTES_BUFFERING_CONFIG,
|
|
||||||
STATESTORE_CACHE_MAX_BYTES_CONFIG,
|
|
||||||
CACHE_MAX_BYTES_BUFFERING_CONFIG);
|
|
||||||
return config.getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG);
|
|
||||||
}
|
|
||||||
// only new or no config set. Use default or user specified value.
|
|
||||||
return config.getLong(STATESTORE_CACHE_MAX_BYTES_CONFIG);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -78,7 +78,7 @@ public interface CogroupedKStream<K, VOut> {
|
||||||
* same key.
|
* same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -128,7 +128,7 @@ public interface CogroupedKStream<K, VOut> {
|
||||||
* same key.
|
* same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -179,7 +179,7 @@ public interface CogroupedKStream<K, VOut> {
|
||||||
* same key.
|
* same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -232,7 +232,7 @@ public interface CogroupedKStream<K, VOut> {
|
||||||
* same key.
|
* same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link org.apache.kafka.streams.state.ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link org.apache.kafka.streams.state.ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
|
|
@ -53,7 +53,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by
|
||||||
|
@ -81,7 +81,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by
|
||||||
|
@ -112,7 +112,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -158,7 +158,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -211,7 +211,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
*
|
*
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -262,7 +262,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -326,7 +326,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -385,7 +385,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
*
|
*
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -431,7 +431,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -490,7 +490,7 @@ public interface KGroupedStream<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
|
|
@ -52,7 +52,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -95,7 +95,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -138,7 +138,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -167,7 +167,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -223,7 +223,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -296,7 +296,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -368,7 +368,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -434,7 +434,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -518,7 +518,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via
|
||||||
|
@ -604,7 +604,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
* The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
|
* The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
|
||||||
|
@ -674,7 +674,7 @@ public interface KGroupedTable<K, V> {
|
||||||
* the same key.
|
* the same key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
* The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
|
* The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
|
||||||
|
|
|
@ -77,7 +77,7 @@ public interface SessionWindowedCogroupedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -122,7 +122,7 @@ public interface SessionWindowedCogroupedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct
|
* The rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -166,7 +166,7 @@ public interface SessionWindowedCogroupedKStream<K, V> {
|
||||||
* the same window and key if caching is enabled on the {@link Materialized} instance.
|
* the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link SessionStore} it must be obtained via
|
* To query the local {@link SessionStore} it must be obtained via
|
||||||
|
@ -226,7 +226,7 @@ public interface SessionWindowedCogroupedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link SessionStore} it must be obtained via
|
* To query the local {@link SessionStore} it must be obtained via
|
||||||
|
|
|
@ -65,7 +65,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* the same session and key.
|
* the same session and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -95,7 +95,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* the same session and key.
|
* the same session and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -126,7 +126,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link SessionStore} it must be obtained via
|
* To query the local {@link SessionStore} it must be obtained via
|
||||||
|
@ -172,7 +172,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link SessionStore} it must be obtained via
|
* To query the local {@link SessionStore} it must be obtained via
|
||||||
|
@ -233,7 +233,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -282,7 +282,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct
|
* The rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -330,7 +330,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* the same window and key if caching is enabled on the {@link Materialized} instance.
|
* the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link SessionStore} it must be obtained via
|
* To query the local {@link SessionStore} it must be obtained via
|
||||||
|
@ -391,7 +391,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link SessionStore} it must be obtained via
|
* To query the local {@link SessionStore} it must be obtained via
|
||||||
|
@ -459,7 +459,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -504,7 +504,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
|
||||||
|
@ -549,7 +549,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link SessionStore} it must be obtained via
|
* To query the local {@link SessionStore} it must be obtained via
|
||||||
|
@ -609,7 +609,7 @@ public interface SessionWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link SessionStore} it must be obtained via
|
* To query the local {@link SessionStore} it must be obtained via
|
||||||
|
|
|
@ -75,7 +75,7 @@ public interface TimeWindowedCogroupedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
||||||
|
@ -115,7 +115,7 @@ public interface TimeWindowedCogroupedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct
|
* The rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
||||||
|
@ -156,7 +156,7 @@ public interface TimeWindowedCogroupedKStream<K, V> {
|
||||||
* the same window and key if caching is enabled on the {@link Materialized} instance.
|
* the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
||||||
|
@ -213,7 +213,7 @@ public interface TimeWindowedCogroupedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
||||||
|
|
|
@ -65,7 +65,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
||||||
|
@ -95,7 +95,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
||||||
|
@ -126,7 +126,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
||||||
|
@ -175,7 +175,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
||||||
|
@ -236,7 +236,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
||||||
|
@ -281,7 +281,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct
|
* The rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
||||||
|
@ -326,7 +326,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* the same window and key if caching is enabled on the {@link Materialized} instance.
|
* the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
||||||
|
@ -387,7 +387,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
||||||
|
@ -457,7 +457,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
||||||
|
@ -502,7 +502,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* the same window and key.
|
* the same window and key.
|
||||||
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
* The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
|
||||||
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
* parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
* For failure and recovery the store (which always will be of type {@link TimestampedWindowStore}) will be backed by
|
||||||
|
@ -547,7 +547,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
||||||
|
@ -610,7 +610,7 @@ public interface TimeWindowedKStream<K, V> {
|
||||||
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
* to the same window and key if caching is enabled on the {@link Materialized} instance.
|
||||||
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
* When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct
|
||||||
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
* keys, the number of parallel running Kafka Streams instances, and the {@link StreamsConfig configuration}
|
||||||
* parameters for {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and
|
* parameters for {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
|
||||||
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
* {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}.
|
||||||
* <p>
|
* <p>
|
||||||
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
* To query the local {@link ReadOnlyWindowStore} it must be obtained via
|
||||||
|
|
|
@ -58,18 +58,16 @@ import java.util.function.Function;
|
||||||
*/
|
*/
|
||||||
public class PartitionGroup {
|
public class PartitionGroup {
|
||||||
|
|
||||||
private final Logger logger;
|
private final Logger logger;
|
||||||
private final Map<TopicPartition, RecordQueue> partitionQueues;
|
private final Map<TopicPartition, RecordQueue> partitionQueues;
|
||||||
private final Function<TopicPartition, OptionalLong> lagProvider;
|
private final Function<TopicPartition, OptionalLong> lagProvider;
|
||||||
private final Sensor enforcedProcessingSensor;
|
private final Sensor enforcedProcessingSensor;
|
||||||
private final long maxTaskIdleMs;
|
private final long maxTaskIdleMs;
|
||||||
private final Sensor recordLatenessSensor;
|
private final Sensor recordLatenessSensor;
|
||||||
private final Sensor totalInputBufferBytesSensor;
|
|
||||||
private final PriorityQueue<RecordQueue> nonEmptyQueuesByTime;
|
private final PriorityQueue<RecordQueue> nonEmptyQueuesByTime;
|
||||||
|
|
||||||
private long streamTime;
|
private long streamTime;
|
||||||
private int totalBuffered;
|
private int totalBuffered;
|
||||||
private long totalBytesBuffered;
|
|
||||||
private boolean allBuffered;
|
private boolean allBuffered;
|
||||||
private final Map<TopicPartition, Long> idlePartitionDeadlines = new HashMap<>();
|
private final Map<TopicPartition, Long> idlePartitionDeadlines = new HashMap<>();
|
||||||
|
|
||||||
|
@ -93,7 +91,6 @@ public class PartitionGroup {
|
||||||
final Map<TopicPartition, RecordQueue> partitionQueues,
|
final Map<TopicPartition, RecordQueue> partitionQueues,
|
||||||
final Function<TopicPartition, OptionalLong> lagProvider,
|
final Function<TopicPartition, OptionalLong> lagProvider,
|
||||||
final Sensor recordLatenessSensor,
|
final Sensor recordLatenessSensor,
|
||||||
final Sensor totalInputBufferBytesSensor,
|
|
||||||
final Sensor enforcedProcessingSensor,
|
final Sensor enforcedProcessingSensor,
|
||||||
final long maxTaskIdleMs) {
|
final long maxTaskIdleMs) {
|
||||||
this.logger = logContext.logger(PartitionGroup.class);
|
this.logger = logContext.logger(PartitionGroup.class);
|
||||||
|
@ -103,7 +100,6 @@ public class PartitionGroup {
|
||||||
this.enforcedProcessingSensor = enforcedProcessingSensor;
|
this.enforcedProcessingSensor = enforcedProcessingSensor;
|
||||||
this.maxTaskIdleMs = maxTaskIdleMs;
|
this.maxTaskIdleMs = maxTaskIdleMs;
|
||||||
this.recordLatenessSensor = recordLatenessSensor;
|
this.recordLatenessSensor = recordLatenessSensor;
|
||||||
this.totalInputBufferBytesSensor = totalInputBufferBytesSensor;
|
|
||||||
totalBuffered = 0;
|
totalBuffered = 0;
|
||||||
allBuffered = false;
|
allBuffered = false;
|
||||||
streamTime = RecordQueue.UNKNOWN;
|
streamTime = RecordQueue.UNKNOWN;
|
||||||
|
@ -229,8 +225,6 @@ public class PartitionGroup {
|
||||||
if (!newInputPartitions.contains(topicPartition)) {
|
if (!newInputPartitions.contains(topicPartition)) {
|
||||||
// if partition is removed should delete its queue
|
// if partition is removed should delete its queue
|
||||||
totalBuffered -= queueEntry.getValue().size();
|
totalBuffered -= queueEntry.getValue().size();
|
||||||
totalBytesBuffered -= queueEntry.getValue().getTotalBytesBuffered();
|
|
||||||
totalInputBufferBytesSensor.record(totalBytesBuffered);
|
|
||||||
queuesIterator.remove();
|
queuesIterator.remove();
|
||||||
removedPartitions.add(topicPartition);
|
removedPartitions.add(topicPartition);
|
||||||
}
|
}
|
||||||
|
@ -266,17 +260,12 @@ public class PartitionGroup {
|
||||||
info.queue = queue;
|
info.queue = queue;
|
||||||
|
|
||||||
if (queue != null) {
|
if (queue != null) {
|
||||||
// get the buffer size of queue before poll
|
|
||||||
final long oldBufferSize = queue.getTotalBytesBuffered();
|
|
||||||
// get the first record from this queue.
|
// get the first record from this queue.
|
||||||
record = queue.poll(wallClockTime);
|
record = queue.poll(wallClockTime);
|
||||||
// After polling, the buffer size would have reduced.
|
|
||||||
final long newBufferSize = queue.getTotalBytesBuffered();
|
|
||||||
|
|
||||||
if (record != null) {
|
if (record != null) {
|
||||||
--totalBuffered;
|
--totalBuffered;
|
||||||
totalBytesBuffered -= oldBufferSize - newBufferSize;
|
|
||||||
totalInputBufferBytesSensor.record(totalBytesBuffered);
|
|
||||||
if (queue.isEmpty()) {
|
if (queue.isEmpty()) {
|
||||||
// if a certain queue has been drained, reset the flag
|
// if a certain queue has been drained, reset the flag
|
||||||
allBuffered = false;
|
allBuffered = false;
|
||||||
|
@ -300,8 +289,8 @@ public class PartitionGroup {
|
||||||
/**
|
/**
|
||||||
* Adds raw records to this partition group
|
* Adds raw records to this partition group
|
||||||
*
|
*
|
||||||
* @param partition the partition
|
* @param partition the partition
|
||||||
* @param rawRecords the raw records
|
* @param rawRecords the raw records
|
||||||
* @return the queue size for the partition
|
* @return the queue size for the partition
|
||||||
*/
|
*/
|
||||||
int addRawRecords(final TopicPartition partition, final Iterable<ConsumerRecord<byte[], byte[]>> rawRecords) {
|
int addRawRecords(final TopicPartition partition, final Iterable<ConsumerRecord<byte[], byte[]>> rawRecords) {
|
||||||
|
@ -312,9 +301,7 @@ public class PartitionGroup {
|
||||||
}
|
}
|
||||||
|
|
||||||
final int oldSize = recordQueue.size();
|
final int oldSize = recordQueue.size();
|
||||||
final long oldBufferSize = recordQueue.getTotalBytesBuffered();
|
|
||||||
final int newSize = recordQueue.addRawRecords(rawRecords);
|
final int newSize = recordQueue.addRawRecords(rawRecords);
|
||||||
final long newBufferSize = recordQueue.getTotalBytesBuffered();
|
|
||||||
|
|
||||||
// add this record queue to be considered for processing in the future if it was empty before
|
// add this record queue to be considered for processing in the future if it was empty before
|
||||||
if (oldSize == 0 && newSize > 0) {
|
if (oldSize == 0 && newSize > 0) {
|
||||||
|
@ -329,8 +316,7 @@ public class PartitionGroup {
|
||||||
}
|
}
|
||||||
|
|
||||||
totalBuffered += newSize - oldSize;
|
totalBuffered += newSize - oldSize;
|
||||||
totalBytesBuffered += newBufferSize - oldBufferSize;
|
|
||||||
totalInputBufferBytesSensor.record(totalBytesBuffered);
|
|
||||||
return newSize;
|
return newSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -368,20 +354,12 @@ public class PartitionGroup {
|
||||||
return recordQueue.size();
|
return recordQueue.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
Set<TopicPartition> getNonEmptyTopicPartitions() {
|
|
||||||
final Set<TopicPartition> nonEmptyTopicPartitions = new HashSet<>();
|
|
||||||
for (final RecordQueue recordQueue : nonEmptyQueuesByTime) {
|
|
||||||
nonEmptyTopicPartitions.add(recordQueue.partition());
|
|
||||||
}
|
|
||||||
return nonEmptyTopicPartitions;
|
|
||||||
}
|
|
||||||
|
|
||||||
int numBuffered() {
|
int numBuffered() {
|
||||||
return totalBuffered;
|
return totalBuffered;
|
||||||
}
|
}
|
||||||
|
|
||||||
long totalBytesBuffered() {
|
boolean allPartitionsBufferedLocally() {
|
||||||
return totalBytesBuffered;
|
return allBuffered;
|
||||||
}
|
}
|
||||||
|
|
||||||
void clear() {
|
void clear() {
|
||||||
|
@ -398,10 +376,4 @@ public class PartitionGroup {
|
||||||
queue.close();
|
queue.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Below methods are for only testing.
|
|
||||||
|
|
||||||
boolean allPartitionsBufferedLocally() {
|
|
||||||
return allBuffered;
|
|
||||||
}
|
|
||||||
}
|
}
|
|
@ -55,7 +55,6 @@ public class RecordQueue {
|
||||||
|
|
||||||
private final Sensor droppedRecordsSensor;
|
private final Sensor droppedRecordsSensor;
|
||||||
private final Sensor consumedSensor;
|
private final Sensor consumedSensor;
|
||||||
private long totalBytesBuffered;
|
|
||||||
private long headRecordSizeInBytes;
|
private long headRecordSizeInBytes;
|
||||||
|
|
||||||
RecordQueue(final TopicPartition partition,
|
RecordQueue(final TopicPartition partition,
|
||||||
|
@ -90,7 +89,6 @@ public class RecordQueue {
|
||||||
droppedRecordsSensor
|
droppedRecordsSensor
|
||||||
);
|
);
|
||||||
this.log = logContext.logger(RecordQueue.class);
|
this.log = logContext.logger(RecordQueue.class);
|
||||||
this.totalBytesBuffered = 0L;
|
|
||||||
this.headRecordSizeInBytes = 0L;
|
this.headRecordSizeInBytes = 0L;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -125,7 +123,6 @@ public class RecordQueue {
|
||||||
int addRawRecords(final Iterable<ConsumerRecord<byte[], byte[]>> rawRecords) {
|
int addRawRecords(final Iterable<ConsumerRecord<byte[], byte[]>> rawRecords) {
|
||||||
for (final ConsumerRecord<byte[], byte[]> rawRecord : rawRecords) {
|
for (final ConsumerRecord<byte[], byte[]> rawRecord : rawRecords) {
|
||||||
fifoQueue.addLast(rawRecord);
|
fifoQueue.addLast(rawRecord);
|
||||||
this.totalBytesBuffered += consumerRecordSizeInBytes(rawRecord);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
updateHead();
|
updateHead();
|
||||||
|
@ -143,7 +140,6 @@ public class RecordQueue {
|
||||||
|
|
||||||
consumedSensor.record(headRecordSizeInBytes, wallClockTime);
|
consumedSensor.record(headRecordSizeInBytes, wallClockTime);
|
||||||
|
|
||||||
totalBytesBuffered -= headRecordSizeInBytes;
|
|
||||||
headRecord = null;
|
headRecord = null;
|
||||||
headRecordSizeInBytes = 0L;
|
headRecordSizeInBytes = 0L;
|
||||||
partitionTime = Math.max(partitionTime, recordToReturn.timestamp);
|
partitionTime = Math.max(partitionTime, recordToReturn.timestamp);
|
||||||
|
@ -251,11 +247,4 @@ public class RecordQueue {
|
||||||
long partitionTime() {
|
long partitionTime() {
|
||||||
return partitionTime;
|
return partitionTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the total bytes buffered for this particular RecordQueue
|
|
||||||
*/
|
|
||||||
long getTotalBytesBuffered() {
|
|
||||||
return totalBytesBuffered;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -176,14 +176,15 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator,
|
||||||
|
|
||||||
recordInfo = new PartitionGroup.RecordInfo();
|
recordInfo = new PartitionGroup.RecordInfo();
|
||||||
|
|
||||||
|
final Sensor enforcedProcessingSensor;
|
||||||
|
enforcedProcessingSensor = TaskMetrics.enforcedProcessingSensor(threadId, taskId, streamsMetrics);
|
||||||
final long maxTaskIdleMs = config.maxTaskIdleMs;
|
final long maxTaskIdleMs = config.maxTaskIdleMs;
|
||||||
partitionGroup = new PartitionGroup(
|
partitionGroup = new PartitionGroup(
|
||||||
logContext,
|
logContext,
|
||||||
createPartitionQueues(),
|
createPartitionQueues(),
|
||||||
mainConsumer::currentLag,
|
mainConsumer::currentLag,
|
||||||
TaskMetrics.recordLatenessSensor(threadId, taskId, streamsMetrics),
|
TaskMetrics.recordLatenessSensor(threadId, taskId, streamsMetrics),
|
||||||
TaskMetrics.totalInputBufferBytesSensor(threadId, taskId, streamsMetrics),
|
enforcedProcessingSensor,
|
||||||
TaskMetrics.enforcedProcessingSensor(threadId, taskId, streamsMetrics),
|
|
||||||
maxTaskIdleMs
|
maxTaskIdleMs
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -729,8 +730,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator,
|
||||||
|
|
||||||
// after processing this record, if its partition queue's buffered size has been
|
// after processing this record, if its partition queue's buffered size has been
|
||||||
// decreased to the threshold, we can then resume the consumption on this partition
|
// decreased to the threshold, we can then resume the consumption on this partition
|
||||||
// TODO maxBufferedSize != -1 would be removed once the deprecated config buffered.records.per.partition is removed
|
if (recordInfo.queue().size() == maxBufferedSize) {
|
||||||
if (maxBufferedSize != -1 && recordInfo.queue().size() == maxBufferedSize) {
|
|
||||||
mainConsumer.resume(singleton(partition));
|
mainConsumer.resume(singleton(partition));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -990,8 +990,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator,
|
||||||
|
|
||||||
// if after adding these records, its partition queue's buffered size has been
|
// if after adding these records, its partition queue's buffered size has been
|
||||||
// increased beyond the threshold, we can then pause the consumption for this partition
|
// increased beyond the threshold, we can then pause the consumption for this partition
|
||||||
// We do this only if the deprecated config buffered.records.per.partition is set
|
if (newQueueSize > maxBufferedSize) {
|
||||||
if (maxBufferedSize != -1 && newQueueSize > maxBufferedSize) {
|
|
||||||
mainConsumer.pause(singleton(partition));
|
mainConsumer.pause(singleton(partition));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1244,14 +1243,6 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator,
|
||||||
return recordCollector;
|
return recordCollector;
|
||||||
}
|
}
|
||||||
|
|
||||||
Set<TopicPartition> getNonEmptyTopicPartitions() {
|
|
||||||
return this.partitionGroup.getNonEmptyTopicPartitions();
|
|
||||||
}
|
|
||||||
|
|
||||||
long totalBytesBuffered() {
|
|
||||||
return partitionGroup.totalBytesBuffered();
|
|
||||||
}
|
|
||||||
|
|
||||||
// below are visible for testing only
|
// below are visible for testing only
|
||||||
int numBuffered() {
|
int numBuffered() {
|
||||||
return partitionGroup.numBuffered();
|
return partitionGroup.numBuffered();
|
||||||
|
|
|
@ -318,7 +318,6 @@ public class StreamThread extends Thread {
|
||||||
// These are used to signal from outside the stream thread, but the variables themselves are internal to the thread
|
// These are used to signal from outside the stream thread, but the variables themselves are internal to the thread
|
||||||
private final AtomicLong cacheResizeSize = new AtomicLong(-1L);
|
private final AtomicLong cacheResizeSize = new AtomicLong(-1L);
|
||||||
private final AtomicBoolean leaveGroupRequested = new AtomicBoolean(false);
|
private final AtomicBoolean leaveGroupRequested = new AtomicBoolean(false);
|
||||||
private final AtomicLong maxBufferSizeBytes = new AtomicLong(-1L);
|
|
||||||
private final boolean eosEnabled;
|
private final boolean eosEnabled;
|
||||||
|
|
||||||
public static StreamThread create(final TopologyMetadata topologyMetadata,
|
public static StreamThread create(final TopologyMetadata topologyMetadata,
|
||||||
|
@ -331,7 +330,6 @@ public class StreamThread extends Thread {
|
||||||
final Time time,
|
final Time time,
|
||||||
final StreamsMetadataState streamsMetadataState,
|
final StreamsMetadataState streamsMetadataState,
|
||||||
final long cacheSizeBytes,
|
final long cacheSizeBytes,
|
||||||
final long maxBufferSizeBytes,
|
|
||||||
final StateDirectory stateDirectory,
|
final StateDirectory stateDirectory,
|
||||||
final StateRestoreListener userStateRestoreListener,
|
final StateRestoreListener userStateRestoreListener,
|
||||||
final int threadIdx,
|
final int threadIdx,
|
||||||
|
@ -433,8 +431,7 @@ public class StreamThread extends Thread {
|
||||||
referenceContainer.nonFatalExceptionsToHandle,
|
referenceContainer.nonFatalExceptionsToHandle,
|
||||||
shutdownErrorHook,
|
shutdownErrorHook,
|
||||||
streamsUncaughtExceptionHandler,
|
streamsUncaughtExceptionHandler,
|
||||||
cache::resize,
|
cache::resize
|
||||||
maxBufferSizeBytes
|
|
||||||
);
|
);
|
||||||
|
|
||||||
return streamThread.updateThreadMetadata(getSharedAdminClientId(clientId));
|
return streamThread.updateThreadMetadata(getSharedAdminClientId(clientId));
|
||||||
|
@ -457,8 +454,7 @@ public class StreamThread extends Thread {
|
||||||
final Queue<StreamsException> nonFatalExceptionsToHandle,
|
final Queue<StreamsException> nonFatalExceptionsToHandle,
|
||||||
final Runnable shutdownErrorHook,
|
final Runnable shutdownErrorHook,
|
||||||
final BiConsumer<Throwable, Boolean> streamsUncaughtExceptionHandler,
|
final BiConsumer<Throwable, Boolean> streamsUncaughtExceptionHandler,
|
||||||
final java.util.function.Consumer<Long> cacheResizer,
|
final java.util.function.Consumer<Long> cacheResizer) {
|
||||||
final long maxBufferSizeBytes) {
|
|
||||||
super(threadId);
|
super(threadId);
|
||||||
this.stateLock = new Object();
|
this.stateLock = new Object();
|
||||||
this.adminClient = adminClient;
|
this.adminClient = adminClient;
|
||||||
|
@ -527,7 +523,6 @@ public class StreamThread extends Thread {
|
||||||
|
|
||||||
this.numIterations = 1;
|
this.numIterations = 1;
|
||||||
this.eosEnabled = eosEnabled(config);
|
this.eosEnabled = eosEnabled(config);
|
||||||
this.maxBufferSizeBytes.set(maxBufferSizeBytes);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final class InternalConsumerConfig extends ConsumerConfig {
|
private static final class InternalConsumerConfig extends ConsumerConfig {
|
||||||
|
@ -710,17 +705,8 @@ public class StreamThread extends Thread {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void resizeCacheAndBufferMemory(final long cacheSize, final long maxBufferSize) {
|
public void resizeCache(final long size) {
|
||||||
cacheResizeSize.set(cacheSize);
|
cacheResizeSize.set(size);
|
||||||
maxBufferSizeBytes.set(maxBufferSize);
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getCacheSize() {
|
|
||||||
return cacheResizeSize.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getMaxBufferSize() {
|
|
||||||
return maxBufferSizeBytes.get();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -795,13 +781,6 @@ public class StreamThread extends Thread {
|
||||||
|
|
||||||
totalProcessed += processed;
|
totalProcessed += processed;
|
||||||
totalRecordsProcessedSinceLastSummary += processed;
|
totalRecordsProcessedSinceLastSummary += processed;
|
||||||
final long bufferSize = taskManager.getInputBufferSizeInBytes();
|
|
||||||
if (bufferSize <= maxBufferSizeBytes.get()) {
|
|
||||||
final Set<TopicPartition> pausedPartitions = mainConsumer.paused();
|
|
||||||
log.info("Buffered records size {} bytes falls below {}. Resuming all the paused partitions {} in the consumer",
|
|
||||||
bufferSize, maxBufferSizeBytes.get(), pausedPartitions);
|
|
||||||
mainConsumer.resume(pausedPartitions);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
log.debug("Processed {} records with {} iterations; invoking punctuators if necessary",
|
log.debug("Processed {} records with {} iterations; invoking punctuators if necessary",
|
||||||
|
@ -920,8 +899,7 @@ public class StreamThread extends Thread {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Visible for testing
|
private long pollPhase() {
|
||||||
long pollPhase() {
|
|
||||||
final ConsumerRecords<byte[], byte[]> records;
|
final ConsumerRecords<byte[], byte[]> records;
|
||||||
log.debug("Invoking poll on main Consumer");
|
log.debug("Invoking poll on main Consumer");
|
||||||
|
|
||||||
|
@ -967,19 +945,6 @@ public class StreamThread extends Thread {
|
||||||
if (!records.isEmpty()) {
|
if (!records.isEmpty()) {
|
||||||
pollRecordsSensor.record(numRecords, now);
|
pollRecordsSensor.record(numRecords, now);
|
||||||
taskManager.addRecordsToTasks(records);
|
taskManager.addRecordsToTasks(records);
|
||||||
// Check buffer size after adding records to tasks
|
|
||||||
final long bufferSize = taskManager.getInputBufferSizeInBytes();
|
|
||||||
// Pausing partitions as the buffer size now exceeds max buffer size
|
|
||||||
if (bufferSize > maxBufferSizeBytes.get()) {
|
|
||||||
final Set<TopicPartition> nonEmptyPartitions = taskManager.nonEmptyPartitions();
|
|
||||||
log.info("Buffered records size {} bytes exceeds {}. Pausing partitions {} from the consumer",
|
|
||||||
bufferSize, maxBufferSizeBytes.get(), nonEmptyPartitions);
|
|
||||||
// Only non-empty partitions are paused here. Reason is that, if a task has multiple partitions with
|
|
||||||
// some of them empty, then in that case pausing even empty partitions would sacrifice ordered processing
|
|
||||||
// and even lead to temporal deadlock. More explanation can be found here:
|
|
||||||
// https://issues.apache.org/jira/browse/KAFKA-13152?focusedCommentId=17400647&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17400647
|
|
||||||
mainConsumer.pause(nonEmptyPartitions);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
while (!nonFatalExceptionsToHandle.isEmpty()) {
|
while (!nonFatalExceptionsToHandle.isEmpty()) {
|
||||||
|
|
|
@ -1071,17 +1071,6 @@ public class TaskManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Fetch all non-empty partitions for pausing
|
|
||||||
*/
|
|
||||||
Set<TopicPartition> nonEmptyPartitions() {
|
|
||||||
final Set<TopicPartition> nonEmptyPartitions = new HashSet<>();
|
|
||||||
for (final Task task : activeTaskIterable()) {
|
|
||||||
nonEmptyPartitions.addAll(((StreamTask) task).getNonEmptyTopicPartitions());
|
|
||||||
}
|
|
||||||
return nonEmptyPartitions;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @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)
|
||||||
|
@ -1185,14 +1174,6 @@ public class TaskManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
long getInputBufferSizeInBytes() {
|
|
||||||
long bytesBuffered = 0L;
|
|
||||||
for (final Task task : activeTaskIterable()) {
|
|
||||||
bytesBuffered += ((StreamTask) task).totalBytesBuffered();
|
|
||||||
}
|
|
||||||
return bytesBuffered;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @throws TaskMigratedException if the task producer got fenced (EOS only)
|
* @throws TaskMigratedException if the task producer got fenced (EOS only)
|
||||||
* @throws StreamsException if any task threw an exception while processing
|
* @throws StreamsException if any task threw an exception while processing
|
||||||
|
|
|
@ -84,11 +84,6 @@ public class TaskMetrics {
|
||||||
private static final String NUM_BUFFERED_RECORDS_DESCRIPTION = "The count of buffered records that are polled " +
|
private static final String NUM_BUFFERED_RECORDS_DESCRIPTION = "The count of buffered records that are polled " +
|
||||||
"from consumer and not yet processed for this active task";
|
"from consumer and not yet processed for this active task";
|
||||||
|
|
||||||
private static final String INPUT_BUFFER_BYTES_TOTAL = "input-buffer-bytes-total";
|
|
||||||
private static final String INPUT_BUFFER_BYTES_TOTAL_DESCRIPTION = "The total number of bytes accumulated in this task's input buffer";
|
|
||||||
private static final String CACHE_SIZE_BYTES_TOTAL = "cache-size-bytes-total";
|
|
||||||
private static final String CACHE_SIZE_BYTES_TOTAL_DESCRIPTION = "The total size in bytes of this task's cache.";
|
|
||||||
|
|
||||||
public static Sensor processLatencySensor(final String threadId,
|
public static Sensor processLatencySensor(final String threadId,
|
||||||
final String taskId,
|
final String taskId,
|
||||||
final StreamsMetricsImpl streamsMetrics) {
|
final StreamsMetricsImpl streamsMetrics) {
|
||||||
|
@ -133,38 +128,6 @@ public class TaskMetrics {
|
||||||
return sensor;
|
return sensor;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Sensor totalInputBufferBytesSensor(final String threadId,
|
|
||||||
final String taskId,
|
|
||||||
final StreamsMetricsImpl streamsMetrics) {
|
|
||||||
final String name = INPUT_BUFFER_BYTES_TOTAL;
|
|
||||||
final Sensor sensor = streamsMetrics.taskLevelSensor(threadId, taskId, name, RecordingLevel.INFO);
|
|
||||||
|
|
||||||
addValueMetricToSensor(
|
|
||||||
sensor,
|
|
||||||
TASK_LEVEL_GROUP,
|
|
||||||
streamsMetrics.taskLevelTagMap(threadId, taskId),
|
|
||||||
name,
|
|
||||||
INPUT_BUFFER_BYTES_TOTAL_DESCRIPTION
|
|
||||||
);
|
|
||||||
return sensor;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Sensor totalCacheSizeBytesSensor(final String threadId,
|
|
||||||
final String taskId,
|
|
||||||
final StreamsMetricsImpl streamsMetrics) {
|
|
||||||
final String name = CACHE_SIZE_BYTES_TOTAL;
|
|
||||||
final Sensor sensor = streamsMetrics.taskLevelSensor(threadId, taskId, name, Sensor.RecordingLevel.INFO);
|
|
||||||
|
|
||||||
addValueMetricToSensor(
|
|
||||||
sensor,
|
|
||||||
TASK_LEVEL_GROUP,
|
|
||||||
streamsMetrics.taskLevelTagMap(threadId, taskId),
|
|
||||||
name,
|
|
||||||
CACHE_SIZE_BYTES_TOTAL_DESCRIPTION
|
|
||||||
);
|
|
||||||
return sensor;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Sensor punctuateSensor(final String threadId,
|
public static Sensor punctuateSensor(final String threadId,
|
||||||
final String taskId,
|
final String taskId,
|
||||||
final StreamsMetricsImpl streamsMetrics) {
|
final StreamsMetricsImpl streamsMetrics) {
|
||||||
|
|
|
@ -20,7 +20,6 @@ import org.apache.kafka.common.metrics.Sensor;
|
||||||
import org.apache.kafka.common.utils.Bytes;
|
import org.apache.kafka.common.utils.Bytes;
|
||||||
import org.apache.kafka.streams.KeyValue;
|
import org.apache.kafka.streams.KeyValue;
|
||||||
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
|
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
|
||||||
import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
|
|
||||||
import org.apache.kafka.streams.state.internals.metrics.NamedCacheMetrics;
|
import org.apache.kafka.streams.state.internals.metrics.NamedCacheMetrics;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -49,7 +48,6 @@ class NamedCache {
|
||||||
|
|
||||||
private final StreamsMetricsImpl streamsMetrics;
|
private final StreamsMetricsImpl streamsMetrics;
|
||||||
private final Sensor hitRatioSensor;
|
private final Sensor hitRatioSensor;
|
||||||
private final Sensor totalCacheSizeSensor;
|
|
||||||
|
|
||||||
// internal stats
|
// internal stats
|
||||||
private long numReadHits = 0;
|
private long numReadHits = 0;
|
||||||
|
@ -68,11 +66,6 @@ class NamedCache {
|
||||||
taskName,
|
taskName,
|
||||||
storeName
|
storeName
|
||||||
);
|
);
|
||||||
totalCacheSizeSensor = TaskMetrics.totalCacheSizeBytesSensor(
|
|
||||||
Thread.currentThread().getName(),
|
|
||||||
taskName,
|
|
||||||
streamsMetrics
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized final String name() {
|
synchronized final String name() {
|
||||||
|
@ -189,7 +182,6 @@ class NamedCache {
|
||||||
dirtyKeys.add(key);
|
dirtyKeys.add(key);
|
||||||
}
|
}
|
||||||
currentSizeBytes += node.size();
|
currentSizeBytes += node.size();
|
||||||
totalCacheSizeSensor.record(currentSizeBytes);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized long sizeInBytes() {
|
synchronized long sizeInBytes() {
|
||||||
|
@ -251,7 +243,6 @@ class NamedCache {
|
||||||
if (eldest.entry.isDirty()) {
|
if (eldest.entry.isDirty()) {
|
||||||
flush(eldest);
|
flush(eldest);
|
||||||
}
|
}
|
||||||
totalCacheSizeSensor.record(currentSizeBytes);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized LRUCacheEntry putIfAbsent(final Bytes key, final LRUCacheEntry value) {
|
synchronized LRUCacheEntry putIfAbsent(final Bytes key, final LRUCacheEntry value) {
|
||||||
|
@ -278,7 +269,6 @@ class NamedCache {
|
||||||
remove(node);
|
remove(node);
|
||||||
dirtyKeys.remove(key);
|
dirtyKeys.remove(key);
|
||||||
currentSizeBytes -= node.size();
|
currentSizeBytes -= node.size();
|
||||||
totalCacheSizeSensor.record(currentSizeBytes);
|
|
||||||
return node.entry();
|
return node.entry();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -150,8 +150,6 @@ public class KafkaStreamsTest {
|
||||||
private GlobalStreamThread globalStreamThread;
|
private GlobalStreamThread globalStreamThread;
|
||||||
@Mock
|
@Mock
|
||||||
private Metrics metrics;
|
private Metrics metrics;
|
||||||
@Mock
|
|
||||||
private State state;
|
|
||||||
|
|
||||||
private StateListenerStub streamsStateListener;
|
private StateListenerStub streamsStateListener;
|
||||||
private Capture<List<MetricsReporter>> metricsReportersCapture;
|
private Capture<List<MetricsReporter>> metricsReportersCapture;
|
||||||
|
@ -225,7 +223,6 @@ public class KafkaStreamsTest {
|
||||||
ClientMetrics.addStateMetric(anyObject(StreamsMetricsImpl.class), anyObject());
|
ClientMetrics.addStateMetric(anyObject(StreamsMetricsImpl.class), anyObject());
|
||||||
ClientMetrics.addNumAliveStreamThreadMetric(anyObject(StreamsMetricsImpl.class), anyObject());
|
ClientMetrics.addNumAliveStreamThreadMetric(anyObject(StreamsMetricsImpl.class), anyObject());
|
||||||
|
|
||||||
|
|
||||||
// setup stream threads
|
// setup stream threads
|
||||||
PowerMock.mockStatic(StreamThread.class);
|
PowerMock.mockStatic(StreamThread.class);
|
||||||
EasyMock.expect(StreamThread.create(
|
EasyMock.expect(StreamThread.create(
|
||||||
|
@ -239,7 +236,6 @@ public class KafkaStreamsTest {
|
||||||
anyObject(Time.class),
|
anyObject(Time.class),
|
||||||
anyObject(StreamsMetadataState.class),
|
anyObject(StreamsMetadataState.class),
|
||||||
anyLong(),
|
anyLong(),
|
||||||
anyLong(),
|
|
||||||
anyObject(StateDirectory.class),
|
anyObject(StateDirectory.class),
|
||||||
anyObject(StateRestoreListener.class),
|
anyObject(StateRestoreListener.class),
|
||||||
anyInt(),
|
anyInt(),
|
||||||
|
@ -250,13 +246,8 @@ public class KafkaStreamsTest {
|
||||||
PowerMock.mockStatic(StreamsConfigUtils.class);
|
PowerMock.mockStatic(StreamsConfigUtils.class);
|
||||||
EasyMock.expect(StreamsConfigUtils.processingMode(anyObject(StreamsConfig.class))).andReturn(StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE).anyTimes();
|
EasyMock.expect(StreamsConfigUtils.processingMode(anyObject(StreamsConfig.class))).andReturn(StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE).anyTimes();
|
||||||
EasyMock.expect(StreamsConfigUtils.eosEnabled(anyObject(StreamsConfig.class))).andReturn(false).anyTimes();
|
EasyMock.expect(StreamsConfigUtils.eosEnabled(anyObject(StreamsConfig.class))).andReturn(false).anyTimes();
|
||||||
EasyMock.expect(StreamsConfigUtils.getTotalCacheSize(anyObject(StreamsConfig.class))).andReturn(10 * 1024 * 1024L).anyTimes();
|
|
||||||
EasyMock.expect(streamThreadOne.getId()).andReturn(1L).anyTimes();
|
EasyMock.expect(streamThreadOne.getId()).andReturn(1L).anyTimes();
|
||||||
EasyMock.expect(streamThreadTwo.getId()).andReturn(2L).anyTimes();
|
EasyMock.expect(streamThreadTwo.getId()).andReturn(2L).anyTimes();
|
||||||
EasyMock.expect(streamThreadOne.getCacheSize()).andReturn(10485760L).anyTimes();
|
|
||||||
EasyMock.expect(streamThreadOne.getMaxBufferSize()).andReturn(536870912L).anyTimes();
|
|
||||||
EasyMock.expect(streamThreadTwo.getCacheSize()).andReturn(10485760L).anyTimes();
|
|
||||||
EasyMock.expect(streamThreadTwo.getMaxBufferSize()).andReturn(536870912L).anyTimes();
|
|
||||||
prepareStreamThread(streamThreadOne, 1, true);
|
prepareStreamThread(streamThreadOne, 1, true);
|
||||||
prepareStreamThread(streamThreadTwo, 2, false);
|
prepareStreamThread(streamThreadTwo, 2, false);
|
||||||
|
|
||||||
|
@ -305,8 +296,6 @@ public class KafkaStreamsTest {
|
||||||
EasyMock.expect(globalStreamThread.stillRunning()).andReturn(globalThreadState.get() == GlobalStreamThread.State.RUNNING).anyTimes();
|
EasyMock.expect(globalStreamThread.stillRunning()).andReturn(globalThreadState.get() == GlobalStreamThread.State.RUNNING).anyTimes();
|
||||||
globalStreamThread.join();
|
globalStreamThread.join();
|
||||||
EasyMock.expectLastCall().anyTimes();
|
EasyMock.expectLastCall().anyTimes();
|
||||||
globalStreamThread.resize(EasyMock.anyLong());
|
|
||||||
EasyMock.expectLastCall().anyTimes();
|
|
||||||
|
|
||||||
PowerMock.replay(
|
PowerMock.replay(
|
||||||
StreamThread.class,
|
StreamThread.class,
|
||||||
|
@ -363,7 +352,7 @@ public class KafkaStreamsTest {
|
||||||
).anyTimes();
|
).anyTimes();
|
||||||
EasyMock.expect(thread.waitOnThreadState(EasyMock.isA(StreamThread.State.class), anyLong())).andStubReturn(true);
|
EasyMock.expect(thread.waitOnThreadState(EasyMock.isA(StreamThread.State.class), anyLong())).andStubReturn(true);
|
||||||
EasyMock.expect(thread.isAlive()).andReturn(true).times(0, 1);
|
EasyMock.expect(thread.isAlive()).andReturn(true).times(0, 1);
|
||||||
thread.resizeCacheAndBufferMemory(EasyMock.anyLong(), EasyMock.anyLong());
|
thread.resizeCache(EasyMock.anyLong());
|
||||||
EasyMock.expectLastCall().anyTimes();
|
EasyMock.expectLastCall().anyTimes();
|
||||||
thread.requestLeaveGroupDuringShutdown();
|
thread.requestLeaveGroupDuringShutdown();
|
||||||
EasyMock.expectLastCall().anyTimes();
|
EasyMock.expectLastCall().anyTimes();
|
||||||
|
@ -648,27 +637,6 @@ public class KafkaStreamsTest {
|
||||||
streams.start();
|
streams.start();
|
||||||
final int oldSize = streams.threads.size();
|
final int oldSize = streams.threads.size();
|
||||||
waitForCondition(() -> streams.state() == KafkaStreams.State.RUNNING, 15L, "wait until running");
|
waitForCondition(() -> streams.state() == KafkaStreams.State.RUNNING, 15L, "wait until running");
|
||||||
EasyMock.reset(streamThreadOne, streamThreadTwo);
|
|
||||||
EasyMock.expect(streamThreadOne.isRunning()).andStubReturn(true);
|
|
||||||
EasyMock.expect(streamThreadTwo.isRunning()).andStubReturn(true);
|
|
||||||
EasyMock.expect(streamThreadOne.state()).andStubReturn(StreamThread.State.RUNNING);
|
|
||||||
EasyMock.expect(streamThreadTwo.state()).andStubReturn(StreamThread.State.RUNNING);
|
|
||||||
EasyMock.expect(streamThreadOne.getName()).andStubReturn("processId-StreamThread-1");
|
|
||||||
EasyMock.expect(streamThreadTwo.getName()).andStubReturn("processId-StreamThread-2");
|
|
||||||
EasyMock.expect(streamThreadTwo.getId()).andStubReturn(2L);
|
|
||||||
EasyMock.expect(streamThreadOne.getCacheSize()).andReturn(10485760L).anyTimes();
|
|
||||||
EasyMock.expect(streamThreadOne.getMaxBufferSize()).andReturn(536870912L).anyTimes();
|
|
||||||
EasyMock.expect(streamThreadTwo.getCacheSize()).andReturn(10485760L).anyTimes();
|
|
||||||
EasyMock.expect(streamThreadTwo.getMaxBufferSize()).andReturn(536870912L).anyTimes();
|
|
||||||
streamThreadTwo.setStateListener(EasyMock.anyObject());
|
|
||||||
streamThreadTwo.start();
|
|
||||||
|
|
||||||
streamThreadOne.resizeCacheAndBufferMemory(5 * 1024 * 1024L, 256 * 1024 * 1024L);
|
|
||||||
streamThreadTwo.resizeCacheAndBufferMemory(5 * 1024 * 1024L, 256 * 1024 * 1024L);
|
|
||||||
streamThreadOne.shutdown();
|
|
||||||
streamThreadTwo.shutdown();
|
|
||||||
EasyMock.expect(state.isRunningOrRebalancing()).andStubReturn(true);
|
|
||||||
EasyMock.replay(streamThreadOne, streamThreadTwo, state);
|
|
||||||
assertThat(streams.addStreamThread(), equalTo(Optional.of("processId-StreamThread-" + 2)));
|
assertThat(streams.addStreamThread(), equalTo(Optional.of("processId-StreamThread-" + 2)));
|
||||||
assertThat(streams.threads.size(), equalTo(oldSize + 1));
|
assertThat(streams.threads.size(), equalTo(oldSize + 1));
|
||||||
}
|
}
|
||||||
|
|
|
@ -65,7 +65,6 @@ import static org.apache.kafka.streams.StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFI
|
||||||
import static org.apache.kafka.streams.StreamsConfig.adminClientPrefix;
|
import static org.apache.kafka.streams.StreamsConfig.adminClientPrefix;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.consumerPrefix;
|
import static org.apache.kafka.streams.StreamsConfig.consumerPrefix;
|
||||||
import static org.apache.kafka.streams.StreamsConfig.producerPrefix;
|
import static org.apache.kafka.streams.StreamsConfig.producerPrefix;
|
||||||
import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
|
|
||||||
import static org.apache.kafka.test.StreamsTestUtils.getStreamsConfig;
|
import static org.apache.kafka.test.StreamsTestUtils.getStreamsConfig;
|
||||||
import static org.hamcrest.CoreMatchers.containsString;
|
import static org.hamcrest.CoreMatchers.containsString;
|
||||||
import static org.hamcrest.CoreMatchers.hasItem;
|
import static org.hamcrest.CoreMatchers.hasItem;
|
||||||
|
@ -1256,36 +1255,6 @@ public class StreamsConfigTest {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
public void shouldUseStateStoreCacheMaxBytesWhenBothOldAndNewConfigsAreSet() {
|
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 100);
|
|
||||||
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10);
|
|
||||||
final StreamsConfig config = new StreamsConfig(props);
|
|
||||||
assertEquals(getTotalCacheSize(config), 100);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
public void shouldUseCacheMaxBytesBufferingConfigWhenOnlyDeprecatedConfigIsSet() {
|
|
||||||
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10);
|
|
||||||
final StreamsConfig config = new StreamsConfig(props);
|
|
||||||
assertEquals(getTotalCacheSize(config), 10);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldUseStateStoreCacheMaxBytesWhenNewConfigIsSet() {
|
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 10);
|
|
||||||
final StreamsConfig config = new StreamsConfig(props);
|
|
||||||
assertEquals(getTotalCacheSize(config), 10);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldUseDefaultStateStoreCacheMaxBytesConfigWhenNoConfigIsSet() {
|
|
||||||
final StreamsConfig config = new StreamsConfig(props);
|
|
||||||
assertEquals(getTotalCacheSize(config), 10 * 1024 * 1024);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testInvalidSecurityProtocol() {
|
public void testInvalidSecurityProtocol() {
|
||||||
props.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc");
|
props.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc");
|
||||||
|
|
|
@ -123,7 +123,7 @@ public abstract class AbstractJoinIntegrationTest {
|
||||||
|
|
||||||
void prepareEnvironment() throws InterruptedException {
|
void prepareEnvironment() throws InterruptedException {
|
||||||
if (!cacheEnabled) {
|
if (!cacheEnabled) {
|
||||||
STREAMS_CONFIG.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath());
|
STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath());
|
||||||
|
|
|
@ -151,7 +151,7 @@ public abstract class AbstractResetIntegrationTest {
|
||||||
streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath());
|
streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath());
|
||||||
streamsConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
|
streamsConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
|
||||||
streamsConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsConfig.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100);
|
streamsConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100);
|
||||||
streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
|
|
|
@ -379,7 +379,7 @@ public class AdjustStreamThreadCountTest {
|
||||||
final Properties props = new Properties();
|
final Properties props = new Properties();
|
||||||
props.putAll(properties);
|
props.putAll(properties);
|
||||||
props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2);
|
props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2);
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, totalCacheBytes);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, totalCacheBytes);
|
||||||
|
|
||||||
try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), props)) {
|
try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), props)) {
|
||||||
addStreamStateChangeListener(kafkaStreams);
|
addStreamStateChangeListener(kafkaStreams);
|
||||||
|
@ -390,32 +390,7 @@ public class AdjustStreamThreadCountTest {
|
||||||
|
|
||||||
for (final String log : appender.getMessages()) {
|
for (final String log : appender.getMessages()) {
|
||||||
// all 10 bytes should be available for remaining thread
|
// all 10 bytes should be available for remaining thread
|
||||||
if (log.contains("Resizing thread cache/max buffer size due to removal of thread ") && log.contains(", new cache size/max buffer size per thread is 10/536870912")) {
|
if (log.endsWith("Resizing thread cache due to thread removal, new cache size per thread is 10")) {
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
fail();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldResizeMaxBufferAfterThreadRemovalTimesOut() throws InterruptedException {
|
|
||||||
final long maxBufferBytes = 10L;
|
|
||||||
final Properties props = new Properties();
|
|
||||||
props.putAll(properties);
|
|
||||||
props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2);
|
|
||||||
props.put(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG, maxBufferBytes);
|
|
||||||
|
|
||||||
try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), props)) {
|
|
||||||
addStreamStateChangeListener(kafkaStreams);
|
|
||||||
startStreamsAndWaitForRunning(kafkaStreams);
|
|
||||||
|
|
||||||
try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KafkaStreams.class)) {
|
|
||||||
assertThrows(TimeoutException.class, () -> kafkaStreams.removeStreamThread(Duration.ofSeconds(0)));
|
|
||||||
for (final String log : appender.getMessages()) {
|
|
||||||
// all 10 bytes should be available for remaining thread
|
|
||||||
if (log.contains("Resizing thread cache/max buffer size due to removal of thread ") && log.contains(", new cache size/max buffer size per thread is 10485760/10")) {
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -426,14 +401,12 @@ public class AdjustStreamThreadCountTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public void shouldResizeCacheAndInputBufferAfterThreadReplacement() throws InterruptedException {
|
public void shouldResizeCacheAfterThreadReplacement() throws InterruptedException {
|
||||||
final long totalCacheBytes = 10L;
|
final long totalCacheBytes = 10L;
|
||||||
final long maxBufferBytes = 100L;
|
|
||||||
final Properties props = new Properties();
|
final Properties props = new Properties();
|
||||||
props.putAll(properties);
|
props.putAll(properties);
|
||||||
props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2);
|
props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2);
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, totalCacheBytes);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, totalCacheBytes);
|
||||||
props.put(StreamsConfig.INPUT_BUFFER_MAX_BYTES_CONFIG, maxBufferBytes);
|
|
||||||
|
|
||||||
final AtomicBoolean injectError = new AtomicBoolean(false);
|
final AtomicBoolean injectError = new AtomicBoolean(false);
|
||||||
|
|
||||||
|
@ -473,9 +446,8 @@ public class AdjustStreamThreadCountTest {
|
||||||
waitForTransitionFromRebalancingToRunning();
|
waitForTransitionFromRebalancingToRunning();
|
||||||
|
|
||||||
for (final String log : appender.getMessages()) {
|
for (final String log : appender.getMessages()) {
|
||||||
// after we replace the thread there should be two remaining threads with 5 bytes each for
|
// after we replace the thread there should be two remaining threads with 5 bytes each
|
||||||
// the cache and 50 for the input buffer
|
if (log.endsWith("Adding StreamThread-3, there will now be 2 live threads and the new cache size per thread is 5")) {
|
||||||
if (log.endsWith("Adding StreamThread-3, there are now 2 threads with cache size/max buffer size values as 5/50 per thread.")) {
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -97,7 +97,7 @@ public class EmitOnChangeIntegrationTest {
|
||||||
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
|
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
|
||||||
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()),
|
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()),
|
||||||
mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1),
|
mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1),
|
||||||
mkEntry(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0),
|
mkEntry(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0),
|
||||||
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 300000L),
|
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 300000L),
|
||||||
mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class),
|
mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class),
|
||||||
mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class),
|
mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class),
|
||||||
|
|
|
@ -249,7 +249,7 @@ public class EosIntegrationTest {
|
||||||
|
|
||||||
final Properties properties = new Properties();
|
final Properties properties = new Properties();
|
||||||
properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
|
properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
|
||||||
properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 1);
|
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 1);
|
||||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), "1000");
|
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), "1000");
|
||||||
|
@ -326,7 +326,7 @@ public class EosIntegrationTest {
|
||||||
|
|
||||||
final Properties properties = new Properties();
|
final Properties properties = new Properties();
|
||||||
properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
|
properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
|
||||||
properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
||||||
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
|
@ -940,7 +940,7 @@ public class EosIntegrationTest {
|
||||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), maxPollIntervalMs);
|
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), maxPollIntervalMs);
|
||||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), maxPollIntervalMs - 1);
|
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), maxPollIntervalMs - 1);
|
||||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), maxPollIntervalMs);
|
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), maxPollIntervalMs);
|
||||||
properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir);
|
properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir);
|
||||||
properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142");
|
properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142");
|
||||||
|
|
||||||
|
|
|
@ -949,7 +949,7 @@ public class EosV2UpgradeIntegrationTest {
|
||||||
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS);
|
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS);
|
||||||
properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), (int) commitInterval);
|
properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), (int) commitInterval);
|
||||||
properties.put(StreamsConfig.producerPrefix(ProducerConfig.PARTITIONER_CLASS_CONFIG), KeyPartitioner.class);
|
properties.put(StreamsConfig.producerPrefix(ProducerConfig.PARTITIONER_CLASS_CONFIG), KeyPartitioner.class);
|
||||||
properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
properties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath() + File.separator + appDir);
|
properties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath() + File.separator + appDir);
|
||||||
properties.put(InternalConfig.ASSIGNMENT_LISTENER, assignmentListener);
|
properties.put(InternalConfig.ASSIGNMENT_LISTENER, assignmentListener);
|
||||||
|
|
||||||
|
|
|
@ -1,161 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.kafka.streams.integration;
|
|
||||||
|
|
||||||
import org.apache.kafka.common.serialization.IntegerDeserializer;
|
|
||||||
import org.apache.kafka.common.serialization.IntegerSerializer;
|
|
||||||
import org.apache.kafka.common.serialization.Serdes;
|
|
||||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
|
||||||
import org.apache.kafka.common.serialization.StringSerializer;
|
|
||||||
import org.apache.kafka.streams.KeyValue;
|
|
||||||
import org.apache.kafka.streams.StreamsConfig;
|
|
||||||
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse;
|
|
||||||
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
|
|
||||||
import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
|
|
||||||
import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper;
|
|
||||||
import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder;
|
|
||||||
import org.apache.kafka.test.IntegrationTest;
|
|
||||||
import org.apache.kafka.test.StreamsTestUtils;
|
|
||||||
import org.apache.kafka.test.TestUtils;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Properties;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
import org.junit.AfterClass;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.TestName;
|
|
||||||
|
|
||||||
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.mkObjectProperties;
|
|
||||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
|
|
||||||
|
|
||||||
import static org.hamcrest.CoreMatchers.equalTo;
|
|
||||||
import static org.hamcrest.MatcherAssert.assertThat;
|
|
||||||
|
|
||||||
@Category(IntegrationTest.class)
|
|
||||||
public class ErrorHandlingIntegrationTest {
|
|
||||||
|
|
||||||
private static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void startCluster() throws IOException {
|
|
||||||
CLUSTER.start();
|
|
||||||
}
|
|
||||||
|
|
||||||
@AfterClass
|
|
||||||
public static void closeCluster() {
|
|
||||||
CLUSTER.stop();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public TestName testName = new TestName();
|
|
||||||
|
|
||||||
private final String testId = safeUniqueTestName(getClass(), testName);
|
|
||||||
private final String appId = "appId_" + testId;
|
|
||||||
private final Properties properties = props();
|
|
||||||
|
|
||||||
// Task 0
|
|
||||||
private final String inputTopic = "input" + testId;
|
|
||||||
private final String outputTopic = "output" + testId;
|
|
||||||
// Task 1
|
|
||||||
private final String errorInputTopic = "error-input" + testId;
|
|
||||||
private final String errorOutputTopic = "error-output" + testId;
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void setup() {
|
|
||||||
IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, errorInputTopic, errorOutputTopic, inputTopic, outputTopic);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Properties props() {
|
|
||||||
return mkObjectProperties(
|
|
||||||
mkMap(
|
|
||||||
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()),
|
|
||||||
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
|
|
||||||
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()),
|
|
||||||
mkEntry(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0),
|
|
||||||
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 15000L),
|
|
||||||
mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class),
|
|
||||||
mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class))
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldBackOffTaskAndEmitDataWithinSameTopology() throws Exception {
|
|
||||||
final AtomicInteger noOutputExpected = new AtomicInteger(0);
|
|
||||||
final AtomicInteger outputExpected = new AtomicInteger(0);
|
|
||||||
|
|
||||||
try (final KafkaStreamsNamedTopologyWrapper kafkaStreams = new KafkaStreamsNamedTopologyWrapper(properties)) {
|
|
||||||
kafkaStreams.setUncaughtExceptionHandler(exception -> StreamThreadExceptionResponse.REPLACE_THREAD);
|
|
||||||
|
|
||||||
final NamedTopologyBuilder builder = kafkaStreams.newNamedTopologyBuilder("topology_A");
|
|
||||||
builder.stream(inputTopic).peek((k, v) -> outputExpected.incrementAndGet()).to(outputTopic);
|
|
||||||
builder.stream(errorInputTopic)
|
|
||||||
.peek((k, v) -> {
|
|
||||||
throw new RuntimeException("Kaboom");
|
|
||||||
})
|
|
||||||
.peek((k, v) -> noOutputExpected.incrementAndGet())
|
|
||||||
.to(errorOutputTopic);
|
|
||||||
|
|
||||||
kafkaStreams.addNamedTopology(builder.build());
|
|
||||||
|
|
||||||
StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams);
|
|
||||||
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
|
|
||||||
errorInputTopic,
|
|
||||||
Arrays.asList(
|
|
||||||
new KeyValue<>(1, "A")
|
|
||||||
),
|
|
||||||
TestUtils.producerConfig(
|
|
||||||
CLUSTER.bootstrapServers(),
|
|
||||||
IntegerSerializer.class,
|
|
||||||
StringSerializer.class,
|
|
||||||
new Properties()),
|
|
||||||
0L);
|
|
||||||
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
|
|
||||||
inputTopic,
|
|
||||||
Arrays.asList(
|
|
||||||
new KeyValue<>(1, "A"),
|
|
||||||
new KeyValue<>(1, "B")
|
|
||||||
),
|
|
||||||
TestUtils.producerConfig(
|
|
||||||
CLUSTER.bootstrapServers(),
|
|
||||||
IntegerSerializer.class,
|
|
||||||
StringSerializer.class,
|
|
||||||
new Properties()),
|
|
||||||
0L);
|
|
||||||
IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived(
|
|
||||||
TestUtils.consumerConfig(
|
|
||||||
CLUSTER.bootstrapServers(),
|
|
||||||
IntegerDeserializer.class,
|
|
||||||
StringDeserializer.class
|
|
||||||
),
|
|
||||||
outputTopic,
|
|
||||||
Arrays.asList(
|
|
||||||
new KeyValue<>(1, "A"),
|
|
||||||
new KeyValue<>(1, "B")
|
|
||||||
)
|
|
||||||
);
|
|
||||||
assertThat(noOutputExpected.get(), equalTo(0));
|
|
||||||
assertThat(outputExpected.get(), equalTo(2));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -147,7 +147,7 @@ public class FineGrainedAutoResetIntegrationTest {
|
||||||
public void setUp() throws IOException {
|
public void setUp() throws IOException {
|
||||||
|
|
||||||
final Properties props = new Properties();
|
final Properties props = new Properties();
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
||||||
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
|
@ -286,7 +286,7 @@ public class FineGrainedAutoResetIntegrationTest {
|
||||||
@Test
|
@Test
|
||||||
public void shouldThrowStreamsExceptionNoResetSpecified() throws InterruptedException {
|
public void shouldThrowStreamsExceptionNoResetSpecified() throws InterruptedException {
|
||||||
final Properties props = new Properties();
|
final Properties props = new Properties();
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
||||||
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
|
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
|
||||||
|
|
|
@ -136,7 +136,7 @@ public class GlobalKTableEOSIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0L);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0L);
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
|
streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
|
||||||
streamsConfiguration.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1L);
|
streamsConfiguration.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1L);
|
||||||
|
|
|
@ -114,7 +114,7 @@ public class GlobalKTableIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
globalTable = builder.globalTable(globalTableTopic, Consumed.with(Serdes.Long(), Serdes.String()),
|
globalTable = builder.globalTable(globalTableTopic, Consumed.with(Serdes.Long(), Serdes.String()),
|
||||||
Materialized.<Long, String, KeyValueStore<Bytes, byte[]>>as(globalStore)
|
Materialized.<Long, String, KeyValueStore<Bytes, byte[]>>as(globalStore)
|
||||||
|
|
|
@ -121,7 +121,7 @@ public class GlobalThreadShutDownOrderTest {
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
|
|
||||||
final Consumed<String, Long> stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long());
|
final Consumed<String, Long> stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long());
|
||||||
|
|
|
@ -109,7 +109,7 @@ public class InternalTopicIntegrationTest {
|
||||||
streamsProp.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
streamsProp.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
streamsProp.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsProp.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsProp.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsProp.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsProp.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsProp.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsProp.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsProp.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -111,7 +111,7 @@ public class KStreamAggregationDedupIntegrationTest {
|
||||||
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL_MS);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL_MS);
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 10 * 1024 * 1024L);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);
|
||||||
|
|
||||||
final KeyValueMapper<Integer, String, String> mapper = MockMapper.selectValueMapper();
|
final KeyValueMapper<Integer, String, String> mapper = MockMapper.selectValueMapper();
|
||||||
stream = builder.stream(streamOneInput, Consumed.with(Serdes.Integer(), Serdes.String()));
|
stream = builder.stream(streamOneInput, Consumed.with(Serdes.Integer(), Serdes.String()));
|
||||||
|
|
|
@ -146,7 +146,7 @@ public class KStreamAggregationIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
|
|
|
@ -142,7 +142,7 @@ public class KStreamRepartitionIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
|
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
|
|
|
@ -185,7 +185,6 @@ public class KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest {
|
||||||
assertEquals(expectedResult, result);
|
assertEquals(expectedResult, result);
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
private static Properties getStreamsConfig() {
|
private static Properties getStreamsConfig() {
|
||||||
final Properties streamsConfig = new Properties();
|
final Properties streamsConfig = new Properties();
|
||||||
streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, "KTable-FKJ-Partitioner");
|
streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, "KTable-FKJ-Partitioner");
|
||||||
|
|
|
@ -212,7 +212,7 @@ public class KTableKTableForeignKeyInnerJoinMultiIntegrationTest {
|
||||||
streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, "KTable-FKJ-Multi");
|
streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, "KTable-FKJ-Multi");
|
||||||
streamsConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
streamsConfig.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
|
|
||||||
return streamsConfig;
|
return streamsConfig;
|
||||||
|
|
|
@ -72,7 +72,7 @@ public class KTableSourceTopicRestartIntegrationTest {
|
||||||
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
STREAMS_CONFIG.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5L);
|
STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5L);
|
||||||
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class);
|
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class);
|
||||||
STREAMS_CONFIG.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
STREAMS_CONFIG.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
||||||
|
|
|
@ -198,8 +198,6 @@ public class MetricsIntegrationTest {
|
||||||
private static final String THREAD_START_TIME = "thread-start-time";
|
private static final String THREAD_START_TIME = "thread-start-time";
|
||||||
private static final String ACTIVE_PROCESS_RATIO = "active-process-ratio";
|
private static final String ACTIVE_PROCESS_RATIO = "active-process-ratio";
|
||||||
private static final String ACTIVE_BUFFER_COUNT = "active-buffer-count";
|
private static final String ACTIVE_BUFFER_COUNT = "active-buffer-count";
|
||||||
private static final String INPUT_BUFFER_BYTES_TOTAL = "input-buffer-bytes-total";
|
|
||||||
private static final String CACHE_SIZE_BYTES_TOTAL = "cache-size-bytes-total";
|
|
||||||
private static final String SKIPPED_RECORDS_RATE = "skipped-records-rate";
|
private static final String SKIPPED_RECORDS_RATE = "skipped-records-rate";
|
||||||
private static final String SKIPPED_RECORDS_TOTAL = "skipped-records-total";
|
private static final String SKIPPED_RECORDS_TOTAL = "skipped-records-total";
|
||||||
private static final String RECORD_LATENESS_AVG = "record-lateness-avg";
|
private static final String RECORD_LATENESS_AVG = "record-lateness-avg";
|
||||||
|
@ -260,7 +258,7 @@ public class MetricsIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.DEBUG.name);
|
streamsConfiguration.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.DEBUG.name);
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 10 * 1024 * 1024L);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);
|
||||||
streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS);
|
streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS);
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
}
|
}
|
||||||
|
@ -537,8 +535,6 @@ public class MetricsIntegrationTest {
|
||||||
checkMetricByName(listMetricTask, PUNCTUATE_TOTAL, 4);
|
checkMetricByName(listMetricTask, PUNCTUATE_TOTAL, 4);
|
||||||
checkMetricByName(listMetricTask, PROCESS_RATE, 4);
|
checkMetricByName(listMetricTask, PROCESS_RATE, 4);
|
||||||
checkMetricByName(listMetricTask, PROCESS_TOTAL, 4);
|
checkMetricByName(listMetricTask, PROCESS_TOTAL, 4);
|
||||||
checkMetricByName(listMetricTask, INPUT_BUFFER_BYTES_TOTAL, 4);
|
|
||||||
checkMetricByName(listMetricTask, CACHE_SIZE_BYTES_TOTAL, 3);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkProcessorNodeLevelMetrics() {
|
private void checkProcessorNodeLevelMetrics() {
|
||||||
|
|
|
@ -820,7 +820,7 @@ public class NamedTopologyIntegrationTest {
|
||||||
try {
|
try {
|
||||||
final AtomicInteger noOutputExpected = new AtomicInteger(0);
|
final AtomicInteger noOutputExpected = new AtomicInteger(0);
|
||||||
final AtomicInteger outputExpected = new AtomicInteger(0);
|
final AtomicInteger outputExpected = new AtomicInteger(0);
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 15000L);
|
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 15000L);
|
||||||
props.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath());
|
props.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath());
|
||||||
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class);
|
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class);
|
||||||
|
|
|
@ -213,7 +213,7 @@ public class OptimizedKTableIntegrationTest {
|
||||||
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
config.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
|
config.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
|
||||||
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
config.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
config.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
config.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 100);
|
config.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 100);
|
||||||
config.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 200);
|
config.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 200);
|
||||||
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
||||||
|
|
|
@ -132,7 +132,7 @@ public class PauseResumeIntegrationTest {
|
||||||
properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
properties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
|
properties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
|
||||||
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
|
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
|
||||||
properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
properties.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100);
|
properties.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100);
|
||||||
properties.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
properties.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
||||||
|
|
|
@ -967,7 +967,7 @@ public class QueryableStateIntegrationTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void verifyCanQueryState(final int cacheSizeBytes) throws Exception {
|
private void verifyCanQueryState(final int cacheSizeBytes) throws Exception {
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, cacheSizeBytes);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes);
|
||||||
final StreamsBuilder builder = new StreamsBuilder();
|
final StreamsBuilder builder = new StreamsBuilder();
|
||||||
final String[] keys = {"hello", "goodbye", "welcome", "go", "kafka"};
|
final String[] keys = {"hello", "goodbye", "welcome", "go", "kafka"};
|
||||||
|
|
||||||
|
|
|
@ -133,7 +133,7 @@ public class RegexSourceIntegrationTest {
|
||||||
public void setUp() throws InterruptedException {
|
public void setUp() throws InterruptedException {
|
||||||
outputTopic = createTopic(topicSuffixGenerator.incrementAndGet());
|
outputTopic = createTopic(topicSuffixGenerator.incrementAndGet());
|
||||||
final Properties properties = new Properties();
|
final Properties properties = new Properties();
|
||||||
properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
|
||||||
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
|
|
|
@ -126,7 +126,7 @@ public class RestoreIntegrationTest {
|
||||||
final Properties streamsConfiguration = new Properties();
|
final Properties streamsConfiguration = new Properties();
|
||||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
|
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
|
|
|
@ -200,7 +200,7 @@ public class RocksDBMetricsIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.DEBUG.name);
|
streamsConfiguration.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.DEBUG.name);
|
||||||
streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee);
|
streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee);
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
return streamsConfiguration;
|
return streamsConfiguration;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -141,7 +141,7 @@ public class SlidingWindowedKStreamIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
|
|
|
@ -403,7 +403,7 @@ public class StandbyTaskEOSIntegrationTest {
|
||||||
final Properties streamsConfiguration = new Properties();
|
final Properties streamsConfiguration = new Properties();
|
||||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
|
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, stateDirPath);
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, stateDirPath);
|
||||||
streamsConfiguration.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
|
streamsConfiguration.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
|
||||||
streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
|
streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
|
||||||
|
|
|
@ -94,7 +94,7 @@ public class StoreUpgradeIntegrationTest {
|
||||||
final String safeTestName = safeUniqueTestName(getClass(), testName);
|
final String safeTestName = safeUniqueTestName(getClass(), testName);
|
||||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
|
|
|
@ -124,7 +124,7 @@ public class StreamTableJoinTopologyOptimizationIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
|
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
|
|
|
@ -148,7 +148,7 @@ public class TimeWindowedKStreamIntegrationTest {
|
||||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||||
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
|
|
|
@ -51,7 +51,7 @@ import static org.hamcrest.Matchers.is;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertNull;
|
import static org.junit.Assert.assertNull;
|
||||||
|
|
||||||
@SuppressWarnings({"unchecked"})
|
@SuppressWarnings("unchecked")
|
||||||
public class KTableFilterTest {
|
public class KTableFilterTest {
|
||||||
private final Consumed<String, Integer> consumed = Consumed.with(Serdes.String(), Serdes.Integer());
|
private final Consumed<String, Integer> consumed = Consumed.with(Serdes.String(), Serdes.Integer());
|
||||||
private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.Integer());
|
private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.Integer());
|
||||||
|
@ -59,7 +59,7 @@ public class KTableFilterTest {
|
||||||
@Before
|
@Before
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
// disable caching at the config level
|
// disable caching at the config level
|
||||||
props.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, "0");
|
props.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, "0");
|
||||||
}
|
}
|
||||||
|
|
||||||
private final Predicate<String, Integer> predicate = (key, value) -> (value % 2) == 0;
|
private final Predicate<String, Integer> predicate = (key, value) -> (value % 2) == 0;
|
||||||
|
|
|
@ -104,7 +104,7 @@ public class SessionWindowedKStreamImplTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldCountSessionWindowedWithCachingDisabled() {
|
public void shouldCountSessionWindowedWithCachingDisabled() {
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
shouldCountSessionWindowed();
|
shouldCountSessionWindowed();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -966,22 +966,9 @@ public class InternalTopologyBuilderTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
public void shouldUseNonDeprecatedConfigToSetCacheBytesWhenBothDeprecatedAndNonDeprecatedConfigsUsed() {
|
|
||||||
final Properties globalProps = StreamsTestUtils.getStreamsConfig();
|
|
||||||
globalProps.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 200L);
|
|
||||||
globalProps.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 100L);
|
|
||||||
final StreamsConfig globalStreamsConfig = new StreamsConfig(globalProps);
|
|
||||||
final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(globalStreamsConfig);
|
|
||||||
assertThat(topologyBuilder.topologyConfigs(), equalTo(new TopologyConfig(null, globalStreamsConfig, new Properties())));
|
|
||||||
assertThat(topologyBuilder.topologyConfigs().cacheSize, equalTo(200L));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() {
|
public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() {
|
||||||
final Properties topologyOverrides = new Properties();
|
final Properties topologyOverrides = new Properties();
|
||||||
topologyOverrides.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 12345L);
|
topologyOverrides.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 12345L);
|
||||||
topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L);
|
topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L);
|
||||||
topologyOverrides.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L);
|
topologyOverrides.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L);
|
||||||
topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15);
|
topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15);
|
||||||
|
@ -1007,10 +994,9 @@ public class InternalTopologyBuilderTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
public void shouldNotOverrideGlobalStreamsConfigWhenGivenUnnamedTopologyProps() {
|
public void shouldNotOverrideGlobalStreamsConfigWhenGivenUnnamedTopologyProps() {
|
||||||
final Properties streamsProps = StreamsTestUtils.getStreamsConfig();
|
final Properties streamsProps = StreamsTestUtils.getStreamsConfig();
|
||||||
streamsProps.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 12345L);
|
streamsProps.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 12345L);
|
||||||
streamsProps.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L);
|
streamsProps.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L);
|
||||||
streamsProps.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L);
|
streamsProps.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L);
|
||||||
streamsProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15);
|
streamsProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15);
|
||||||
|
|
|
@ -19,11 +19,9 @@ package org.apache.kafka.streams.processor.internals;
|
||||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||||
import org.apache.kafka.common.MetricName;
|
import org.apache.kafka.common.MetricName;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.header.internals.RecordHeaders;
|
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
import org.apache.kafka.common.metrics.Sensor;
|
import org.apache.kafka.common.metrics.Sensor;
|
||||||
import org.apache.kafka.common.metrics.stats.Value;
|
import org.apache.kafka.common.metrics.stats.Value;
|
||||||
import org.apache.kafka.common.record.TimestampType;
|
|
||||||
import org.apache.kafka.common.serialization.Deserializer;
|
import org.apache.kafka.common.serialization.Deserializer;
|
||||||
import org.apache.kafka.common.serialization.IntegerDeserializer;
|
import org.apache.kafka.common.serialization.IntegerDeserializer;
|
||||||
import org.apache.kafka.common.serialization.IntegerSerializer;
|
import org.apache.kafka.common.serialization.IntegerSerializer;
|
||||||
|
@ -46,13 +44,10 @@ import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.OptionalLong;
|
import java.util.OptionalLong;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
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.mkMap;
|
||||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||||
import static org.apache.kafka.streams.processor.internals.ClientUtils.consumerRecordSizeInBytes;
|
|
||||||
|
|
||||||
import static org.hamcrest.CoreMatchers.is;
|
import static org.hamcrest.CoreMatchers.is;
|
||||||
import static org.hamcrest.MatcherAssert.assertThat;
|
import static org.hamcrest.MatcherAssert.assertThat;
|
||||||
|
@ -89,7 +84,6 @@ public class PartitionGroupTest {
|
||||||
private final Metrics metrics = new Metrics();
|
private final Metrics metrics = new Metrics();
|
||||||
private final Sensor enforcedProcessingSensor = metrics.sensor(UUID.randomUUID().toString());
|
private final Sensor enforcedProcessingSensor = metrics.sensor(UUID.randomUUID().toString());
|
||||||
private final MetricName lastLatenessValue = new MetricName("record-lateness-last-value", "", "", mkMap());
|
private final MetricName lastLatenessValue = new MetricName("record-lateness-last-value", "", "", mkMap());
|
||||||
private final MetricName totalBytesValue = new MetricName("total-bytes-last-value", "", "", mkMap());
|
|
||||||
|
|
||||||
|
|
||||||
private static Sensor getValueSensor(final Metrics metrics, final MetricName metricName) {
|
private static Sensor getValueSensor(final Metrics metrics, final MetricName metricName) {
|
||||||
|
@ -491,7 +485,6 @@ public class PartitionGroupTest {
|
||||||
mkMap(mkEntry(partition1, queue1)),
|
mkMap(mkEntry(partition1, queue1)),
|
||||||
tp -> OptionalLong.of(0L),
|
tp -> OptionalLong.of(0L),
|
||||||
getValueSensor(metrics, lastLatenessValue),
|
getValueSensor(metrics, lastLatenessValue),
|
||||||
getValueSensor(metrics, totalBytesValue),
|
|
||||||
enforcedProcessingSensor,
|
enforcedProcessingSensor,
|
||||||
maxTaskIdleMs
|
maxTaskIdleMs
|
||||||
);
|
);
|
||||||
|
@ -525,7 +518,6 @@ public class PartitionGroupTest {
|
||||||
mkMap(mkEntry(partition1, queue1)),
|
mkMap(mkEntry(partition1, queue1)),
|
||||||
tp -> OptionalLong.of(0L),
|
tp -> OptionalLong.of(0L),
|
||||||
getValueSensor(metrics, lastLatenessValue),
|
getValueSensor(metrics, lastLatenessValue),
|
||||||
getValueSensor(metrics, totalBytesValue),
|
|
||||||
enforcedProcessingSensor,
|
enforcedProcessingSensor,
|
||||||
maxTaskIdleMs
|
maxTaskIdleMs
|
||||||
);
|
);
|
||||||
|
@ -561,7 +553,6 @@ public class PartitionGroupTest {
|
||||||
),
|
),
|
||||||
tp -> OptionalLong.of(0L),
|
tp -> OptionalLong.of(0L),
|
||||||
getValueSensor(metrics, lastLatenessValue),
|
getValueSensor(metrics, lastLatenessValue),
|
||||||
getValueSensor(metrics, totalBytesValue),
|
|
||||||
enforcedProcessingSensor,
|
enforcedProcessingSensor,
|
||||||
StreamsConfig.MAX_TASK_IDLE_MS_DISABLED
|
StreamsConfig.MAX_TASK_IDLE_MS_DISABLED
|
||||||
);
|
);
|
||||||
|
@ -600,7 +591,6 @@ public class PartitionGroupTest {
|
||||||
),
|
),
|
||||||
tp -> OptionalLong.of(0L),
|
tp -> OptionalLong.of(0L),
|
||||||
getValueSensor(metrics, lastLatenessValue),
|
getValueSensor(metrics, lastLatenessValue),
|
||||||
getValueSensor(metrics, totalBytesValue),
|
|
||||||
enforcedProcessingSensor,
|
enforcedProcessingSensor,
|
||||||
0L
|
0L
|
||||||
);
|
);
|
||||||
|
@ -640,7 +630,6 @@ public class PartitionGroupTest {
|
||||||
),
|
),
|
||||||
tp -> lags.getOrDefault(tp, OptionalLong.empty()),
|
tp -> lags.getOrDefault(tp, OptionalLong.empty()),
|
||||||
getValueSensor(metrics, lastLatenessValue),
|
getValueSensor(metrics, lastLatenessValue),
|
||||||
getValueSensor(metrics, totalBytesValue),
|
|
||||||
enforcedProcessingSensor,
|
enforcedProcessingSensor,
|
||||||
0L
|
0L
|
||||||
);
|
);
|
||||||
|
@ -677,7 +666,6 @@ public class PartitionGroupTest {
|
||||||
),
|
),
|
||||||
tp -> lags.getOrDefault(tp, OptionalLong.empty()),
|
tp -> lags.getOrDefault(tp, OptionalLong.empty()),
|
||||||
getValueSensor(metrics, lastLatenessValue),
|
getValueSensor(metrics, lastLatenessValue),
|
||||||
getValueSensor(metrics, totalBytesValue),
|
|
||||||
enforcedProcessingSensor,
|
enforcedProcessingSensor,
|
||||||
0L
|
0L
|
||||||
);
|
);
|
||||||
|
@ -714,7 +702,6 @@ public class PartitionGroupTest {
|
||||||
),
|
),
|
||||||
tp -> OptionalLong.of(0L),
|
tp -> OptionalLong.of(0L),
|
||||||
getValueSensor(metrics, lastLatenessValue),
|
getValueSensor(metrics, lastLatenessValue),
|
||||||
getValueSensor(metrics, totalBytesValue),
|
|
||||||
enforcedProcessingSensor,
|
enforcedProcessingSensor,
|
||||||
1L
|
1L
|
||||||
);
|
);
|
||||||
|
@ -777,81 +764,6 @@ public class PartitionGroupTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldUpdateTotalBytesBufferedOnRecordsAdditionAndConsumption() {
|
|
||||||
final PartitionGroup group = getBasicGroup();
|
|
||||||
|
|
||||||
assertEquals(0, group.numBuffered());
|
|
||||||
assertEquals(0L, group.totalBytesBuffered());
|
|
||||||
|
|
||||||
// add three 3 records with timestamp 1, 5, 3 to partition-1
|
|
||||||
final List<ConsumerRecord<byte[], byte[]>> list1 = Arrays.asList(
|
|
||||||
new ConsumerRecord<>("topic", 1, 1L, new MockTime().milliseconds(), TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()),
|
|
||||||
new ConsumerRecord<>("topic", 1, 5L, new MockTime().milliseconds(), TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()),
|
|
||||||
new ConsumerRecord<>("topic", 1, 3L, new MockTime().milliseconds(), TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()));
|
|
||||||
|
|
||||||
long partition1TotalBytes = getBytesBufferedForRawRecords(list1);
|
|
||||||
group.addRawRecords(partition1, list1);
|
|
||||||
|
|
||||||
verifyBuffered(3, 3, 0, group);
|
|
||||||
assertEquals(group.totalBytesBuffered(), partition1TotalBytes);
|
|
||||||
assertEquals(-1L, group.streamTime());
|
|
||||||
assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue());
|
|
||||||
assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition1TotalBytes));
|
|
||||||
|
|
||||||
StampedRecord record;
|
|
||||||
final PartitionGroup.RecordInfo info = new PartitionGroup.RecordInfo();
|
|
||||||
|
|
||||||
// get first two records from partition 1
|
|
||||||
record = group.nextRecord(info, time.milliseconds());
|
|
||||||
assertEquals(record.timestamp, 1L);
|
|
||||||
record = group.nextRecord(info, time.milliseconds());
|
|
||||||
assertEquals(record.timestamp, 5L);
|
|
||||||
|
|
||||||
partition1TotalBytes -= getBytesBufferedForRawRecords(Arrays.asList(list1.get(0), list1.get(0)));
|
|
||||||
assertEquals(group.totalBytesBuffered(), partition1TotalBytes);
|
|
||||||
assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition1TotalBytes));
|
|
||||||
|
|
||||||
// add three 3 records with timestamp 2, 4, 6 to partition-2
|
|
||||||
final List<ConsumerRecord<byte[], byte[]>> list2 = Arrays.asList(
|
|
||||||
new ConsumerRecord<>("topic", 2, 2L, record.timestamp, TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()),
|
|
||||||
new ConsumerRecord<>("topic", 2, 4L, record.timestamp, TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()),
|
|
||||||
new ConsumerRecord<>("topic", 2, 6L, record.timestamp, TimestampType.CREATE_TIME, recordKey.length, recordValue.length, recordKey, recordValue, new RecordHeaders(), Optional.empty()));
|
|
||||||
|
|
||||||
long partition2TotalBytes = getBytesBufferedForRawRecords(list2);
|
|
||||||
group.addRawRecords(partition2, list2);
|
|
||||||
// 1:[3]
|
|
||||||
// 2:[2, 4, 6]
|
|
||||||
assertEquals(group.totalBytesBuffered(), partition2TotalBytes + partition1TotalBytes);
|
|
||||||
assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition2TotalBytes + partition1TotalBytes));
|
|
||||||
|
|
||||||
// get one record, next record should be ts=2 from partition 2
|
|
||||||
record = group.nextRecord(info, time.milliseconds());
|
|
||||||
// 1:[3]
|
|
||||||
// 2:[4, 6]
|
|
||||||
partition2TotalBytes -= getBytesBufferedForRawRecords(Collections.singletonList(list2.get(0)));
|
|
||||||
assertEquals(group.totalBytesBuffered(), partition2TotalBytes + partition1TotalBytes);
|
|
||||||
assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition2TotalBytes + partition1TotalBytes));
|
|
||||||
assertEquals(record.timestamp, 2L);
|
|
||||||
|
|
||||||
// get one record, next up should have ts=3 from partition 1 (even though it has seen a larger max timestamp =5)
|
|
||||||
record = group.nextRecord(info, time.milliseconds());
|
|
||||||
// 1:[]
|
|
||||||
// 2:[4, 6]
|
|
||||||
partition1TotalBytes -= getBytesBufferedForRawRecords(Collections.singletonList(list2.get(2)));
|
|
||||||
assertEquals(group.totalBytesBuffered(), partition2TotalBytes + partition1TotalBytes);
|
|
||||||
assertThat(metrics.metric(totalBytesValue).metricValue(), is((double) partition2TotalBytes + partition1TotalBytes));
|
|
||||||
assertEquals(record.timestamp, 3L);
|
|
||||||
}
|
|
||||||
|
|
||||||
private long getBytesBufferedForRawRecords(final List<ConsumerRecord<byte[], byte[]>> rawRecords) {
|
|
||||||
long rawRecordsSizeInBytes = 0L;
|
|
||||||
for (final ConsumerRecord<byte[], byte[]> rawRecord : rawRecords) {
|
|
||||||
rawRecordsSizeInBytes += consumerRecordSizeInBytes(rawRecord);
|
|
||||||
}
|
|
||||||
return rawRecordsSizeInBytes;
|
|
||||||
}
|
|
||||||
|
|
||||||
private PartitionGroup getBasicGroup() {
|
private PartitionGroup getBasicGroup() {
|
||||||
return new PartitionGroup(
|
return new PartitionGroup(
|
||||||
logContext,
|
logContext,
|
||||||
|
@ -861,7 +773,6 @@ public class PartitionGroupTest {
|
||||||
),
|
),
|
||||||
tp -> OptionalLong.of(0L),
|
tp -> OptionalLong.of(0L),
|
||||||
getValueSensor(metrics, lastLatenessValue),
|
getValueSensor(metrics, lastLatenessValue),
|
||||||
getValueSensor(metrics, totalBytesValue),
|
|
||||||
enforcedProcessingSensor,
|
enforcedProcessingSensor,
|
||||||
maxTaskIdleMs
|
maxTaskIdleMs
|
||||||
);
|
);
|
||||||
|
|
|
@ -110,7 +110,7 @@ public class RepartitionOptimizingTest {
|
||||||
@Before
|
@Before
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
streamsConfiguration = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
|
streamsConfiguration = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
|
||||||
streamsConfiguration.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, Integer.toString(1024 * 10));
|
streamsConfiguration.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, Integer.toString(1024 * 10));
|
||||||
streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(5000));
|
streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(5000));
|
||||||
|
|
||||||
processorValueCollector.clear();
|
processorValueCollector.clear();
|
||||||
|
|
|
@ -86,7 +86,7 @@ public class RepartitionWithMergeOptimizingTest {
|
||||||
@Before
|
@Before
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
streamsConfiguration = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
|
streamsConfiguration = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
|
||||||
streamsConfiguration.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, Integer.toString(1024 * 10));
|
streamsConfiguration.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, Integer.toString(1024 * 10));
|
||||||
streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(5000));
|
streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(5000));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -108,6 +108,7 @@ public class StandbyTaskTest {
|
||||||
return new StreamsConfig(mkProperties(mkMap(
|
return new StreamsConfig(mkProperties(mkMap(
|
||||||
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId),
|
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId),
|
||||||
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"),
|
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"),
|
||||||
|
mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"),
|
||||||
mkEntry(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()),
|
mkEntry(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()),
|
||||||
mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName())
|
mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName())
|
||||||
)));
|
)));
|
||||||
|
|
|
@ -227,7 +227,6 @@ public class StreamTaskTest {
|
||||||
return createConfig(eosConfig, enforcedProcessingValue, LogAndFailExceptionHandler.class.getName());
|
return createConfig(eosConfig, enforcedProcessingValue, LogAndFailExceptionHandler.class.getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
private static StreamsConfig createConfig(
|
private static StreamsConfig createConfig(
|
||||||
final String eosConfig,
|
final String eosConfig,
|
||||||
final String enforcedProcessingValue,
|
final String enforcedProcessingValue,
|
||||||
|
|
|
@ -121,7 +121,6 @@ import static org.apache.kafka.common.utils.Utils.mkProperties;
|
||||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||||
import static org.apache.kafka.streams.processor.internals.ClientUtils.getSharedAdminClientId;
|
import static org.apache.kafka.streams.processor.internals.ClientUtils.getSharedAdminClientId;
|
||||||
import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME;
|
import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME;
|
||||||
import static org.easymock.EasyMock.anyInt;
|
|
||||||
import static org.easymock.EasyMock.anyObject;
|
import static org.easymock.EasyMock.anyObject;
|
||||||
import static org.easymock.EasyMock.expect;
|
import static org.easymock.EasyMock.expect;
|
||||||
import static org.easymock.EasyMock.expectLastCall;
|
import static org.easymock.EasyMock.expectLastCall;
|
||||||
|
@ -164,7 +163,6 @@ public class StreamThreadTest {
|
||||||
private final StateDirectory stateDirectory = new StateDirectory(config, mockTime, true, false);
|
private final StateDirectory stateDirectory = new StateDirectory(config, mockTime, true, false);
|
||||||
private final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder();
|
private final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder();
|
||||||
private final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder);
|
private final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder);
|
||||||
private final long defaultMaxBufferSizeInBytes = 512 * 1024 * 1024;
|
|
||||||
|
|
||||||
private StreamsMetadataState streamsMetadataState;
|
private StreamsMetadataState streamsMetadataState;
|
||||||
private final static BiConsumer<Throwable, Boolean> HANDLER = (e, b) -> {
|
private final static BiConsumer<Throwable, Boolean> HANDLER = (e, b) -> {
|
||||||
|
@ -204,6 +202,7 @@ public class StreamThreadTest {
|
||||||
return mkProperties(mkMap(
|
return mkProperties(mkMap(
|
||||||
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID),
|
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID),
|
||||||
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"),
|
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"),
|
||||||
|
mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"),
|
||||||
mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()),
|
mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()),
|
||||||
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()),
|
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()),
|
||||||
mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, enableEoS ? StreamsConfig.EXACTLY_ONCE_V2 : StreamsConfig.AT_LEAST_ONCE),
|
mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, enableEoS ? StreamsConfig.EXACTLY_ONCE_V2 : StreamsConfig.AT_LEAST_ONCE),
|
||||||
|
@ -254,7 +253,6 @@ public class StreamThreadTest {
|
||||||
mockTime,
|
mockTime,
|
||||||
streamsMetadataState,
|
streamsMetadataState,
|
||||||
0,
|
0,
|
||||||
defaultMaxBufferSizeInBytes,
|
|
||||||
stateDirectory,
|
stateDirectory,
|
||||||
new MockStateRestoreListener(),
|
new MockStateRestoreListener(),
|
||||||
threadIdx,
|
threadIdx,
|
||||||
|
@ -533,7 +531,6 @@ public class StreamThreadTest {
|
||||||
mockTime,
|
mockTime,
|
||||||
streamsMetadataState,
|
streamsMetadataState,
|
||||||
0,
|
0,
|
||||||
defaultMaxBufferSizeInBytes,
|
|
||||||
stateDirectory,
|
stateDirectory,
|
||||||
new MockStateRestoreListener(),
|
new MockStateRestoreListener(),
|
||||||
threadIdx,
|
threadIdx,
|
||||||
|
@ -1204,8 +1201,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
HANDLER,
|
HANDLER,
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID));
|
).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID));
|
||||||
|
|
||||||
final StreamsException thrown = assertThrows(StreamsException.class, thread::run);
|
final StreamsException thrown = assertThrows(StreamsException.class, thread::run);
|
||||||
|
@ -1593,7 +1589,6 @@ public class StreamThreadTest {
|
||||||
mockTime,
|
mockTime,
|
||||||
streamsMetadataState,
|
streamsMetadataState,
|
||||||
0,
|
0,
|
||||||
defaultMaxBufferSizeInBytes,
|
|
||||||
stateDirectory,
|
stateDirectory,
|
||||||
new MockStateRestoreListener(),
|
new MockStateRestoreListener(),
|
||||||
threadIdx,
|
threadIdx,
|
||||||
|
@ -2343,8 +2338,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
HANDLER,
|
HANDLER,
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
) {
|
) {
|
||||||
@Override
|
@Override
|
||||||
void runOnce() {
|
void runOnce() {
|
||||||
|
@ -2411,8 +2405,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
HANDLER,
|
HANDLER,
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
) {
|
) {
|
||||||
@Override
|
@Override
|
||||||
void runOnce() {
|
void runOnce() {
|
||||||
|
@ -2487,8 +2480,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
HANDLER,
|
HANDLER,
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
) {
|
) {
|
||||||
@Override
|
@Override
|
||||||
void runOnce() {
|
void runOnce() {
|
||||||
|
@ -2558,8 +2550,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
HANDLER,
|
HANDLER,
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
) {
|
) {
|
||||||
@Override
|
@Override
|
||||||
void runOnce() {
|
void runOnce() {
|
||||||
|
@ -2627,8 +2618,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
HANDLER,
|
HANDLER,
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
) {
|
) {
|
||||||
@Override
|
@Override
|
||||||
void runOnce() {
|
void runOnce() {
|
||||||
|
@ -2781,205 +2771,6 @@ public class StreamThreadTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldPauseNonEmptyPartitionsWhenTotalBufferSizeExceedsMaxBufferSize() {
|
|
||||||
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
|
|
||||||
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
|
||||||
expect(consumer.groupMetadata()).andStubReturn(consumerGroupMetadata);
|
|
||||||
expect(consumerGroupMetadata.groupInstanceId()).andReturn(Optional.empty());
|
|
||||||
|
|
||||||
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> records = new HashMap<>();
|
|
||||||
final List<TopicPartition> assignedPartitions = Collections.singletonList(t1p1);
|
|
||||||
consumer.assign(assignedPartitions);
|
|
||||||
records.put(t1p1, Collections.singletonList(new ConsumerRecord<>(
|
|
||||||
t1p1.topic(),
|
|
||||||
t1p1.partition(),
|
|
||||||
1,
|
|
||||||
mockTime.milliseconds(),
|
|
||||||
TimestampType.CREATE_TIME,
|
|
||||||
2,
|
|
||||||
6,
|
|
||||||
new byte[2],
|
|
||||||
new byte[6],
|
|
||||||
new RecordHeaders(),
|
|
||||||
Optional.empty())));
|
|
||||||
expect(consumer.poll(anyObject())).andReturn(new ConsumerRecords<>(records)).anyTimes();
|
|
||||||
EasyMock.replay(consumer, consumerGroupMetadata);
|
|
||||||
final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class);
|
|
||||||
|
|
||||||
final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>());
|
|
||||||
final Metric testMetric = new KafkaMetric(
|
|
||||||
new Object(),
|
|
||||||
testMetricName,
|
|
||||||
(Measurable) (config, now) -> 0,
|
|
||||||
null,
|
|
||||||
new MockTime());
|
|
||||||
final Map<MetricName, Metric> dummyProducerMetrics = singletonMap(testMetricName, testMetric);
|
|
||||||
|
|
||||||
expect(taskManager.producerMetrics()).andReturn(dummyProducerMetrics);
|
|
||||||
EasyMock.replay(taskManager);
|
|
||||||
|
|
||||||
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
|
|
||||||
topologyMetadata.buildAndRewriteTopology();
|
|
||||||
|
|
||||||
final StreamsMetricsImpl streamsMetrics =
|
|
||||||
new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime);
|
|
||||||
final StreamThread thread = new StreamThread(
|
|
||||||
mockTime,
|
|
||||||
config,
|
|
||||||
null,
|
|
||||||
consumer,
|
|
||||||
consumer,
|
|
||||||
changelogReader,
|
|
||||||
null,
|
|
||||||
taskManager,
|
|
||||||
streamsMetrics,
|
|
||||||
topologyMetadata,
|
|
||||||
CLIENT_ID,
|
|
||||||
new LogContext(""),
|
|
||||||
new AtomicInteger(),
|
|
||||||
new AtomicLong(Long.MAX_VALUE),
|
|
||||||
new LinkedList<>(),
|
|
||||||
null,
|
|
||||||
HANDLER,
|
|
||||||
null,
|
|
||||||
10
|
|
||||||
);
|
|
||||||
thread.setState(StreamThread.State.STARTING);
|
|
||||||
thread.setState(StreamThread.State.PARTITIONS_ASSIGNED);
|
|
||||||
thread.pollPhase();
|
|
||||||
thread.setState(StreamThread.State.PARTITIONS_REVOKED);
|
|
||||||
thread.pollPhase();
|
|
||||||
EasyMock.reset(consumer);
|
|
||||||
consumer.pause(anyObject());
|
|
||||||
// Consumer.pause should be called only once, when we added the second record.
|
|
||||||
EasyMock.expectLastCall().times(1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldResumePartitionsAfterConsumptionWhenTotalBufferSizeIsLTEMaxBufferSize() {
|
|
||||||
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
|
|
||||||
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
|
|
||||||
final ChangelogReader changelogReader = EasyMock.createNiceMock(ChangelogReader.class);
|
|
||||||
expect(consumer.groupMetadata()).andStubReturn(consumerGroupMetadata);
|
|
||||||
expect(consumerGroupMetadata.groupInstanceId()).andReturn(Optional.empty());
|
|
||||||
changelogReader.restore(anyObject());
|
|
||||||
expectLastCall().andVoid();
|
|
||||||
|
|
||||||
final Task task1 = mock(Task.class);
|
|
||||||
final Task task2 = mock(Task.class);
|
|
||||||
|
|
||||||
final TaskId taskId1 = new TaskId(0, 1);
|
|
||||||
final TaskId taskId2 = new TaskId(0, 2);
|
|
||||||
|
|
||||||
expect(task1.state()).andReturn(Task.State.RUNNING).anyTimes();
|
|
||||||
expect(task1.id()).andReturn(taskId1).anyTimes();
|
|
||||||
expect(task1.inputPartitions()).andReturn(mkSet(t1p1)).anyTimes();
|
|
||||||
expect(task1.committedOffsets()).andReturn(new HashMap<>()).anyTimes();
|
|
||||||
expect(task1.highWaterMark()).andReturn(new HashMap<>()).anyTimes();
|
|
||||||
expect(task1.timeCurrentIdlingStarted()).andReturn(Optional.empty()).anyTimes();
|
|
||||||
|
|
||||||
expect(task2.state()).andReturn(Task.State.RUNNING).anyTimes();
|
|
||||||
expect(task2.id()).andReturn(taskId2).anyTimes();
|
|
||||||
expect(task2.inputPartitions()).andReturn(mkSet(t1p2)).anyTimes();
|
|
||||||
expect(task2.committedOffsets()).andReturn(new HashMap<>()).anyTimes();
|
|
||||||
expect(task2.highWaterMark()).andReturn(new HashMap<>()).anyTimes();
|
|
||||||
expect(task2.timeCurrentIdlingStarted()).andReturn(Optional.empty()).anyTimes();
|
|
||||||
EasyMock.replay(task1, task2);
|
|
||||||
|
|
||||||
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> records = new HashMap<>();
|
|
||||||
records.put(t1p1, Collections.singletonList(new ConsumerRecord<>(
|
|
||||||
t1p1.topic(),
|
|
||||||
t1p1.partition(),
|
|
||||||
1,
|
|
||||||
mockTime.milliseconds(),
|
|
||||||
TimestampType.CREATE_TIME,
|
|
||||||
2,
|
|
||||||
6,
|
|
||||||
new byte[2],
|
|
||||||
new byte[6],
|
|
||||||
new RecordHeaders(),
|
|
||||||
Optional.empty())));
|
|
||||||
records.put(t1p2, Collections.singletonList(new ConsumerRecord<>(
|
|
||||||
t1p2.topic(),
|
|
||||||
t1p2.partition(),
|
|
||||||
1,
|
|
||||||
mockTime.milliseconds(),
|
|
||||||
TimestampType.CREATE_TIME,
|
|
||||||
2,
|
|
||||||
6,
|
|
||||||
new byte[2],
|
|
||||||
new byte[6],
|
|
||||||
new RecordHeaders(),
|
|
||||||
Optional.empty())));
|
|
||||||
|
|
||||||
final List<TopicPartition> assignedPartitions = Arrays.asList(t1p1, t1p2);
|
|
||||||
consumer.assign(assignedPartitions);
|
|
||||||
expect(consumer.poll(anyObject())).andReturn(new ConsumerRecords<>(records));
|
|
||||||
EasyMock.replay(consumer, consumerGroupMetadata);
|
|
||||||
|
|
||||||
final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class);
|
|
||||||
|
|
||||||
final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>());
|
|
||||||
final Metric testMetric = new KafkaMetric(
|
|
||||||
new Object(),
|
|
||||||
testMetricName,
|
|
||||||
(Measurable) (config, now) -> 0,
|
|
||||||
null,
|
|
||||||
new MockTime());
|
|
||||||
final Map<MetricName, Metric> dummyProducerMetrics = singletonMap(testMetricName, testMetric);
|
|
||||||
expect(taskManager.producerClientIds()).andStubReturn(Collections.emptySet());
|
|
||||||
expect(taskManager.producerMetrics()).andReturn(dummyProducerMetrics);
|
|
||||||
expect(taskManager.activeTaskMap()).andReturn(mkMap(
|
|
||||||
mkEntry(taskId1, task1),
|
|
||||||
mkEntry(taskId2, task2)
|
|
||||||
));
|
|
||||||
expect(taskManager.tasks()).andStubReturn(mkMap(
|
|
||||||
mkEntry(taskId1, task1),
|
|
||||||
mkEntry(taskId2, task2)
|
|
||||||
));
|
|
||||||
expect(taskManager.standbyTaskMap()).andReturn(new HashMap<>());
|
|
||||||
expect(taskManager.commit(anyObject())).andReturn(0);
|
|
||||||
expect(taskManager.process(anyInt(), anyObject())).andReturn(1);
|
|
||||||
expect(taskManager.process(anyInt(), anyObject())).andReturn(1);
|
|
||||||
expect(taskManager.process(anyInt(), anyObject())).andReturn(0);
|
|
||||||
|
|
||||||
EasyMock.replay(taskManager);
|
|
||||||
|
|
||||||
final StreamsMetricsImpl streamsMetrics =
|
|
||||||
new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime);
|
|
||||||
final StreamThread thread = new StreamThread(
|
|
||||||
mockTime,
|
|
||||||
new StreamsConfig(configProps(true)),
|
|
||||||
null,
|
|
||||||
consumer,
|
|
||||||
consumer,
|
|
||||||
changelogReader,
|
|
||||||
null,
|
|
||||||
taskManager,
|
|
||||||
streamsMetrics,
|
|
||||||
new TopologyMetadata(internalTopologyBuilder, config),
|
|
||||||
CLIENT_ID,
|
|
||||||
new LogContext(""),
|
|
||||||
new AtomicInteger(),
|
|
||||||
new AtomicLong(Long.MAX_VALUE),
|
|
||||||
new LinkedList<>(),
|
|
||||||
null,
|
|
||||||
HANDLER,
|
|
||||||
null,
|
|
||||||
6
|
|
||||||
).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID));
|
|
||||||
thread.setState(StreamThread.State.STARTING);
|
|
||||||
thread.setState(StreamThread.State.PARTITIONS_ASSIGNED);
|
|
||||||
thread.setState(StreamThread.State.RUNNING);
|
|
||||||
|
|
||||||
thread.runOnce();
|
|
||||||
EasyMock.reset(consumer);
|
|
||||||
consumer.resume(anyObject());
|
|
||||||
// Consumer.resume should be called only once, when we added the second record.
|
|
||||||
EasyMock.expectLastCall().times(1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldTransmitTaskManagerMetrics() {
|
public void shouldTransmitTaskManagerMetrics() {
|
||||||
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
|
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
|
||||||
|
@ -3048,8 +2839,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
HANDLER,
|
HANDLER,
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
);
|
);
|
||||||
final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>());
|
final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>());
|
||||||
final Metric testMetric = new KafkaMetric(
|
final Metric testMetric = new KafkaMetric(
|
||||||
|
@ -3106,8 +2896,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
(e, b) -> { },
|
(e, b) -> { },
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
) {
|
) {
|
||||||
@Override
|
@Override
|
||||||
void runOnce() {
|
void runOnce() {
|
||||||
|
@ -3251,8 +3040,7 @@ public class StreamThreadTest {
|
||||||
new LinkedList<>(),
|
new LinkedList<>(),
|
||||||
null,
|
null,
|
||||||
HANDLER,
|
HANDLER,
|
||||||
null,
|
null
|
||||||
defaultMaxBufferSizeInBytes
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -91,52 +91,6 @@ public class TaskMetricsTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldGetTotalBytesSensor() {
|
|
||||||
final String operation = "input-buffer-bytes-total";
|
|
||||||
when(streamsMetrics.taskLevelSensor(THREAD_ID, TASK_ID, operation, RecordingLevel.INFO))
|
|
||||||
.thenReturn(expectedSensor);
|
|
||||||
final String totalBytesDescription = "The total number of bytes accumulated in this task's input buffer";
|
|
||||||
when(streamsMetrics.taskLevelTagMap(THREAD_ID, TASK_ID)).thenReturn(tagMap);
|
|
||||||
|
|
||||||
try (final MockedStatic<StreamsMetricsImpl> streamsMetricsStaticMock = mockStatic(StreamsMetricsImpl.class)) {
|
|
||||||
final Sensor sensor = TaskMetrics.totalInputBufferBytesSensor(THREAD_ID, TASK_ID, streamsMetrics);
|
|
||||||
streamsMetricsStaticMock.verify(
|
|
||||||
() -> StreamsMetricsImpl.addValueMetricToSensor(
|
|
||||||
expectedSensor,
|
|
||||||
TASK_LEVEL_GROUP,
|
|
||||||
tagMap,
|
|
||||||
operation,
|
|
||||||
totalBytesDescription
|
|
||||||
)
|
|
||||||
);
|
|
||||||
assertThat(sensor, is(expectedSensor));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void shouldGetTotalCacheSizeInBytesSensor() {
|
|
||||||
final String operation = "cache-size-bytes-total";
|
|
||||||
when(streamsMetrics.taskLevelSensor(THREAD_ID, TASK_ID, operation, RecordingLevel.INFO))
|
|
||||||
.thenReturn(expectedSensor);
|
|
||||||
final String totalBytesDescription = "The total size in bytes of this task's cache.";
|
|
||||||
when(streamsMetrics.taskLevelTagMap(THREAD_ID, TASK_ID)).thenReturn(tagMap);
|
|
||||||
|
|
||||||
try (final MockedStatic<StreamsMetricsImpl> streamsMetricsStaticMock = mockStatic(StreamsMetricsImpl.class)) {
|
|
||||||
final Sensor sensor = TaskMetrics.totalCacheSizeBytesSensor(THREAD_ID, TASK_ID, streamsMetrics);
|
|
||||||
streamsMetricsStaticMock.verify(
|
|
||||||
() -> StreamsMetricsImpl.addValueMetricToSensor(
|
|
||||||
expectedSensor,
|
|
||||||
TASK_LEVEL_GROUP,
|
|
||||||
tagMap,
|
|
||||||
operation,
|
|
||||||
totalBytesDescription
|
|
||||||
)
|
|
||||||
);
|
|
||||||
assertThat(sensor, is(expectedSensor));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldGetProcessLatencySensor() {
|
public void shouldGetProcessLatencySensor() {
|
||||||
final String operation = "process-latency";
|
final String operation = "process-latency";
|
||||||
|
@ -299,4 +253,4 @@ public class TaskMetricsTest {
|
||||||
assertThat(sensor, is(expectedSensor));
|
assertThat(sensor, is(expectedSensor));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,7 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.streams.state.internals;
|
package org.apache.kafka.streams.state.internals;
|
||||||
|
|
||||||
import org.apache.kafka.common.MetricName;
|
|
||||||
import org.apache.kafka.common.header.Header;
|
import org.apache.kafka.common.header.Header;
|
||||||
import org.apache.kafka.common.header.Headers;
|
import org.apache.kafka.common.header.Headers;
|
||||||
import org.apache.kafka.common.header.internals.RecordHeader;
|
import org.apache.kafka.common.header.internals.RecordHeader;
|
||||||
|
@ -33,10 +32,6 @@ import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
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.hamcrest.CoreMatchers.is;
|
|
||||||
import static org.hamcrest.MatcherAssert.assertThat;
|
|
||||||
import static org.junit.Assert.assertArrayEquals;
|
import static org.junit.Assert.assertArrayEquals;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertNotNull;
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
@ -48,12 +43,11 @@ public class NamedCacheTest {
|
||||||
|
|
||||||
private final Headers headers = new RecordHeaders(new Header[]{new RecordHeader("key", "value".getBytes())});
|
private final Headers headers = new RecordHeaders(new Header[]{new RecordHeader("key", "value".getBytes())});
|
||||||
private NamedCache cache;
|
private NamedCache cache;
|
||||||
private final Metrics innerMetrics = new Metrics();
|
|
||||||
private final StreamsMetricsImpl metrics = new MockStreamsMetrics(innerMetrics);
|
|
||||||
private final MetricName cacheSizeBytesTotal = new MetricName("cache-size-bytes-total", "stream-task-metrics", "", mkMap(mkEntry("thread-id", "Test worker"), mkEntry("task-id", "dummy")));
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
|
final Metrics innerMetrics = new Metrics();
|
||||||
|
final StreamsMetricsImpl metrics = new MockStreamsMetrics(innerMetrics);
|
||||||
cache = new NamedCache("dummy-name", metrics);
|
cache = new NamedCache("dummy-name", metrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -88,7 +82,6 @@ public class NamedCacheTest {
|
||||||
cache.put(Bytes.wrap(new byte[]{1}), value);
|
cache.put(Bytes.wrap(new byte[]{1}), value);
|
||||||
cache.put(Bytes.wrap(new byte[]{2}), value);
|
cache.put(Bytes.wrap(new byte[]{2}), value);
|
||||||
final long size = cache.sizeInBytes();
|
final long size = cache.sizeInBytes();
|
||||||
assertThat(metrics.metrics().get(cacheSizeBytesTotal).metricValue(), is((double) size));
|
|
||||||
// 1 byte key + 24 bytes overhead
|
// 1 byte key + 24 bytes overhead
|
||||||
assertEquals((value.size() + 25) * 3, size);
|
assertEquals((value.size() + 25) * 3, size);
|
||||||
}
|
}
|
||||||
|
@ -121,7 +114,6 @@ public class NamedCacheTest {
|
||||||
final LRUCacheEntry deleted = cache.delete(Bytes.wrap(new byte[]{0}));
|
final LRUCacheEntry deleted = cache.delete(Bytes.wrap(new byte[]{0}));
|
||||||
assertArrayEquals(new byte[] {10}, deleted.value());
|
assertArrayEquals(new byte[] {10}, deleted.value());
|
||||||
assertEquals(0, cache.sizeInBytes());
|
assertEquals(0, cache.sizeInBytes());
|
||||||
assertThat(metrics.metrics().get(cacheSizeBytesTotal).metricValue(), is((double) 0));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -72,7 +72,7 @@ public class BrokerCompatibilityTest {
|
||||||
streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsProperties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsProperties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsProperties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingMode);
|
streamsProperties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingMode);
|
||||||
final int timeout = 6000;
|
final int timeout = 6000;
|
||||||
streamsProperties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), timeout);
|
streamsProperties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), timeout);
|
||||||
|
|
|
@ -106,7 +106,7 @@ public class EosTestClient extends SmokeTestUtil {
|
||||||
props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, Duration.ofMinutes(1).toMillis());
|
props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, Duration.ofMinutes(1).toMillis());
|
||||||
props.put(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, Integer.MAX_VALUE);
|
props.put(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, Integer.MAX_VALUE);
|
||||||
props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3);
|
props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3);
|
||||||
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5000L); // increase commit interval to make sure a client is killed having an open transaction
|
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5000L); // increase commit interval to make sure a client is killed having an open transaction
|
||||||
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||||
|
|
|
@ -87,7 +87,7 @@ public class StreamsNamedRepartitionTest {
|
||||||
final Properties config = new Properties();
|
final Properties config = new Properties();
|
||||||
|
|
||||||
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsNamedRepartitionTest");
|
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsNamedRepartitionTest");
|
||||||
config.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, "0");
|
config.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, "0");
|
||||||
config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
|
|
||||||
|
|
|
@ -110,7 +110,7 @@ public class StreamsOptimizedTest {
|
||||||
|
|
||||||
|
|
||||||
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsOptimizedTest");
|
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsOptimizedTest");
|
||||||
config.setProperty(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, "0");
|
config.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, "0");
|
||||||
config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
|
||||||
config.setProperty(StreamsConfig.adminClientPrefix(AdminClientConfig.RETRIES_CONFIG), "100");
|
config.setProperty(StreamsConfig.adminClientPrefix(AdminClientConfig.RETRIES_CONFIG), "100");
|
||||||
|
|
|
@ -67,7 +67,7 @@ public class StreamsStandByReplicaTest {
|
||||||
streamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, "kafka-streams-standby-tasks");
|
streamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, "kafka-streams-standby-tasks");
|
||||||
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
streamsProperties.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
|
streamsProperties.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
|
||||||
streamsProperties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
|
streamsProperties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
|
||||||
streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
streamsProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||||
streamsProperties.put(StreamsConfig.producerPrefix(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG), true);
|
streamsProperties.put(StreamsConfig.producerPrefix(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG), true);
|
||||||
|
|
|
@ -117,7 +117,6 @@ import java.util.regex.Pattern;
|
||||||
import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE;
|
import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE;
|
||||||
import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_ALPHA;
|
import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_ALPHA;
|
||||||
import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_V2;
|
import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_V2;
|
||||||
import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
|
|
||||||
import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
|
import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -156,7 +155,7 @@ import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
|
||||||
*
|
*
|
||||||
* <p> Note that the {@code TopologyTestDriver} processes input records synchronously.
|
* <p> Note that the {@code TopologyTestDriver} processes input records synchronously.
|
||||||
* This implies that {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit.interval.ms} and
|
* This implies that {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit.interval.ms} and
|
||||||
* {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache.max.bytes.buffering} configuration have no effect.
|
* {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache.max.bytes.buffering} configuration have no effect.
|
||||||
* The driver behaves as if both configs would be set to zero, i.e., as if a "commit" (and thus "flush") would happen
|
* The driver behaves as if both configs would be set to zero, i.e., as if a "commit" (and thus "flush") would happen
|
||||||
* after each input record.
|
* after each input record.
|
||||||
*
|
*
|
||||||
|
@ -310,7 +309,6 @@ public class TopologyTestDriver implements Closeable {
|
||||||
* @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({"unchecked", "deprecation"})
|
|
||||||
private TopologyTestDriver(final InternalTopologyBuilder builder,
|
private TopologyTestDriver(final InternalTopologyBuilder builder,
|
||||||
final Properties config,
|
final Properties config,
|
||||||
final long initialWallClockTimeMs) {
|
final long initialWallClockTimeMs) {
|
||||||
|
@ -331,7 +329,7 @@ public class TopologyTestDriver implements Closeable {
|
||||||
|
|
||||||
final ThreadCache cache = new ThreadCache(
|
final ThreadCache cache = new ThreadCache(
|
||||||
logContext,
|
logContext,
|
||||||
Math.max(0, getTotalCacheSize(streamsConfig)),
|
Math.max(0, streamsConfig.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG)),
|
||||||
streamsMetrics
|
streamsMetrics
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue