From e553083ce470a4bafd8eaae40e8af63b925e2242 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Wed, 4 Aug 2021 22:29:34 -0700 Subject: [PATCH 01/18] WIP --- .../internals/ActiveTaskCreator.java | 22 +++++----- .../processor/internals/StandbyTask.java | 8 ++-- .../internals/StandbyTaskCreator.java | 14 ++++--- .../processor/internals/StreamTask.java | 14 ++++--- .../processor/internals/TopologyMetadata.java | 37 ++++++++++++++++- .../KafkaStreamsNamedTopologyWrapper.java | 22 +++++++--- .../internals/namedtopology/TaskConfig.java | 41 +++++++++++++++++++ .../NamedTopologyIntegrationTest.java | 2 +- 8 files changed, 126 insertions(+), 34 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TaskConfig.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java index 75ec24c686aad..4a156a7bde47b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java @@ -30,6 +30,7 @@ import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; +import org.apache.kafka.streams.processor.internals.namedtopology.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.slf4j.Logger; @@ -51,7 +52,7 @@ class ActiveTaskCreator { private final TopologyMetadata topologyMetadata; - private final StreamsConfig config; + private final StreamsConfig applicationConfig; private final StreamsMetricsImpl streamsMetrics; private final StateDirectory stateDirectory; private final ChangelogReader storeChangelogReader; @@ -71,7 +72,7 @@ class ActiveTaskCreator { private final Map> unknownTasksToBeCreated = new HashMap<>(); ActiveTaskCreator(final TopologyMetadata topologyMetadata, - final StreamsConfig config, + final StreamsConfig applicationConfig, final StreamsMetricsImpl streamsMetrics, final StateDirectory stateDirectory, final ChangelogReader storeChangelogReader, @@ -82,7 +83,7 @@ class ActiveTaskCreator { final UUID processId, final Logger log) { this.topologyMetadata = topologyMetadata; - this.config = config; + this.applicationConfig = applicationConfig; this.streamsMetrics = streamsMetrics; this.stateDirectory = stateDirectory; this.storeChangelogReader = storeChangelogReader; @@ -93,7 +94,7 @@ class ActiveTaskCreator { this.log = log; createTaskSensor = ThreadMetrics.createTaskSensor(threadId, streamsMetrics); - processingMode = StreamThread.processingMode(config); + processingMode = StreamThread.processingMode(applicationConfig); if (processingMode == EXACTLY_ONCE_ALPHA) { threadProducer = null; @@ -105,7 +106,7 @@ class ActiveTaskCreator { final LogContext logContext = new LogContext(threadIdPrefix); threadProducer = new StreamsProducer( - config, + applicationConfig, threadId, clientSupplier, null, @@ -170,7 +171,7 @@ Collection createTasks(final Consumer consumer, final ProcessorStateManager stateManager = new ProcessorStateManager( taskId, Task.TaskType.ACTIVE, - StreamThread.eosEnabled(config), + StreamThread.eosEnabled(applicationConfig), logContext, stateDirectory, storeChangelogReader, @@ -180,7 +181,7 @@ Collection createTasks(final Consumer consumer, final InternalProcessorContext context = new ProcessorContextImpl( taskId, - config, + applicationConfig, stateManager, streamsMetrics, cache @@ -239,7 +240,7 @@ private StreamTask createActiveTask(final TaskId taskId, if (processingMode == StreamThread.ProcessingMode.EXACTLY_ONCE_ALPHA) { log.info("Creating producer client for task {}", taskId); streamsProducer = new StreamsProducer( - config, + applicationConfig, threadId, clientSupplier, taskId, @@ -255,7 +256,7 @@ private StreamTask createActiveTask(final TaskId taskId, logContext, taskId, streamsProducer, - config.defaultProductionExceptionHandler(), + applicationConfig.defaultProductionExceptionHandler(), streamsMetrics ); @@ -264,7 +265,8 @@ private StreamTask createActiveTask(final TaskId taskId, inputPartitions, topology, consumer, - config, + new TaskConfig(applicationConfig, topologyMetadata.getConfigForTask(taskId)), + StreamThread.eosEnabled(applicationConfig), streamsMetrics, stateDirectory, cache, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index 802bca1624827..c028e737d9efe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -27,6 +27,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; +import org.apache.kafka.streams.processor.internals.namedtopology.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import java.util.Collections; @@ -55,7 +56,8 @@ public class StandbyTask extends AbstractTask implements Task { StandbyTask(final TaskId id, final Set inputPartitions, final ProcessorTopology topology, - final StreamsConfig config, + final TaskConfig config, + final boolean eosEnabled, final StreamsMetricsImpl streamsMetrics, final ProcessorStateManager stateMgr, final StateDirectory stateDirectory, @@ -67,7 +69,7 @@ public class StandbyTask extends AbstractTask implements Task { stateDirectory, stateMgr, inputPartitions, - config.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG), + config.maxTaskIdleMs, "standby-task", StandbyTask.class ); @@ -76,7 +78,7 @@ public class StandbyTask extends AbstractTask implements Task { processorContext.transitionToStandby(cache); closeTaskSensor = ThreadMetrics.closeTaskSensor(Thread.currentThread().getName(), streamsMetrics); - eosEnabled = StreamThread.eosEnabled(config); + this.eosEnabled = eosEnabled; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java index ee2a3e1938060..6251d97cd0366 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; +import org.apache.kafka.streams.processor.internals.namedtopology.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.slf4j.Logger; @@ -38,7 +39,7 @@ class StandbyTaskCreator { private final TopologyMetadata topologyMetadata; - private final StreamsConfig config; + private final StreamsConfig applicationConfig; private final StreamsMetricsImpl streamsMetrics; private final StateDirectory stateDirectory; private final ChangelogReader storeChangelogReader; @@ -50,14 +51,14 @@ class StandbyTaskCreator { private final Map> unknownTasksToBeCreated = new HashMap<>(); StandbyTaskCreator(final TopologyMetadata topologyMetadata, - final StreamsConfig config, + final StreamsConfig applicationConfig, final StreamsMetricsImpl streamsMetrics, final StateDirectory stateDirectory, final ChangelogReader storeChangelogReader, final String threadId, final Logger log) { this.topologyMetadata = topologyMetadata; - this.config = config; + this.applicationConfig = applicationConfig; this.streamsMetrics = streamsMetrics; this.stateDirectory = stateDirectory; this.storeChangelogReader = storeChangelogReader; @@ -101,7 +102,7 @@ Collection createTasks(final Map> tasksToBeCre final ProcessorStateManager stateManager = new ProcessorStateManager( taskId, Task.TaskType.STANDBY, - StreamThread.eosEnabled(config), + StreamThread.eosEnabled(applicationConfig), getLogContext(taskId), stateDirectory, storeChangelogReader, @@ -111,7 +112,7 @@ Collection createTasks(final Map> tasksToBeCre final InternalProcessorContext context = new ProcessorContextImpl( taskId, - config, + applicationConfig, stateManager, streamsMetrics, dummyCache @@ -160,7 +161,8 @@ StandbyTask createStandbyTask(final TaskId taskId, taskId, inputPartitions, topology, - config, + new TaskConfig(applicationConfig, topologyMetadata.getConfigForTask(taskId)), + StreamThread.eosEnabled(applicationConfig), streamsMetrics, stateManager, stateDirectory, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 7c858696975fe..660908a1a1d59 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -44,6 +44,7 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; +import org.apache.kafka.streams.processor.internals.namedtopology.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import java.io.IOException; @@ -117,7 +118,8 @@ public StreamTask(final TaskId id, final Set inputPartitions, final ProcessorTopology topology, final Consumer mainConsumer, - final StreamsConfig config, + final TaskConfig config, + final boolean eosEnabled, final StreamsMetricsImpl streamsMetrics, final StateDirectory stateDirectory, final ThreadCache cache, @@ -132,7 +134,7 @@ public StreamTask(final TaskId id, stateDirectory, stateMgr, inputPartitions, - config.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG), + config.taskTimeoutMs, "task", StreamTask.class ); @@ -143,7 +145,7 @@ public StreamTask(final TaskId id, this.time = time; this.recordCollector = recordCollector; - eosEnabled = StreamThread.eosEnabled(config); + this.eosEnabled = eosEnabled; final String threadId = Thread.currentThread().getName(); this.streamsMetrics = streamsMetrics; @@ -171,19 +173,19 @@ public StreamTask(final TaskId id, streamTimePunctuationQueue = new PunctuationQueue(); systemTimePunctuationQueue = new PunctuationQueue(); - maxBufferedSize = config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + maxBufferedSize = config.maxBufferedSize; // initialize the consumed and committed offset cache consumedOffsets = new HashMap<>(); resetOffsetsForPartitions = new HashSet<>(); - recordQueueCreator = new RecordQueueCreator(this.logContext, config.defaultTimestampExtractor(), config.defaultDeserializationExceptionHandler()); + recordQueueCreator = new RecordQueueCreator(this.logContext, config.timestampExtractor, config.deserializationExceptionHandler); recordInfo = new PartitionGroup.RecordInfo(); final Sensor enforcedProcessingSensor; enforcedProcessingSensor = TaskMetrics.enforcedProcessingSensor(threadId, taskId, streamsMetrics); - final long maxTaskIdleMs = config.getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG); + final long maxTaskIdleMs = config.maxTaskIdleMs; partitionGroup = new PartitionGroup( logContext, createPartitionQueues(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java index 129ca09ad5c74..b05d263df33a9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java @@ -17,7 +17,10 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.internals.KafkaCompletableFuture; +import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.StateStore; @@ -30,12 +33,14 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -70,6 +75,20 @@ public static class TopologyVersion { public AtomicLong topologyVersion = new AtomicLong(0L); // the local topology version public ReentrantLock topologyLock = new ReentrantLock(); public Condition topologyCV = topologyLock.newCondition(); + // TODO KAFKA-12648: Pt.4 + public List activeTopologyWaiters = new LinkedList<>(); + } + + public static class TopologyVersionWaiters { + final long topologyVersion; // the (minimum) version to wait for these threads to cross + final Semaphore threadSemaphore; // semaphore to count threads initially below this version that have updated + final KafkaCompletableFuture future; // the future waiting on all threads to be updated + + public TopologyVersionWaiters(final Semaphore threadSemaphore, final long topologyVersion, final KafkaCompletableFuture future) { + this.topologyVersion = topologyVersion; + this.threadSemaphore = threadSemaphore; + this.future = future; + } } public TopologyMetadata(final InternalTopologyBuilder builder, @@ -107,6 +126,10 @@ private void unlock() { version.topologyLock.unlock(); } + public StreamsConfig getConfigForTask(final TaskId taskId) { + return lookupBuilderForTask(taskId).getStreamsConfig(); + } + public void wakeupThreads() { try { lock(); @@ -134,7 +157,11 @@ public void maybeWaitForNonEmptyTopology(final Supplier threadState) { } } - public void registerAndBuildNewTopology(final InternalTopologyBuilder newTopologyBuilder) { + /** + * Adds the topology and registers a future that listens for all threads on the older version to see the update + */ + public KafkaFuture registerAndBuildNewTopology(final InternalTopologyBuilder newTopologyBuilder) { + final KafkaFuture future = new KafkaFutureImpl<>(); try { lock(); version.topologyVersion.incrementAndGet(); @@ -145,9 +172,14 @@ public void registerAndBuildNewTopology(final InternalTopologyBuilder newTopolog } finally { unlock(); } + return future; } - public void unregisterTopology(final String topologyName) { + /** + * Removes the topology and registers a future that listens for all threads on the older version to see the update + */ + public KafkaFuture unregisterTopology(final String topologyName) { + final KafkaFuture future = new KafkaFutureImpl<>(); try { lock(); version.topologyVersion.incrementAndGet(); @@ -159,6 +191,7 @@ public void unregisterTopology(final String topologyName) { } finally { unlock(); } + return future; } public void buildAndRewriteTopology() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java index d4170d499c595..71ad45a98b1b4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.namedtopology; +import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.annotation.InterfaceStability.Unstable; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; @@ -121,40 +122,49 @@ public Optional getTopologyByName(final String name) { /** * Add a new NamedTopology to a running Kafka Streams app. If multiple instances of the application are running, - * you should inform all of them by calling {@link #addNamedTopology(NamedTopology)} on each client in order for + * you should inform all of them by calling {@code #addNamedTopology(NamedTopology)} on each client in order for * it to begin processing the new topology. * + * @return a {@link KafkaFuture} that completes successfully when all threads on this client have picked up the + * new {@link NamedTopology}. Note that the new topology is not guaranteed to begin processing on this client or + * any others until its addition has been completed by all instances of the application. + * * @throws IllegalArgumentException if this topology name is already in use * @throws IllegalStateException if streams has not been started or has already shut down * @throws TopologyException if this topology subscribes to any input topics or pattern already in use */ - public void addNamedTopology(final NamedTopology newTopology) { + public KafkaFuture addNamedTopology(final NamedTopology newTopology) { if (hasStartedOrFinishedShuttingDown()) { throw new IllegalStateException("Cannot add a NamedTopology while the state is " + super.state); } else if (getTopologyByName(newTopology.name()).isPresent()) { throw new IllegalArgumentException("Unable to add the new NamedTopology " + newTopology.name() + " as another of the same name already exists"); } - topologyMetadata.registerAndBuildNewTopology(newTopology.internalTopologyBuilder()); + return topologyMetadata.registerAndBuildNewTopology(newTopology.internalTopologyBuilder()); } /** * Remove an existing NamedTopology from a running Kafka Streams app. If multiple instances of the application are - * running, you should inform all of them by calling {@link #removeNamedTopology(String)} on each client to ensure + * running, you should inform all of them by calling {@code #removeNamedTopology(String)} on each client to ensure * it stops processing the old topology. * + * @return a {@link KafkaFuture} that completes successfully when all threads on this client have removed the + * corresponding {@link NamedTopology}. At this point no more of its tasks will be processed by the current client, + * but there may still be other clients which do. It is only guaranteed that this {@link NamedTopology} has fully + * stopped processing when all clients have successfully completed their corresponding {@link KafkaFuture}. + * * @throws IllegalArgumentException if this topology name cannot be found * @throws IllegalStateException if streams has not been started or has already shut down * @throws TopologyException if this topology subscribes to any input topics or pattern already in use */ - public void removeNamedTopology(final String topologyToRemove) { + public KafkaFuture removeNamedTopology(final String topologyToRemove) { if (!isRunningOrRebalancing()) { throw new IllegalStateException("Cannot remove a NamedTopology while the state is " + super.state); } else if (!getTopologyByName(topologyToRemove).isPresent()) { throw new IllegalArgumentException("Unable to locate for removal a NamedTopology called " + topologyToRemove); } - topologyMetadata.unregisterTopology(topologyToRemove); + return topologyMetadata.unregisterTopology(topologyToRemove); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TaskConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TaskConfig.java new file mode 100644 index 0000000000000..6c813a4e57cad --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TaskConfig.java @@ -0,0 +1,41 @@ +package org.apache.kafka.streams.processor.internals.namedtopology; + +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.processor.TimestampExtractor; + +import java.util.Properties; +import java.util.function.Function; + +/** + * Streams configs that apply at the task level. At the moment they can be specified either via the configs passed in + * to the {@link org.apache.kafka.streams.KafkaStreams} or {@link KafkaStreamsNamedTopologyWrapper} constructor, or + * else by passing them in to {@link org.apache.kafka.streams.StreamsBuilder#build(Properties)} or + * {@link NamedTopologyStreamsBuilder#buildNamedTopology(Properties)} + */ +public class TaskConfig { + public final long maxTaskIdleMs; + public final long taskTimeoutMs; + public final int maxBufferedSize; + public final TimestampExtractor timestampExtractor; + public final DeserializationExceptionHandler deserializationExceptionHandler; + + public TaskConfig(final StreamsConfig applicationConfig, final StreamsConfig topologyConfig) { + maxTaskIdleMs = getConfig(applicationConfig, topologyConfig, config -> config.getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG)); + taskTimeoutMs = getConfig(applicationConfig, topologyConfig, config -> config.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG)); + maxBufferedSize = getConfig(applicationConfig, topologyConfig, config -> config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)); + timestampExtractor = getConfig(applicationConfig, topologyConfig, StreamsConfig::defaultTimestampExtractor); + deserializationExceptionHandler = getConfig(applicationConfig, topologyConfig, StreamsConfig::defaultDeserializationExceptionHandler); + } + + /** + * @return the value of this config passed in to the topology if it exists, otherwise default application-wide config + */ + static V getConfig(final StreamsConfig applicationConfig, + final StreamsConfig topologyConfig, + final Function configGetter) { + final V topologyOverride = configGetter.apply(topologyConfig); + final V appValue = configGetter.apply(applicationConfig); + return topologyOverride == null ? appValue : topologyOverride; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java index 2b01feec691a1..ca2bfb150d548 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java @@ -73,7 +73,7 @@ public class NamedTopologyIntegrationTest { public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); - // TODO KAFKA-12648: + // TODO KAFKA-12648: Pt.4 // 1) full test coverage for add/removeNamedTopology, covering: // - the "last topology removed" case // - test using multiple clients, with standbys From 57b30c04baf466913bd8b81cf7afd9751d3edaaa Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 26 Aug 2021 17:00:45 -0700 Subject: [PATCH 02/18] pull out Pt. blocking API changes --- .../processor/internals/TopologyMetadata.java | 37 +------------------ .../KafkaStreamsNamedTopologyWrapper.java | 22 +++-------- .../NamedTopologyIntegrationTest.java | 2 +- 3 files changed, 9 insertions(+), 52 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java index b05d263df33a9..129ca09ad5c74 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java @@ -17,10 +17,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.internals.KafkaCompletableFuture; -import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.StateStore; @@ -33,14 +30,12 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -75,20 +70,6 @@ public static class TopologyVersion { public AtomicLong topologyVersion = new AtomicLong(0L); // the local topology version public ReentrantLock topologyLock = new ReentrantLock(); public Condition topologyCV = topologyLock.newCondition(); - // TODO KAFKA-12648: Pt.4 - public List activeTopologyWaiters = new LinkedList<>(); - } - - public static class TopologyVersionWaiters { - final long topologyVersion; // the (minimum) version to wait for these threads to cross - final Semaphore threadSemaphore; // semaphore to count threads initially below this version that have updated - final KafkaCompletableFuture future; // the future waiting on all threads to be updated - - public TopologyVersionWaiters(final Semaphore threadSemaphore, final long topologyVersion, final KafkaCompletableFuture future) { - this.topologyVersion = topologyVersion; - this.threadSemaphore = threadSemaphore; - this.future = future; - } } public TopologyMetadata(final InternalTopologyBuilder builder, @@ -126,10 +107,6 @@ private void unlock() { version.topologyLock.unlock(); } - public StreamsConfig getConfigForTask(final TaskId taskId) { - return lookupBuilderForTask(taskId).getStreamsConfig(); - } - public void wakeupThreads() { try { lock(); @@ -157,11 +134,7 @@ public void maybeWaitForNonEmptyTopology(final Supplier threadState) { } } - /** - * Adds the topology and registers a future that listens for all threads on the older version to see the update - */ - public KafkaFuture registerAndBuildNewTopology(final InternalTopologyBuilder newTopologyBuilder) { - final KafkaFuture future = new KafkaFutureImpl<>(); + public void registerAndBuildNewTopology(final InternalTopologyBuilder newTopologyBuilder) { try { lock(); version.topologyVersion.incrementAndGet(); @@ -172,14 +145,9 @@ public KafkaFuture registerAndBuildNewTopology(final InternalTopologyBuild } finally { unlock(); } - return future; } - /** - * Removes the topology and registers a future that listens for all threads on the older version to see the update - */ - public KafkaFuture unregisterTopology(final String topologyName) { - final KafkaFuture future = new KafkaFutureImpl<>(); + public void unregisterTopology(final String topologyName) { try { lock(); version.topologyVersion.incrementAndGet(); @@ -191,7 +159,6 @@ public KafkaFuture unregisterTopology(final String topologyName) { } finally { unlock(); } - return future; } public void buildAndRewriteTopology() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java index 71ad45a98b1b4..d4170d499c595 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals.namedtopology; -import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.annotation.InterfaceStability.Unstable; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; @@ -122,49 +121,40 @@ public Optional getTopologyByName(final String name) { /** * Add a new NamedTopology to a running Kafka Streams app. If multiple instances of the application are running, - * you should inform all of them by calling {@code #addNamedTopology(NamedTopology)} on each client in order for + * you should inform all of them by calling {@link #addNamedTopology(NamedTopology)} on each client in order for * it to begin processing the new topology. * - * @return a {@link KafkaFuture} that completes successfully when all threads on this client have picked up the - * new {@link NamedTopology}. Note that the new topology is not guaranteed to begin processing on this client or - * any others until its addition has been completed by all instances of the application. - * * @throws IllegalArgumentException if this topology name is already in use * @throws IllegalStateException if streams has not been started or has already shut down * @throws TopologyException if this topology subscribes to any input topics or pattern already in use */ - public KafkaFuture addNamedTopology(final NamedTopology newTopology) { + public void addNamedTopology(final NamedTopology newTopology) { if (hasStartedOrFinishedShuttingDown()) { throw new IllegalStateException("Cannot add a NamedTopology while the state is " + super.state); } else if (getTopologyByName(newTopology.name()).isPresent()) { throw new IllegalArgumentException("Unable to add the new NamedTopology " + newTopology.name() + " as another of the same name already exists"); } - return topologyMetadata.registerAndBuildNewTopology(newTopology.internalTopologyBuilder()); + topologyMetadata.registerAndBuildNewTopology(newTopology.internalTopologyBuilder()); } /** * Remove an existing NamedTopology from a running Kafka Streams app. If multiple instances of the application are - * running, you should inform all of them by calling {@code #removeNamedTopology(String)} on each client to ensure + * running, you should inform all of them by calling {@link #removeNamedTopology(String)} on each client to ensure * it stops processing the old topology. * - * @return a {@link KafkaFuture} that completes successfully when all threads on this client have removed the - * corresponding {@link NamedTopology}. At this point no more of its tasks will be processed by the current client, - * but there may still be other clients which do. It is only guaranteed that this {@link NamedTopology} has fully - * stopped processing when all clients have successfully completed their corresponding {@link KafkaFuture}. - * * @throws IllegalArgumentException if this topology name cannot be found * @throws IllegalStateException if streams has not been started or has already shut down * @throws TopologyException if this topology subscribes to any input topics or pattern already in use */ - public KafkaFuture removeNamedTopology(final String topologyToRemove) { + public void removeNamedTopology(final String topologyToRemove) { if (!isRunningOrRebalancing()) { throw new IllegalStateException("Cannot remove a NamedTopology while the state is " + super.state); } else if (!getTopologyByName(topologyToRemove).isPresent()) { throw new IllegalArgumentException("Unable to locate for removal a NamedTopology called " + topologyToRemove); } - return topologyMetadata.unregisterTopology(topologyToRemove); + topologyMetadata.unregisterTopology(topologyToRemove); } /** diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java index ca2bfb150d548..2b01feec691a1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java @@ -73,7 +73,7 @@ public class NamedTopologyIntegrationTest { public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); - // TODO KAFKA-12648: Pt.4 + // TODO KAFKA-12648: // 1) full test coverage for add/removeNamedTopology, covering: // - the "last topology removed" case // - test using multiple clients, with standbys From c74f990e466cbef1086e4b4c3088cea76232018c Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 26 Aug 2021 19:09:53 -0700 Subject: [PATCH 03/18] much refactoring to set us up for the future --- .../apache/kafka/streams/StreamsBuilder.java | 1 + .../apache/kafka/streams/StreamsConfig.java | 4 +- .../internals/ActiveTaskCreator.java | 3 +- .../internals/InternalTopologyBuilder.java | 21 ++++-- .../processor/internals/StandbyTask.java | 2 +- .../internals/StandbyTaskCreator.java | 3 +- .../processor/internals/StreamTask.java | 3 +- .../processor/internals/TopologyMetadata.java | 5 ++ .../internals/namedtopology/TaskConfig.java | 41 ----------- .../namedtopology/TopologyConfig.java | 73 +++++++++++++++++++ .../InternalTopologyBuilderTest.java | 22 +++++- .../processor/internals/StandbyTaskTest.java | 6 +- .../kafka/streams/TopologyTestDriver.java | 9 ++- 13 files changed, 130 insertions(+), 63 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TaskConfig.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index f10dc9356f783..00f7e67929321 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -611,6 +611,7 @@ public synchronized Topology build() { */ public synchronized Topology build(final Properties props) { internalStreamsBuilder.buildAndOptimizeTopology(props); + internalTopologyBuilder.setTopologyProperties(props); return topology; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 098539a46eb6d..9c41250afe9d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1075,8 +1075,8 @@ public StreamsConfig(final Map props) { this(props, true); } - protected StreamsConfig(final Map props, - final boolean doLog) { + public StreamsConfig(final Map props, + final boolean doLog) { super(CONFIG, props, doLog); eosEnabled = StreamThread.eosEnabled(this); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java index 4a156a7bde47b..b08a4d32ba081 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java @@ -30,7 +30,6 @@ import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; -import org.apache.kafka.streams.processor.internals.namedtopology.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.slf4j.Logger; @@ -265,7 +264,7 @@ private StreamTask createActiveTask(final TaskId taskId, inputPartitions, topology, consumer, - new TaskConfig(applicationConfig, topologyMetadata.getConfigForTask(taskId)), + topologyMetadata.getTaskConfigFor(taskId), StreamThread.eosEnabled(applicationConfig), streamsMetrics, stateDirectory, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index dd07c107e98a0..1f11df76d2176 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -31,6 +31,7 @@ import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.internals.SessionStoreBuilder; import org.apache.kafka.streams.state.internals.TimestampedWindowStoreBuilder; @@ -52,6 +53,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Properties; import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; @@ -133,7 +135,9 @@ public class InternalTopologyBuilder { private Map> nodeGroups = null; - private StreamsConfig config = null; + private StreamsConfig applicationConfig = null; // the global streams configs and default topology props + private Properties topologyProperties = null; // this topology's config overrides + private TopologyConfig topologyConfigs = null; // the configs for tasks in this topology // The name of the topology this builder belongs to, or null if none private String topologyName; @@ -359,15 +363,18 @@ public final InternalTopologyBuilder setApplicationId(final String applicationId return this; } - public synchronized final InternalTopologyBuilder setStreamsConfig(final StreamsConfig config) { - Objects.requireNonNull(config, "config can't be null"); - this.config = config; + public synchronized final void setTopologyProperties(final Properties props) { + this.topologyProperties = props; + } - return this; + public synchronized final void setStreamsConfig(final StreamsConfig config) { + Objects.requireNonNull(config, "config can't be null"); + this.applicationConfig = config; + topologyConfigs = new TopologyConfig(applicationConfig, topologyProperties); } - public synchronized final StreamsConfig getStreamsConfig() { - return config; + public synchronized final TopologyConfig topologyConfig() { + return topologyConfigs; } public String topologyName() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index c028e737d9efe..b3fd9fb9e378c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -27,7 +27,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; -import org.apache.kafka.streams.processor.internals.namedtopology.TaskConfig; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import java.util.Collections; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java index 6251d97cd0366..3eaee48319320 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java @@ -24,7 +24,6 @@ import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; -import org.apache.kafka.streams.processor.internals.namedtopology.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.slf4j.Logger; @@ -161,7 +160,7 @@ StandbyTask createStandbyTask(final TaskId taskId, taskId, inputPartitions, topology, - new TaskConfig(applicationConfig, topologyMetadata.getConfigForTask(taskId)), + topologyMetadata.getTaskConfigFor(taskId), StreamThread.eosEnabled(applicationConfig), streamsMetrics, stateManager, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 660908a1a1d59..9c23f1d3681b1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -27,7 +27,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.StreamsException; @@ -44,7 +43,7 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; -import org.apache.kafka.streams.processor.internals.namedtopology.TaskConfig; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import java.io.IOException; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java index 129ca09ad5c74..23a2d073f7653 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; import org.apache.kafka.streams.processor.internals.StreamThread.State; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import java.util.ArrayList; import java.util.Collection; @@ -161,6 +162,10 @@ public void unregisterTopology(final String topologyName) { } } + public TaskConfig getTaskConfigFor(final TaskId taskId) { + return lookupBuilderForTask(taskId).topologyConfig().getTaskConfig(); + } + public void buildAndRewriteTopology() { applyToEachBuilder(this::buildAndVerifyTopology); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TaskConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TaskConfig.java deleted file mode 100644 index 6c813a4e57cad..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TaskConfig.java +++ /dev/null @@ -1,41 +0,0 @@ -package org.apache.kafka.streams.processor.internals.namedtopology; - -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.errors.DeserializationExceptionHandler; -import org.apache.kafka.streams.processor.TimestampExtractor; - -import java.util.Properties; -import java.util.function.Function; - -/** - * Streams configs that apply at the task level. At the moment they can be specified either via the configs passed in - * to the {@link org.apache.kafka.streams.KafkaStreams} or {@link KafkaStreamsNamedTopologyWrapper} constructor, or - * else by passing them in to {@link org.apache.kafka.streams.StreamsBuilder#build(Properties)} or - * {@link NamedTopologyStreamsBuilder#buildNamedTopology(Properties)} - */ -public class TaskConfig { - public final long maxTaskIdleMs; - public final long taskTimeoutMs; - public final int maxBufferedSize; - public final TimestampExtractor timestampExtractor; - public final DeserializationExceptionHandler deserializationExceptionHandler; - - public TaskConfig(final StreamsConfig applicationConfig, final StreamsConfig topologyConfig) { - maxTaskIdleMs = getConfig(applicationConfig, topologyConfig, config -> config.getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG)); - taskTimeoutMs = getConfig(applicationConfig, topologyConfig, config -> config.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG)); - maxBufferedSize = getConfig(applicationConfig, topologyConfig, config -> config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)); - timestampExtractor = getConfig(applicationConfig, topologyConfig, StreamsConfig::defaultTimestampExtractor); - deserializationExceptionHandler = getConfig(applicationConfig, topologyConfig, StreamsConfig::defaultDeserializationExceptionHandler); - } - - /** - * @return the value of this config passed in to the topology if it exists, otherwise default application-wide config - */ - static V getConfig(final StreamsConfig applicationConfig, - final StreamsConfig topologyConfig, - final Function configGetter) { - final V topologyOverride = configGetter.apply(topologyConfig); - final V appValue = configGetter.apply(applicationConfig); - return topologyOverride == null ? appValue : topologyOverride; - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java new file mode 100644 index 0000000000000..a864357e7ef8a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -0,0 +1,73 @@ +package org.apache.kafka.streams.processor.internals.namedtopology; + +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.processor.TimestampExtractor; + +import java.util.Properties; +import java.util.function.Function; + +/** + * Streams configs that apply at the topology level. The values in the {@link StreamsConfig} parameter of the + * {@link org.apache.kafka.streams.KafkaStreams} or {@link KafkaStreamsNamedTopologyWrapper} constructors will + * determine the defaults, which can then be overridden for specific topologies by passing them in when creating the + * topology via the {@link org.apache.kafka.streams.StreamsBuilder#build(Properties)} or + * {@link NamedTopologyStreamsBuilder#buildNamedTopology(Properties)} methods. + */ +public class TopologyConfig extends StreamsConfig { + final StreamsConfig applicationConfig; + final Properties topologyOverrides; + + public TopologyConfig(final StreamsConfig applicationConfig, final Properties topologyProps) { + super(topologyProps, containsOverrides(topologyProps)); // skip logging if there aren't any topology overrides + this.applicationConfig = applicationConfig; + this.topologyOverrides = topologyProps; + } + + public TaskConfig getTaskConfig() { + return new TaskConfig( + getConfig(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, config -> config.getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG)), + getConfig(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, config -> config.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG)), + getConfig(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, config -> config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)), + getConfig(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, StreamsConfig::defaultTimestampExtractor), + getConfig(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, StreamsConfig::defaultDeserializationExceptionHandler) + ); + } + + /** + * @return the value of this config passed in to the topology if it exists, otherwise default application-wide config + */ + V getConfig(final String config, final Function configGetter) { + return topologyOverrides.containsKey(config) ? + configGetter.apply(this) : + configGetter.apply(applicationConfig); + } + + private static boolean containsOverrides(final Properties props) { + return props.containsKey(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG) || + props.containsKey(StreamsConfig.TASK_TIMEOUT_MS_CONFIG) || + props.containsKey(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG) || + props.containsKey(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG) || + props.containsKey(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG); + } + + public static class TaskConfig { + public final long maxTaskIdleMs; + public final long taskTimeoutMs; + public final int maxBufferedSize; + public final TimestampExtractor timestampExtractor; + public final DeserializationExceptionHandler deserializationExceptionHandler; + + private TaskConfig(final long maxTaskIdleMs, + final long taskTimeoutMs, + final int maxBufferedSize, + final TimestampExtractor timestampExtractor, + final DeserializationExceptionHandler deserializationExceptionHandler) { + this.maxTaskIdleMs = maxTaskIdleMs; + this.taskTimeoutMs = taskTimeoutMs; + this.maxBufferedSize = maxBufferedSize; + this.timestampExtractor = timestampExtractor; + this.deserializationExceptionHandler = deserializationExceptionHandler; + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 74f40599e9e0c..95842c430dbeb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -23,12 +23,15 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyDescription; +import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; @@ -47,6 +50,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.regex.Pattern; @@ -927,10 +931,24 @@ public void shouldSetCorrectSourceNodesWithRegexUpdatedTopics() { } @Test - public void shouldSetStreamsConfigOnRewriteTopology() { + public void shouldSetTopologyConfigWithoutOverridesOnRewriteTopology() { final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); - assertThat(topologyBuilder.getStreamsConfig(), equalTo(config)); + assertThat(topologyBuilder.topologyConfig(), equalTo(new TopologyConfig(config, new Properties()))); + } + + @Test + public void shouldSetTopologyConfigWithOverridesOnRewriteTopology() { + final Properties topologyOverrides = new Properties(); + topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); + topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); + builder.setTopologyProperties(topologyOverrides); + + final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); + final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); + + assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxTaskIdleMs, equalTo(500L)); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().deserializationExceptionHandler.getClass(), equalTo(LogAndContinueExceptionHandler.class)); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 50f4c33db7925..8755d44f66e46 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -36,6 +36,8 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.MockKeyValueStoreBuilder; @@ -56,6 +58,7 @@ import java.time.Duration; import java.util.Collections; import java.util.List; +import java.util.Properties; import java.util.stream.Collectors; import static java.util.Arrays.asList; @@ -611,7 +614,8 @@ private StandbyTask createStandbyTask() { taskId, Collections.singleton(partition), topology, - config, + new TopologyConfig(config, new Properties()).getTaskConfig(), + StreamThread.eosEnabled(config), streamsMetrics, stateManager, stateDirectory, diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index 05f10e98f2ef9..885b1af289a59 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -70,6 +70,7 @@ import org.apache.kafka.streams.processor.internals.Task; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; @@ -367,7 +368,7 @@ public Consumer getGlobalConsumer(final Map conf ); setupGlobalTask(mockWallClockTime, streamsConfig, streamsMetrics, cache); - setupTask(streamsConfig, streamsMetrics, cache); + setupTask(streamsConfig, streamsMetrics, cache, internalTopologyBuilder.topologyConfig().getTaskConfig()); } private static void logIfTaskIdleEnabled(final StreamsConfig streamsConfig) { @@ -469,7 +470,8 @@ private void setupGlobalTask(final Time mockWallClockTime, @SuppressWarnings("deprecation") private void setupTask(final StreamsConfig streamsConfig, final StreamsMetricsImpl streamsMetrics, - final ThreadCache cache) { + final ThreadCache cache, + final TaskConfig taskConfig) { if (!partitionsByInputTopic.isEmpty()) { consumer.assign(partitionsByInputTopic.values()); final Map startOffsets = new HashMap<>(); @@ -509,7 +511,8 @@ private void setupTask(final StreamsConfig streamsConfig, new HashSet<>(partitionsByInputTopic.values()), processorTopology, consumer, - streamsConfig, + taskConfig, + StreamThread.eosEnabled(streamsConfig), streamsMetrics, stateDirectory, cache, From 97c288964a1a08adbdf1b5cbb01333b2cb6ee746 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 26 Aug 2021 19:12:19 -0700 Subject: [PATCH 04/18] clean up for checkstyle --- .../apache/kafka/streams/StreamsConfig.java | 4 +-- .../internals/ActiveTaskCreator.java | 1 - .../processor/internals/StandbyTask.java | 3 +-- .../internals/StandbyTaskCreator.java | 1 - .../processor/internals/StreamTask.java | 3 +-- .../namedtopology/TopologyConfig.java | 25 +++++++++++++++++-- .../processor/internals/StandbyTaskTest.java | 1 - .../processor/internals/StreamTaskTest.java | 24 ++++++++++-------- .../StreamThreadStateStoreProviderTest.java | 3 ++- .../kafka/streams/TopologyTestDriver.java | 1 - 10 files changed, 43 insertions(+), 23 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 9c41250afe9d0..098539a46eb6d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1075,8 +1075,8 @@ public StreamsConfig(final Map props) { this(props, true); } - public StreamsConfig(final Map props, - final boolean doLog) { + protected StreamsConfig(final Map props, + final boolean doLog) { super(CONFIG, props, doLog); eosEnabled = StreamThread.eosEnabled(this); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java index b08a4d32ba081..3f6c7f9b821ec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java @@ -265,7 +265,6 @@ private StreamTask createActiveTask(final TaskId taskId, topology, consumer, topologyMetadata.getTaskConfigFor(taskId), - StreamThread.eosEnabled(applicationConfig), streamsMetrics, stateDirectory, cache, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index b3fd9fb9e378c..352b4b7980789 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -57,7 +57,6 @@ public class StandbyTask extends AbstractTask implements Task { final Set inputPartitions, final ProcessorTopology topology, final TaskConfig config, - final boolean eosEnabled, final StreamsMetricsImpl streamsMetrics, final ProcessorStateManager stateMgr, final StateDirectory stateDirectory, @@ -78,7 +77,7 @@ public class StandbyTask extends AbstractTask implements Task { processorContext.transitionToStandby(cache); closeTaskSensor = ThreadMetrics.closeTaskSensor(Thread.currentThread().getName(), streamsMetrics); - this.eosEnabled = eosEnabled; + this.eosEnabled = config.eosEnabled; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java index 3eaee48319320..a2854d4f1b5bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java @@ -161,7 +161,6 @@ StandbyTask createStandbyTask(final TaskId taskId, inputPartitions, topology, topologyMetadata.getTaskConfigFor(taskId), - StreamThread.eosEnabled(applicationConfig), streamsMetrics, stateManager, stateDirectory, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 9c23f1d3681b1..32369c983f19b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -118,7 +118,6 @@ public StreamTask(final TaskId id, final ProcessorTopology topology, final Consumer mainConsumer, final TaskConfig config, - final boolean eosEnabled, final StreamsMetricsImpl streamsMetrics, final StateDirectory stateDirectory, final ThreadCache cache, @@ -144,7 +143,7 @@ public StreamTask(final TaskId id, this.time = time; this.recordCollector = recordCollector; - this.eosEnabled = eosEnabled; + this.eosEnabled = config.eosEnabled; final String threadId = Thread.currentThread().getName(); this.streamsMetrics = streamsMetrics; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java index a864357e7ef8a..95d85e34a505f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -1,8 +1,25 @@ +/* + * 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.processor.internals.namedtopology; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.internals.StreamThread; import java.util.Properties; import java.util.function.Function; @@ -30,7 +47,8 @@ public TaskConfig getTaskConfig() { getConfig(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, config -> config.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG)), getConfig(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, config -> config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)), getConfig(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, StreamsConfig::defaultTimestampExtractor), - getConfig(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, StreamsConfig::defaultDeserializationExceptionHandler) + getConfig(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, StreamsConfig::defaultDeserializationExceptionHandler), + StreamThread.eosEnabled(applicationConfig) ); } @@ -57,17 +75,20 @@ public static class TaskConfig { public final int maxBufferedSize; public final TimestampExtractor timestampExtractor; public final DeserializationExceptionHandler deserializationExceptionHandler; + public final boolean eosEnabled; private TaskConfig(final long maxTaskIdleMs, final long taskTimeoutMs, final int maxBufferedSize, final TimestampExtractor timestampExtractor, - final DeserializationExceptionHandler deserializationExceptionHandler) { + final DeserializationExceptionHandler deserializationExceptionHandler, + final boolean eosEnabled) { this.maxTaskIdleMs = maxTaskIdleMs; this.taskTimeoutMs = taskTimeoutMs; this.maxBufferedSize = maxBufferedSize; this.timestampExtractor = timestampExtractor; this.deserializationExceptionHandler = deserializationExceptionHandler; + this.eosEnabled = eosEnabled; } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 8755d44f66e46..ce190c131bc1c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -615,7 +615,6 @@ private StandbyTask createStandbyTask() { Collections.singleton(partition), topology, new TopologyConfig(config, new Properties()).getTaskConfig(), - StreamThread.eosEnabled(config), streamsMetrics, stateManager, stateDirectory, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index c372234ad3551..9800bca585cc5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -57,6 +57,8 @@ import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.MockProcessorNode; @@ -83,6 +85,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -99,6 +102,7 @@ import static org.apache.kafka.common.utils.Utils.mkProperties; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.streams.StreamsConfig.AT_LEAST_ONCE; +import static org.apache.kafka.streams.StreamsConfig.METRICS_LATEST; import static org.apache.kafka.streams.processor.internals.StreamTask.encodeTimestamp; import static org.apache.kafka.streams.processor.internals.Task.State.CREATED; import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING; @@ -1593,7 +1597,7 @@ public void shouldReturnOffsetsForRepartitionTopicsForPurging() { mkSet(partition1, repartition), topology, consumer, - config, + new TopologyConfig(config, new Properties()).getTaskConfig(), streamsMetrics, stateDirectory, cache, @@ -2179,7 +2183,7 @@ public void shouldThrowTopologyExceptionIfTaskCreatedForUnknownTopic() { partitions, topology, consumer, - createConfig("100"), + new TopologyConfig(createConfig("100"), new Properties()).getTaskConfig(), metrics, stateDirectory, cache, @@ -2246,7 +2250,7 @@ private StreamTask createOptimizedStatefulTask(final StreamsConfig config, final mkSet(partition1), topology, consumer, - config, + new TopologyConfig(config, new Properties()).getTaskConfig(), streamsMetrics, stateDirectory, cache, @@ -2287,7 +2291,7 @@ public Map committed(final Set Date: Thu, 26 Aug 2021 20:49:46 -0700 Subject: [PATCH 05/18] avoid requiring appid, handle case of null topology props --- .../internals/InternalTopologyBuilder.java | 8 +- .../namedtopology/TopologyConfig.java | 108 ++++++++++++------ .../InternalTopologyBuilderTest.java | 7 +- .../processor/internals/StandbyTaskTest.java | 3 +- .../processor/internals/StreamTaskTest.java | 22 ++-- .../StreamThreadStateStoreProviderTest.java | 2 +- 6 files changed, 96 insertions(+), 54 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 1f11df76d2176..e5774518ed34c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -370,7 +370,13 @@ public synchronized final void setTopologyProperties(final Properties props) { public synchronized final void setStreamsConfig(final StreamsConfig config) { Objects.requireNonNull(config, "config can't be null"); this.applicationConfig = config; - topologyConfigs = new TopologyConfig(applicationConfig, topologyProperties); + topologyConfigs = new TopologyConfig( + topologyName, + applicationConfig, + topologyProperties == null ? + new Properties() : + topologyProperties + ); } public synchronized final TopologyConfig topologyConfig() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java index 95d85e34a505f..4972cd847a789 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -16,13 +16,17 @@ */ package org.apache.kafka.streams.processor.internals.namedtopology; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.internals.StreamThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Properties; -import java.util.function.Function; +import java.util.function.Supplier; /** * Streams configs that apply at the topology level. The values in the {@link StreamsConfig} parameter of the @@ -31,42 +35,78 @@ * topology via the {@link org.apache.kafka.streams.StreamsBuilder#build(Properties)} or * {@link NamedTopologyStreamsBuilder#buildNamedTopology(Properties)} methods. */ -public class TopologyConfig extends StreamsConfig { - final StreamsConfig applicationConfig; - final Properties topologyOverrides; - - public TopologyConfig(final StreamsConfig applicationConfig, final Properties topologyProps) { - super(topologyProps, containsOverrides(topologyProps)); // skip logging if there aren't any topology overrides - this.applicationConfig = applicationConfig; - this.topologyOverrides = topologyProps; - } +public class TopologyConfig extends AbstractConfig { + private final Logger log = LoggerFactory.getLogger(TopologyConfig.class); - public TaskConfig getTaskConfig() { - return new TaskConfig( - getConfig(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, config -> config.getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG)), - getConfig(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, config -> config.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG)), - getConfig(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, config -> config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)), - getConfig(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, StreamsConfig::defaultTimestampExtractor), - getConfig(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, StreamsConfig::defaultDeserializationExceptionHandler), - StreamThread.eosEnabled(applicationConfig) - ); + public final String topologyName; + public final boolean eosEnabled; + + final long maxTaskIdleMs; + final long taskTimeoutMs; + final int maxBufferedSize; + final Supplier timestampExtractorSupplier; + final Supplier deserializationExceptionHandlerSupplier; + + public TopologyConfig(final String topologyName, final StreamsConfig globalAppConfigs, final Properties topologyOverrides) { + super(new ConfigDef(), topologyOverrides, false); + + this.topologyName = topologyName; + this.eosEnabled = StreamThread.eosEnabled(globalAppConfigs); + + if (isTopologyOverride(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) { + maxTaskIdleMs = getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs); + } else { + maxTaskIdleMs = globalAppConfigs.getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG); + } + + if (isTopologyOverride(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, topologyOverrides)) { + taskTimeoutMs = getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.TASK_TIMEOUT_MS_CONFIG, taskTimeoutMs); + } else { + taskTimeoutMs = globalAppConfigs.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG); + } + + if (isTopologyOverride(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, topologyOverrides)) { + maxBufferedSize = getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize); + } else { + maxBufferedSize = globalAppConfigs.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + } + + if (isTopologyOverride(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, topologyOverrides)) { + timestampExtractorSupplier = () -> getConfiguredInstance(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, getClass(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG)); + } else { + timestampExtractorSupplier = () -> globalAppConfigs.getConfiguredInstance(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + } + + if (isTopologyOverride(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, topologyOverrides)) { + deserializationExceptionHandlerSupplier = () -> getConfiguredInstance(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); + log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, getClass(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG)); + } else { + deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); + + } } /** - * @return the value of this config passed in to the topology if it exists, otherwise default application-wide config + * @return true if there is an override for this config in the properties of this NamedTopology. Applications that + * don't use named topologies will just refer to the global defaults regardless of the topology properties */ - V getConfig(final String config, final Function configGetter) { - return topologyOverrides.containsKey(config) ? - configGetter.apply(this) : - configGetter.apply(applicationConfig); + private boolean isTopologyOverride(final String config, final Properties topologyOverrides) { + return topologyName != null && topologyOverrides.containsKey(config); } - private static boolean containsOverrides(final Properties props) { - return props.containsKey(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG) || - props.containsKey(StreamsConfig.TASK_TIMEOUT_MS_CONFIG) || - props.containsKey(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG) || - props.containsKey(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG) || - props.containsKey(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG); + public TaskConfig getTaskConfig() { + return new TaskConfig( + maxTaskIdleMs, + taskTimeoutMs, + maxBufferedSize, + timestampExtractorSupplier.get(), + deserializationExceptionHandlerSupplier.get(), + eosEnabled + ); } public static class TaskConfig { @@ -78,10 +118,10 @@ public static class TaskConfig { public final boolean eosEnabled; private TaskConfig(final long maxTaskIdleMs, - final long taskTimeoutMs, - final int maxBufferedSize, - final TimestampExtractor timestampExtractor, - final DeserializationExceptionHandler deserializationExceptionHandler, + final long taskTimeoutMs, + final int maxBufferedSize, + final TimestampExtractor timestampExtractor, + final DeserializationExceptionHandler deserializationExceptionHandler, final boolean eosEnabled) { this.maxTaskIdleMs = maxTaskIdleMs; this.taskTimeoutMs = taskTimeoutMs; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 95842c430dbeb..ea4d0cf40c25c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyDescription; -import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.StateStore; @@ -931,14 +930,14 @@ public void shouldSetCorrectSourceNodesWithRegexUpdatedTopics() { } @Test - public void shouldSetTopologyConfigWithoutOverridesOnRewriteTopology() { + public void shouldSetTopologyConfigOnRewriteTopology() { final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); - assertThat(topologyBuilder.topologyConfig(), equalTo(new TopologyConfig(config, new Properties()))); + assertThat(topologyBuilder.topologyConfig(), equalTo(new TopologyConfig(null, config, new Properties()))); } @Test - public void shouldSetTopologyConfigWithOverridesOnRewriteTopology() { + public void shouldOverrideGlobalStreamsConfigWithTopologyProps() { final Properties topologyOverrides = new Properties(); topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index ce190c131bc1c..9c366fbeeb9d1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -37,7 +37,6 @@ import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.MockKeyValueStoreBuilder; @@ -614,7 +613,7 @@ private StandbyTask createStandbyTask() { taskId, Collections.singleton(partition), topology, - new TopologyConfig(config, new Properties()).getTaskConfig(), + new TopologyConfig(null, config, new Properties()).getTaskConfig(), streamsMetrics, stateManager, stateDirectory, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 9800bca585cc5..1b3c5001d17a0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -58,7 +58,6 @@ import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; -import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.MockProcessorNode; @@ -102,7 +101,6 @@ import static org.apache.kafka.common.utils.Utils.mkProperties; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.streams.StreamsConfig.AT_LEAST_ONCE; -import static org.apache.kafka.streams.StreamsConfig.METRICS_LATEST; import static org.apache.kafka.streams.processor.internals.StreamTask.encodeTimestamp; import static org.apache.kafka.streams.processor.internals.Task.State.CREATED; import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING; @@ -1597,7 +1595,7 @@ public void shouldReturnOffsetsForRepartitionTopicsForPurging() { mkSet(partition1, repartition), topology, consumer, - new TopologyConfig(config, new Properties()).getTaskConfig(), + new TopologyConfig(null, config, new Properties()).getTaskConfig(), streamsMetrics, stateDirectory, cache, @@ -2183,7 +2181,7 @@ public void shouldThrowTopologyExceptionIfTaskCreatedForUnknownTopic() { partitions, topology, consumer, - new TopologyConfig(createConfig("100"), new Properties()).getTaskConfig(), + new TopologyConfig(null, createConfig("100"), new Properties()).getTaskConfig(), metrics, stateDirectory, cache, @@ -2250,7 +2248,7 @@ private StreamTask createOptimizedStatefulTask(final StreamsConfig config, final mkSet(partition1), topology, consumer, - new TopologyConfig(config, new Properties()).getTaskConfig(), + new TopologyConfig(null, config, new Properties()).getTaskConfig(), streamsMetrics, stateDirectory, cache, @@ -2291,7 +2289,7 @@ public Map committed(final Set Date: Thu, 26 Aug 2021 21:38:28 -0700 Subject: [PATCH 06/18] cleaning up and adding tests --- .../apache/kafka/streams/StreamsBuilder.java | 10 +-- .../apache/kafka/streams/StreamsConfig.java | 8 +- .../internals/InternalTopologyBuilder.java | 1 - .../namedtopology/NamedTopology.java | 15 +--- .../NamedTopologyStreamsBuilder.java | 15 +--- .../namedtopology/TopologyConfig.java | 81 ++++++++++++++----- .../InternalTopologyBuilderTest.java | 10 ++- 7 files changed, 82 insertions(+), 58 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index 00f7e67929321..9789d082a89f9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -75,13 +75,13 @@ public class StreamsBuilder { protected final InternalStreamsBuilder internalStreamsBuilder; public StreamsBuilder() { - topology = getNewTopology(); - internalTopologyBuilder = topology.internalTopologyBuilder; - internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); + this(new Topology()); } - protected Topology getNewTopology() { - return new Topology(); + protected StreamsBuilder(final Topology topology) { + this.topology = topology; + internalTopologyBuilder = topology.internalTopologyBuilder; + internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 098539a46eb6d..7a0b420910d2e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -349,7 +349,7 @@ public class StreamsConfig extends AbstractConfig { /** {@code buffered.records.per.partition} */ @SuppressWarnings("WeakerAccess") public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; - private 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 built.in.metrics.version} */ public static final String BUILT_IN_METRICS_VERSION_CONFIG = "built.in.metrics.version"; @@ -380,7 +380,7 @@ public class StreamsConfig extends AbstractConfig { /** {@code default.deserialization.exception.handler} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.deserialization.exception.handler"; - private static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.DeserializationExceptionHandler interface."; + public static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.DeserializationExceptionHandler interface."; /** {@code default.production.exception.handler} */ @SuppressWarnings("WeakerAccess") @@ -423,11 +423,11 @@ public class StreamsConfig extends AbstractConfig { /** {@code default.timestamp.extractor} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "default.timestamp.extractor"; - private static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC = "Default timestamp extractor class that implements the org.apache.kafka.streams.processor.TimestampExtractor interface."; + public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC = "Default timestamp extractor class that implements the org.apache.kafka.streams.processor.TimestampExtractor interface."; /** {@code max.task.idle.ms} */ public static final String MAX_TASK_IDLE_MS_CONFIG = "max.task.idle.ms"; - private static final String MAX_TASK_IDLE_MS_DOC = "This config controls whether joins and merges" + public static final String MAX_TASK_IDLE_MS_DOC = "This config controls whether joins and merges" + " may produce out-of-order results." + " The config value is the maximum amount of time in milliseconds a stream task will stay idle" + " when it is fully caught up on some (but not all) input partitions" diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index e5774518ed34c..7265258fced85 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -342,7 +342,6 @@ Sink describe() { } } - public void setNamedTopology(final NamedTopology topology) { final String topologyName = topology.name(); Objects.requireNonNull(topologyName, "topology name can't be null"); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java index 0cdfd5b61c5cc..defb217d526f6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java @@ -19,22 +19,13 @@ import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.List; public class NamedTopology extends Topology { - - private final Logger log = LoggerFactory.getLogger(NamedTopology.class); - private String name; + private final String name; - void setTopologyName(final String newTopologyName) { - if (name != null) { - log.error("Unable to set topologyName = {} because the name is already set to {}", newTopologyName, name); - throw new IllegalStateException("Tried to set topologyName but the name was already set"); - } - name = newTopologyName; + public NamedTopology(final String topologyName) { + name = topologyName; internalTopologyBuilder.setNamedTopology(this); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java index 5d3fad83732fd..6134522dcf654 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java @@ -17,21 +17,17 @@ package org.apache.kafka.streams.processor.internals.namedtopology; import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.processor.TaskId; import java.util.Properties; public class NamedTopologyStreamsBuilder extends StreamsBuilder { - final String topologyName; - /** * @param topologyName any string representing your NamedTopology, all characters allowed except for "__" * @throws IllegalArgumentException if the name contains the character sequence "__" */ public NamedTopologyStreamsBuilder(final String topologyName) { - super(); - this.topologyName = topologyName; + super(new NamedTopology(topologyName)); if (topologyName.contains(TaskId.NAMED_TOPOLOGY_DELIMITER)) { throw new IllegalArgumentException("The character sequence '__' is not allowed in a NamedTopology, please select a new name"); } @@ -39,13 +35,6 @@ public NamedTopologyStreamsBuilder(final String topologyName) { public synchronized NamedTopology buildNamedTopology(final Properties props) { super.build(props); - final NamedTopology namedTopology = (NamedTopology) super.topology; - namedTopology.setTopologyName(topologyName); - return namedTopology; - } - - @Override - public Topology getNewTopology() { - return new NamedTopology(); + return (NamedTopology) super.topology; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java index 4972cd847a789..5cf17dc007270 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -18,6 +18,8 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.processor.TimestampExtractor; @@ -28,6 +30,17 @@ import java.util.Properties; import java.util.function.Supplier; +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.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_TIMESTAMP_EXTRACTOR_CLASS_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC; +import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_DOC; +import static org.apache.kafka.streams.StreamsConfig.TASK_TIMEOUT_MS_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.TASK_TIMEOUT_MS_DOC; + /** * Streams configs that apply at the topology level. The values in the {@link StreamsConfig} parameter of the * {@link org.apache.kafka.streams.KafkaStreams} or {@link KafkaStreamsNamedTopologyWrapper} constructors will @@ -36,6 +49,30 @@ * {@link NamedTopologyStreamsBuilder#buildNamedTopology(Properties)} methods. */ public class TopologyConfig extends AbstractConfig { + private static final ConfigDef CONFIG; + static { + CONFIG = new ConfigDef() + .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + Importance.MEDIUM, + DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) + .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, + Type.CLASS, + Importance.MEDIUM, + DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) + .define(MAX_TASK_IDLE_MS_CONFIG, + Type.LONG, + Importance.MEDIUM, + MAX_TASK_IDLE_MS_DOC) + .define(TASK_TIMEOUT_MS_CONFIG, + Type.LONG, + Importance.MEDIUM, + TASK_TIMEOUT_MS_DOC); + } private final Logger log = LoggerFactory.getLogger(TopologyConfig.class); public final String topologyName; @@ -48,44 +85,44 @@ public class TopologyConfig extends AbstractConfig { final Supplier deserializationExceptionHandlerSupplier; public TopologyConfig(final String topologyName, final StreamsConfig globalAppConfigs, final Properties topologyOverrides) { - super(new ConfigDef(), topologyOverrides, false); + super(CONFIG, topologyOverrides, false); this.topologyName = topologyName; this.eosEnabled = StreamThread.eosEnabled(globalAppConfigs); - - if (isTopologyOverride(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) { - maxTaskIdleMs = getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG); - log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs); + + if (isTopologyOverride(MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) { + maxTaskIdleMs = getLong(MAX_TASK_IDLE_MS_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs); } else { - maxTaskIdleMs = globalAppConfigs.getLong(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG); + maxTaskIdleMs = globalAppConfigs.getLong(MAX_TASK_IDLE_MS_CONFIG); } - if (isTopologyOverride(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, topologyOverrides)) { - taskTimeoutMs = getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG); - log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.TASK_TIMEOUT_MS_CONFIG, taskTimeoutMs); + if (isTopologyOverride(TASK_TIMEOUT_MS_CONFIG, topologyOverrides)) { + taskTimeoutMs = getLong(TASK_TIMEOUT_MS_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, TASK_TIMEOUT_MS_CONFIG, taskTimeoutMs); } else { - taskTimeoutMs = globalAppConfigs.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG); + taskTimeoutMs = globalAppConfigs.getLong(TASK_TIMEOUT_MS_CONFIG); } - if (isTopologyOverride(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, topologyOverrides)) { - maxBufferedSize = getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize); + if (isTopologyOverride(BUFFERED_RECORDS_PER_PARTITION_CONFIG, topologyOverrides)) { + maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize); } else { - maxBufferedSize = globalAppConfigs.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + maxBufferedSize = globalAppConfigs.getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); } - if (isTopologyOverride(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, topologyOverrides)) { - timestampExtractorSupplier = () -> getConfiguredInstance(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); - log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, getClass(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG)); + if (isTopologyOverride(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, topologyOverrides)) { + timestampExtractorSupplier = () -> getConfiguredInstance(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, getClass(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG)); } else { - timestampExtractorSupplier = () -> globalAppConfigs.getConfiguredInstance(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + timestampExtractorSupplier = () -> globalAppConfigs.getConfiguredInstance(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); } - if (isTopologyOverride(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, topologyOverrides)) { - deserializationExceptionHandlerSupplier = () -> getConfiguredInstance(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); - log.info("Topology {} is overriding {} to {}", topologyName, StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, getClass(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG)); + if (isTopologyOverride(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, topologyOverrides)) { + deserializationExceptionHandlerSupplier = () -> getConfiguredInstance(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); + log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, getClass(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG)); } else { - deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); + deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index ea4d0cf40c25c..dc4ab65b8a203 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; +import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; @@ -937,16 +938,23 @@ public void shouldSetTopologyConfigOnRewriteTopology() { } @Test - public void shouldOverrideGlobalStreamsConfigWithTopologyProps() { + public void shouldOverrideGlobalStreamsConfigWithNamedTopologyProps() { final Properties topologyOverrides = new Properties(); topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); + topologyOverrides.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L); + topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); + topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); + builder.setNamedTopology(new NamedTopology("test-topology")); builder.setTopologyProperties(topologyOverrides); final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxTaskIdleMs, equalTo(500L)); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().taskTimeoutMs, equalTo(1000L)); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxBufferedSize, equalTo(15)); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().timestampExtractor.getClass(), equalTo(MockTimestampExtractor.class)); assertThat(topologyBuilder.topologyConfig().getTaskConfig().deserializationExceptionHandler.getClass(), equalTo(LogAndContinueExceptionHandler.class)); } From 1013826fea1066dda8816c7a449f18691044bba3 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 26 Aug 2021 21:59:10 -0700 Subject: [PATCH 07/18] fix and expand tests --- .../namedtopology/TopologyConfig.java | 5 +++ .../InternalTopologyBuilderTest.java | 31 ++++++++++++++++--- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java index 5cf17dc007270..36b5f9ff8c820 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -54,22 +54,27 @@ public class TopologyConfig extends AbstractConfig { CONFIG = new ConfigDef() .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, Type.INT, + null, Importance.LOW, BUFFERED_RECORDS_PER_PARTITION_DOC) .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, Type.CLASS, + null, Importance.MEDIUM, DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, Type.CLASS, + null, Importance.MEDIUM, DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) .define(MAX_TASK_IDLE_MS_CONFIG, Type.LONG, + null, Importance.MEDIUM, MAX_TASK_IDLE_MS_DOC) .define(TASK_TIMEOUT_MS_CONFIG, Type.LONG, + null, Importance.MEDIUM, TASK_TIMEOUT_MS_DOC); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index dc4ab65b8a203..bdf4979022c47 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -932,13 +932,16 @@ public void shouldSetCorrectSourceNodesWithRegexUpdatedTopics() { @Test public void shouldSetTopologyConfigOnRewriteTopology() { - final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); - final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); - assertThat(topologyBuilder.topologyConfig(), equalTo(new TopologyConfig(null, config, new Properties()))); + final Properties globalProps = StreamsTestUtils.getStreamsConfig(); + globalProps.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 100L); + final StreamsConfig globalStreamsConfig = new StreamsConfig(globalProps); + final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(globalStreamsConfig); + assertThat(topologyBuilder.topologyConfig(), equalTo(new TopologyConfig(null, globalStreamsConfig, new Properties()))); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxTaskIdleMs, equalTo(100L)); } @Test - public void shouldOverrideGlobalStreamsConfigWithNamedTopologyProps() { + public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { final Properties topologyOverrides = new Properties(); topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); topologyOverrides.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L); @@ -958,6 +961,26 @@ public void shouldOverrideGlobalStreamsConfigWithNamedTopologyProps() { assertThat(topologyBuilder.topologyConfig().getTaskConfig().deserializationExceptionHandler.getClass(), equalTo(LogAndContinueExceptionHandler.class)); } + @Test + public void shouldNotOverrideGlobalStreamsConfigWhenGivenUnnamedTopologyProps() { + final Properties topologyOverrides = new Properties(); + topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); + topologyOverrides.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L); + topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); + topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); + topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); + builder.setTopologyProperties(topologyOverrides); + + final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); + final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); + + assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxTaskIdleMs, not(500L)); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().taskTimeoutMs, not(1000L)); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxBufferedSize, not(15)); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().timestampExtractor.getClass(), not(MockTimestampExtractor.class)); + assertThat(topologyBuilder.topologyConfig().getTaskConfig().deserializationExceptionHandler.getClass(), not(LogAndContinueExceptionHandler.class)); + } + @Test public void shouldAddTimestampExtractorPerSource() { builder.addSource(null, "source", new MockTimestampExtractor(), null, null, "topic"); From aaa3dcb1ea886f417e2144217008e909c10102df Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Wed, 8 Sep 2021 19:41:05 -0700 Subject: [PATCH 08/18] review feedback --- .../org/apache/kafka/streams/StreamsBuilder.java | 2 +- .../processor/internals/InternalTopologyBuilder.java | 12 ++++++------ .../internals/namedtopology/TopologyConfig.java | 5 +++-- .../internals/InternalTopologyBuilderTest.java | 4 ++-- 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index 9789d082a89f9..92fca87681c62 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -611,7 +611,7 @@ public synchronized Topology build() { */ public synchronized Topology build(final Properties props) { internalStreamsBuilder.buildAndOptimizeTopology(props); - internalTopologyBuilder.setTopologyProperties(props); + internalTopologyBuilder.setTopologyOverrides(props); return topology; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 7265258fced85..7ed5f0e6919a2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -136,8 +136,8 @@ public class InternalTopologyBuilder { private Map> nodeGroups = null; private StreamsConfig applicationConfig = null; // the global streams configs and default topology props - private Properties topologyProperties = null; // this topology's config overrides - private TopologyConfig topologyConfigs = null; // the configs for tasks in this topology + private Properties topologyOverrides = null; // this topology's config overrides + private TopologyConfig topologyConfigs = null; // The name of the topology this builder belongs to, or null if none private String topologyName; @@ -362,8 +362,8 @@ public final InternalTopologyBuilder setApplicationId(final String applicationId return this; } - public synchronized final void setTopologyProperties(final Properties props) { - this.topologyProperties = props; + public synchronized final void setTopologyOverrides(final Properties props) { + this.topologyOverrides = props; } public synchronized final void setStreamsConfig(final StreamsConfig config) { @@ -372,9 +372,9 @@ public synchronized final void setStreamsConfig(final StreamsConfig config) { topologyConfigs = new TopologyConfig( topologyName, applicationConfig, - topologyProperties == null ? + topologyOverrides == null ? new Properties() : - topologyProperties + topologyOverrides ); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java index 36b5f9ff8c820..b40343afed63e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -57,7 +57,7 @@ public class TopologyConfig extends AbstractConfig { null, Importance.LOW, BUFFERED_RECORDS_PER_PARTITION_DOC) - .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, + .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, Type.CLASS, null, Importance.MEDIUM, @@ -128,7 +128,6 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, getClass(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG)); } else { deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); - } } @@ -137,6 +136,8 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo * don't use named topologies will just refer to the global defaults regardless of the topology properties */ private boolean isTopologyOverride(final String config, final Properties topologyOverrides) { + // TODO KAFKA-13283: eventually we should always have the topology props override the global ones regardless + // of whether it's a named topology or not return topologyName != null && topologyOverrides.containsKey(config); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index bdf4979022c47..35dda2e7ff4e5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -949,7 +949,7 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); builder.setNamedTopology(new NamedTopology("test-topology")); - builder.setTopologyProperties(topologyOverrides); + builder.setTopologyOverrides(topologyOverrides); final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); @@ -969,7 +969,7 @@ public void shouldNotOverrideGlobalStreamsConfigWhenGivenUnnamedTopologyProps() topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); - builder.setTopologyProperties(topologyOverrides); + builder.setTopologyOverrides(topologyOverrides); final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); From 9169099487bb1720704ff532fa4157ce4213614f Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Fri, 10 Sep 2021 03:54:32 -0700 Subject: [PATCH 09/18] introduce topology overrides during topology construction --- .../apache/kafka/streams/KafkaStreams.java | 70 +++++++++---------- .../apache/kafka/streams/StreamsBuilder.java | 1 - .../org/apache/kafka/streams/Topology.java | 6 +- .../internals/InternalTopologyBuilder.java | 42 +++++------ .../KafkaStreamsNamedTopologyWrapper.java | 41 +++++++++-- .../namedtopology/NamedTopology.java | 19 ++++- ...Builder.java => NamedTopologyBuilder.java} | 27 ++++--- .../namedtopology/TopologyConfig.java | 8 ++- .../NamedTopologyIntegrationTest.java | 22 +++--- .../StoreQueryIntegrationTest.java | 8 +-- .../InternalTopologyBuilderTest.java | 2 +- .../internals/NamedTopologyTest.java | 18 ++--- 12 files changed, 159 insertions(+), 105 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/{NamedTopologyStreamsBuilder.java => NamedTopologyBuilder.java} (53%) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 5067da6902480..a44a6c41748b6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -152,7 +152,7 @@ public class KafkaStreams implements AutoCloseable { private final Logger log; private final String clientId; private final Metrics metrics; - private final StreamsConfig config; + protected final StreamsConfig applicationConfigs; protected final List threads; protected final StateDirectory stateDirectory; private final StreamsMetadataState streamsMetadataState; @@ -795,12 +795,12 @@ public KafkaStreams(final Topology topology, * you still must {@link #close()} it to avoid resource leaks. * * @param topology the topology specifying the computational logic - * @param config configs for Kafka Streams + * @param applicationConfigs configs for Kafka Streams * @throws StreamsException if any fatal error occurs */ public KafkaStreams(final Topology topology, - final StreamsConfig config) { - this(topology, config, new DefaultKafkaClientSupplier()); + final StreamsConfig applicationConfigs) { + this(topology, applicationConfigs, new DefaultKafkaClientSupplier()); } /** @@ -810,15 +810,15 @@ public KafkaStreams(final Topology topology, * you still must {@link #close()} it to avoid resource leaks. * * @param topology the topology specifying the computational logic - * @param config configs for Kafka Streams + * @param applicationConfigs configs for Kafka Streams * @param clientSupplier the Kafka clients supplier which provides underlying producer and consumer clients * for the new {@code KafkaStreams} instance * @throws StreamsException if any fatal error occurs */ public KafkaStreams(final Topology topology, - final StreamsConfig config, + final StreamsConfig applicationConfigs, final KafkaClientSupplier clientSupplier) { - this(new TopologyMetadata(topology.internalTopologyBuilder, config), config, clientSupplier); + this(new TopologyMetadata(topology.internalTopologyBuilder, applicationConfigs), applicationConfigs, clientSupplier); } /** @@ -828,34 +828,34 @@ public KafkaStreams(final Topology topology, * you still must {@link #close()} it to avoid resource leaks. * * @param topology the topology specifying the computational logic - * @param config configs for Kafka Streams + * @param applicationConfigs configs for Kafka Streams * @param time {@code Time} implementation; cannot be null * @throws StreamsException if any fatal error occurs */ public KafkaStreams(final Topology topology, - final StreamsConfig config, + final StreamsConfig applicationConfigs, final Time time) { - this(new TopologyMetadata(topology.internalTopologyBuilder, config), config, new DefaultKafkaClientSupplier(), time); + this(new TopologyMetadata(topology.internalTopologyBuilder, applicationConfigs), applicationConfigs, new DefaultKafkaClientSupplier(), time); } private KafkaStreams(final Topology topology, - final StreamsConfig config, + final StreamsConfig applicationConfigs, final KafkaClientSupplier clientSupplier, final Time time) throws StreamsException { - this(new TopologyMetadata(topology.internalTopologyBuilder, config), config, clientSupplier, time); + this(new TopologyMetadata(topology.internalTopologyBuilder, applicationConfigs), applicationConfigs, clientSupplier, time); } protected KafkaStreams(final TopologyMetadata topologyMetadata, - final StreamsConfig config, + final StreamsConfig applicationConfigs, final KafkaClientSupplier clientSupplier) throws StreamsException { - this(topologyMetadata, config, clientSupplier, Time.SYSTEM); + this(topologyMetadata, applicationConfigs, clientSupplier, Time.SYSTEM); } private KafkaStreams(final TopologyMetadata topologyMetadata, - final StreamsConfig config, + final StreamsConfig applicationConfigs, final KafkaClientSupplier clientSupplier, final Time time) throws StreamsException { - this.config = config; + this.applicationConfigs = applicationConfigs; this.time = time; this.topologyMetadata = topologyMetadata; @@ -864,15 +864,15 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, final boolean hasGlobalTopology = topologyMetadata.hasGlobalTopology(); try { - stateDirectory = new StateDirectory(config, time, topologyMetadata.hasPersistentStores(), topologyMetadata.hasNamedTopologies()); + stateDirectory = new StateDirectory(applicationConfigs, time, topologyMetadata.hasPersistentStores(), topologyMetadata.hasNamedTopologies()); processId = stateDirectory.initializeProcessId(); } catch (final ProcessorStateException fatal) { throw new StreamsException(fatal); } // The application ID is a required config and hence should always have value - final String userClientId = config.getString(StreamsConfig.CLIENT_ID_CONFIG); - final String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG); + final String userClientId = applicationConfigs.getString(StreamsConfig.CLIENT_ID_CONFIG); + final String applicationId = applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG); if (userClientId.length() <= 0) { clientId = applicationId + "-" + processId; } else { @@ -883,22 +883,22 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, // use client id instead of thread client id since this admin client may be shared among threads this.clientSupplier = clientSupplier; - adminClient = clientSupplier.getAdmin(config.getAdminConfigs(ClientUtils.getSharedAdminClientId(clientId))); + adminClient = clientSupplier.getAdmin(applicationConfigs.getAdminConfigs(ClientUtils.getSharedAdminClientId(clientId))); log.info("Kafka Streams version: {}", ClientMetrics.version()); log.info("Kafka Streams commit ID: {}", ClientMetrics.commitId()); - metrics = getMetrics(config, time, clientId); + metrics = getMetrics(applicationConfigs, time, clientId); streamsMetrics = new StreamsMetricsImpl( metrics, clientId, - config.getString(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG), + applicationConfigs.getString(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG), time ); ClientMetrics.addVersionMetric(streamsMetrics); ClientMetrics.addCommitIdMetric(streamsMetrics); - ClientMetrics.addApplicationIdMetric(streamsMetrics, config.getString(StreamsConfig.APPLICATION_ID_CONFIG)); + ClientMetrics.addApplicationIdMetric(streamsMetrics, applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG)); ClientMetrics.addTopologyDescriptionMetric(streamsMetrics, (metricsConfig, now) -> this.topologyMetadata.topologyDescriptionString()); ClientMetrics.addStateMetric(streamsMetrics, (metricsConfig, now) -> state); threads = Collections.synchronizedList(new LinkedList<>()); @@ -906,14 +906,14 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, streamsMetadataState = new StreamsMetadataState( this.topologyMetadata, - parseHostInfo(config.getString(StreamsConfig.APPLICATION_SERVER_CONFIG))); + parseHostInfo(applicationConfigs.getString(StreamsConfig.APPLICATION_SERVER_CONFIG))); oldHandler = false; streamsUncaughtExceptionHandler = this::defaultStreamsUncaughtExceptionHandler; delegatingStateRestoreListener = new DelegatingStateRestoreListener(); - totalCacheSize = config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG); - final int numStreamThreads = topologyMetadata.getNumStreamThreads(config); + totalCacheSize = applicationConfigs.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG); + final int numStreamThreads = topologyMetadata.getNumStreamThreads(applicationConfigs); final long cacheSizePerThread = getCacheSizePerThread(numStreamThreads); GlobalStreamThread.State globalThreadState = null; @@ -921,8 +921,8 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, final String globalThreadId = clientId + "-GlobalStreamThread"; globalStreamThread = new GlobalStreamThread( topologyMetadata.globalTaskTopology(), - config, - clientSupplier.getGlobalConsumer(config.getGlobalConsumerConfigs(clientId)), + applicationConfigs, + clientSupplier.getGlobalConsumer(applicationConfigs.getGlobalConsumerConfigs(clientId)), stateDirectory, cacheSizePerThread, streamsMetrics, @@ -949,13 +949,13 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, } stateDirCleaner = setupStateDirCleaner(); - rocksDBMetricsRecordingService = maybeCreateRocksDBMetricsRecordingService(clientId, config); + rocksDBMetricsRecordingService = maybeCreateRocksDBMetricsRecordingService(clientId, applicationConfigs); } private StreamThread createAndAddStreamThread(final long cacheSizePerThread, final int threadIdx) { final StreamThread streamThread = StreamThread.create( topologyMetadata, - config, + applicationConfigs, clientSupplier, adminClient, processId, @@ -1117,7 +1117,7 @@ private Optional removeStreamThread(final long timeoutMs) throws Timeout final Collection membersToRemove = Collections.singletonList(memberToRemove); final RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroupResult = adminClient.removeMembersFromConsumerGroup( - config.getString(StreamsConfig.APPLICATION_ID_CONFIG), + applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG), new RemoveMembersFromConsumerGroupOptions(membersToRemove) ); try { @@ -1125,16 +1125,16 @@ private Optional removeStreamThread(final long timeoutMs) throws Timeout removeMembersFromConsumerGroupResult.memberResult(memberToRemove).get(remainingTimeMs, TimeUnit.MILLISECONDS); } catch (final java.util.concurrent.TimeoutException e) { log.error("Could not remove static member {} from consumer group {} due to a timeout: {}", - groupInstanceID.get(), config.getString(StreamsConfig.APPLICATION_ID_CONFIG), e); + groupInstanceID.get(), applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG), e); throw new TimeoutException(e.getMessage(), e); } catch (final InterruptedException e) { Thread.currentThread().interrupt(); } catch (final ExecutionException e) { log.error("Could not remove static member {} from consumer group {} due to: {}", - groupInstanceID.get(), config.getString(StreamsConfig.APPLICATION_ID_CONFIG), e); + groupInstanceID.get(), applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG), e); throw new StreamsException( "Could not remove static member " + groupInstanceID.get() - + " from consumer group " + config.getString(StreamsConfig.APPLICATION_ID_CONFIG) + + " from consumer group " + applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG) + " for the following reason: ", e.getCause() ); @@ -1286,7 +1286,7 @@ public synchronized void start() throws IllegalStateException, StreamsException processStreamThread(StreamThread::start); - final Long cleanupDelay = config.getLong(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG); + final Long cleanupDelay = applicationConfigs.getLong(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG); stateDirCleaner.scheduleAtFixedRate(() -> { // we do not use lock here since we only read on the value and act on it if (state == State.RUNNING) { diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index 92fca87681c62..e5befd2b3b3a9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -611,7 +611,6 @@ public synchronized Topology build() { */ public synchronized Topology build(final Properties props) { internalStreamsBuilder.buildAndOptimizeTopology(props); - internalTopologyBuilder.setTopologyOverrides(props); return topology; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/Topology.java b/streams/src/main/java/org/apache/kafka/streams/Topology.java index 7c45de7f47c9a..821ab6377fcf8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/Topology.java +++ b/streams/src/main/java/org/apache/kafka/streams/Topology.java @@ -54,7 +54,11 @@ */ public class Topology { - protected final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder(); + protected final InternalTopologyBuilder internalTopologyBuilder = newInternalTopologyBuilder(); + + protected InternalTopologyBuilder newInternalTopologyBuilder() { + return new InternalTopologyBuilder(); + } /** * Sets the {@code auto.offset.reset} configuration when diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 7ed5f0e6919a2..dc0d7481f4322 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -66,6 +66,17 @@ public class InternalTopologyBuilder { + public InternalTopologyBuilder() { + this.topologyName = null; + this.namedTopology = null; + } + + public InternalTopologyBuilder(final NamedTopology namedTopology, final TopologyConfig topologyConfigs) { + this.topologyName = namedTopology.name(); + this.namedTopology = namedTopology; + this.topologyConfigs = topologyConfigs; + } + private static final Logger log = LoggerFactory.getLogger(InternalTopologyBuilder.class); private static final String[] NO_PREDECESSORS = {}; @@ -135,13 +146,15 @@ public class InternalTopologyBuilder { private Map> nodeGroups = null; - private StreamsConfig applicationConfig = null; // the global streams configs and default topology props - private Properties topologyOverrides = null; // this topology's config overrides - private TopologyConfig topologyConfigs = null; - // The name of the topology this builder belongs to, or null if none - private String topologyName; - private NamedTopology namedTopology; + private final String topologyName; + private final NamedTopology namedTopology; + + // TODO KAFKA-13283: we still need to save the topology overrides here since we don't get the app configs until we + // get passed in to the KafkaStreams, once we enforce all configs be passed in when constructing the topology then + // we can make topologyConfigs final and drop the other + private TopologyConfig topologyConfigs; // the configs for this topology, including overrides and global defaults + private Properties topologyOverrides = null; private boolean hasPersistentStores = false; @@ -342,18 +355,6 @@ Sink describe() { } } - public void setNamedTopology(final NamedTopology topology) { - final String topologyName = topology.name(); - Objects.requireNonNull(topologyName, "topology name can't be null"); - Objects.requireNonNull(topology, "named topology can't be null"); - if (this.topologyName != null) { - log.error("Tried to reset the topologyName to {} but it was already set to {}", topologyName, this.topologyName); - throw new IllegalStateException("The topologyName has already been set to " + this.topologyName); - } - this.namedTopology = topology; - this.topologyName = topologyName; - } - // public for testing only public final InternalTopologyBuilder setApplicationId(final String applicationId) { Objects.requireNonNull(applicationId, "applicationId can't be null"); @@ -366,9 +367,8 @@ public synchronized final void setTopologyOverrides(final Properties props) { this.topologyOverrides = props; } - public synchronized final void setStreamsConfig(final StreamsConfig config) { - Objects.requireNonNull(config, "config can't be null"); - this.applicationConfig = config; + public synchronized final void setStreamsConfig(final StreamsConfig applicationConfig) { + Objects.requireNonNull(applicationConfig, "config can't be null"); topologyConfigs = new TopologyConfig( topologyName, applicationConfig, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java index d4170d499c595..1736c663241bc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.TopologyMetadata; @@ -112,10 +113,42 @@ private KafkaStreamsNamedTopologyWrapper(final Collection topolog ); } + /** + * Provides a high-level DSL for specifying the processing logic of your application and building it into an + * independent topology that can be executed by this {@link KafkaStreams}. + * + * @param topologyName The name for this topology + * @param topologyConfigs The properties and any config overrides for this topology + * + * @throws IllegalArgumentException if the name contains the character sequence "__" + */ + public NamedTopologyBuilder newNamedTopologyBuilder(final String topologyName, final Properties topologyConfigs) { + if (topologyName.contains(TaskId.NAMED_TOPOLOGY_DELIMITER)) { + throw new IllegalArgumentException("The character sequence '__' is not allowed in a NamedTopology, please select a new name"); + } + return new NamedTopologyBuilder(topologyName, applicationConfigs, topologyConfigs); + } + + /** + * Returns an empty topology for full control over the graph of streams and processor nodes that define the processing + * logic to be executed by this {@link KafkaStreams}. + * + * @param topologyName The name for this topology + * @param topologyConfigs The properties and any config overrides for this topology + * + * @throws IllegalArgumentException if the name contains the character sequence "__" + */ + public NamedTopology newNamedTopology(final String topologyName, final Properties topologyConfigs) { + if (topologyName.contains(TaskId.NAMED_TOPOLOGY_DELIMITER)) { + throw new IllegalArgumentException("The character sequence '__' is not allowed in a NamedTopology, please select a new name"); + } + return new NamedTopology(topologyName, applicationConfigs, topologyConfigs); + } + /** * @return the NamedTopology for the specific name, or Optional.empty() if the application has no NamedTopology of that name */ - public Optional getTopologyByName(final String name) { + public Optional lookupTopologyByName(final String name) { return Optional.ofNullable(topologyMetadata.lookupBuilderForNamedTopology(name)).map(InternalTopologyBuilder::namedTopology); } @@ -131,7 +164,7 @@ public Optional getTopologyByName(final String name) { public void addNamedTopology(final NamedTopology newTopology) { if (hasStartedOrFinishedShuttingDown()) { throw new IllegalStateException("Cannot add a NamedTopology while the state is " + super.state); - } else if (getTopologyByName(newTopology.name()).isPresent()) { + } else if (lookupTopologyByName(newTopology.name()).isPresent()) { throw new IllegalArgumentException("Unable to add the new NamedTopology " + newTopology.name() + " as another of the same name already exists"); } @@ -150,7 +183,7 @@ public void addNamedTopology(final NamedTopology newTopology) { public void removeNamedTopology(final String topologyToRemove) { if (!isRunningOrRebalancing()) { throw new IllegalStateException("Cannot remove a NamedTopology while the state is " + super.state); - } else if (!getTopologyByName(topologyToRemove).isPresent()) { + } else if (!lookupTopologyByName(topologyToRemove).isPresent()) { throw new IllegalArgumentException("Unable to locate for removal a NamedTopology called " + topologyToRemove); } @@ -171,7 +204,7 @@ public void removeNamedTopology(final String topologyToRemove) { * @throws StreamsException if cleanup failed */ public void cleanUpNamedTopology(final String name) { - if (getTopologyByName(name).isPresent()) { + if (lookupTopologyByName(name).isPresent()) { throw new IllegalStateException("Can't clean up local state for an active NamedTopology: " + name); } stateDirectory.clearLocalStateForNamedTopology(name); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java index defb217d526f6..37500fa15a5e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java @@ -16,17 +16,21 @@ */ package org.apache.kafka.streams.processor.internals.namedtopology; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import java.util.List; +import java.util.Properties; public class NamedTopology extends Topology { private final String name; - - public NamedTopology(final String topologyName) { + + private final TopologyConfig topologyConfigs; + + NamedTopology(final String topologyName, final StreamsConfig applicationConfigs, final Properties topologyOverrides) { name = topologyName; - internalTopologyBuilder.setNamedTopology(this); + topologyConfigs = new TopologyConfig(topologyName, applicationConfigs, topologyOverrides); } public String name() { @@ -40,4 +44,13 @@ public List sourceTopics() { InternalTopologyBuilder internalTopologyBuilder() { return internalTopologyBuilder; } + + TopologyConfig topologyConfigs() { + return topologyConfigs; + } + + @Override + protected InternalTopologyBuilder newInternalTopologyBuilder() { + return new InternalTopologyBuilder(this, topologyConfigs); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java similarity index 53% rename from streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java index 6134522dcf654..fc703f493f6e2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java @@ -17,24 +17,23 @@ package org.apache.kafka.streams.processor.internals.namedtopology; import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.StreamsConfig; import java.util.Properties; -public class NamedTopologyStreamsBuilder extends StreamsBuilder { - /** - * @param topologyName any string representing your NamedTopology, all characters allowed except for "__" - * @throws IllegalArgumentException if the name contains the character sequence "__" - */ - public NamedTopologyStreamsBuilder(final String topologyName) { - super(new NamedTopology(topologyName)); - if (topologyName.contains(TaskId.NAMED_TOPOLOGY_DELIMITER)) { - throw new IllegalArgumentException("The character sequence '__' is not allowed in a NamedTopology, please select a new name"); - } +public class NamedTopologyBuilder extends StreamsBuilder { + + NamedTopologyBuilder(final String topologyName, final StreamsConfig applicationConfigs, final Properties topologyOverrides) { + super(new NamedTopology(topologyName, applicationConfigs, topologyOverrides)); + } + + @Override + public synchronized NamedTopology build() { + super.build(namedTopology().topologyConfigs().topologyOverrides); + return namedTopology(); } - public synchronized NamedTopology buildNamedTopology(final Properties props) { - super.build(props); - return (NamedTopology) super.topology; + private NamedTopology namedTopology() { + return (NamedTopology) topology; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java index b40343afed63e..fa12055ad3938 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -46,7 +46,7 @@ * {@link org.apache.kafka.streams.KafkaStreams} or {@link KafkaStreamsNamedTopologyWrapper} constructors will * determine the defaults, which can then be overridden for specific topologies by passing them in when creating the * topology via the {@link org.apache.kafka.streams.StreamsBuilder#build(Properties)} or - * {@link NamedTopologyStreamsBuilder#buildNamedTopology(Properties)} methods. + * {@link NamedTopologyBuilder#buildNamedTopology(Properties)} methods. */ public class TopologyConfig extends AbstractConfig { private static final ConfigDef CONFIG; @@ -83,6 +83,9 @@ public class TopologyConfig extends AbstractConfig { public final String topologyName; public final boolean eosEnabled; + public final StreamsConfig applicationConfigs; + public final Properties topologyOverrides; + final long maxTaskIdleMs; final long taskTimeoutMs; final int maxBufferedSize; @@ -95,6 +98,9 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo this.topologyName = topologyName; this.eosEnabled = StreamThread.eosEnabled(globalAppConfigs); + this.applicationConfigs = globalAppConfigs; + this.topologyOverrides = topologyOverrides; + if (isTopologyOverride(MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) { maxTaskIdleMs = getLong(MAX_TASK_IDLE_MS_CONFIG); log.info("Topology {} is overriding {} to {}", topologyName, MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java index 2b01feec691a1..6828822a25a4e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java @@ -36,7 +36,7 @@ import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; -import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyStreamsBuilder; +import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.utils.UniqueTopicSerdeScope; @@ -142,14 +142,14 @@ public static void closeCluster() { private final KafkaClientSupplier clientSupplier = new DefaultKafkaClientSupplier(); // builders for the 1st Streams instance (default) - private final NamedTopologyStreamsBuilder topology1Builder = new NamedTopologyStreamsBuilder("topology-1"); - private final NamedTopologyStreamsBuilder topology2Builder = new NamedTopologyStreamsBuilder("topology-2"); - private final NamedTopologyStreamsBuilder topology3Builder = new NamedTopologyStreamsBuilder("topology-3"); + private final NamedTopologyBuilder topology1Builder = new NamedTopologyBuilder("topology-1"); + private final NamedTopologyBuilder topology2Builder = new NamedTopologyBuilder("topology-2"); + private final NamedTopologyBuilder topology3Builder = new NamedTopologyBuilder("topology-3"); // builders for the 2nd Streams instance - private final NamedTopologyStreamsBuilder topology1Builder2 = new NamedTopologyStreamsBuilder("topology-1"); - private final NamedTopologyStreamsBuilder topology2Builder2 = new NamedTopologyStreamsBuilder("topology-2"); - private final NamedTopologyStreamsBuilder topology3Builder2 = new NamedTopologyStreamsBuilder("topology-3"); + private final NamedTopologyBuilder topology1Builder2 = new NamedTopologyBuilder("topology-1"); + private final NamedTopologyBuilder topology2Builder2 = new NamedTopologyBuilder("topology-2"); + private final NamedTopologyBuilder topology3Builder2 = new NamedTopologyBuilder("topology-3"); private Properties props; private Properties props2; @@ -203,10 +203,10 @@ public void shouldPrefixAllInternalTopicNamesWithNamedTopology() throws Exceptio final String countTopologyName = "count-topology"; final String fkjTopologyName = "FKJ-topology"; - final NamedTopologyStreamsBuilder countBuilder = new NamedTopologyStreamsBuilder(countTopologyName); + final NamedTopologyBuilder countBuilder = new NamedTopologyBuilder(countTopologyName); countBuilder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(); - final NamedTopologyStreamsBuilder fkjBuilder = new NamedTopologyStreamsBuilder(fkjTopologyName); + final NamedTopologyBuilder fkjBuilder = new NamedTopologyBuilder(fkjTopologyName); final UniqueTopicSerdeScope serdeScope = new UniqueTopicSerdeScope(); final KTable left = fkjBuilder.table( @@ -434,9 +434,9 @@ private static void produceToInputTopics(final String topic, final Collection buildNamedTopologies(final NamedTopologyStreamsBuilder... builders) { + private List buildNamedTopologies(final NamedTopologyBuilder... builders) { final List topologies = new ArrayList<>(); - for (final NamedTopologyStreamsBuilder builder : builders) { + for (final NamedTopologyBuilder builder : builders) { topologies.add(builder.buildNamedTopology(props)); } return topologies; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java index fb2d0a181768e..cbf28bff2ba33 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java @@ -34,7 +34,7 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; -import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyStreamsBuilder; +import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreType; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; @@ -397,10 +397,10 @@ public void shouldQuerySpecificStalePartitionStoresMultiStreamThreadsNamedTopolo final Semaphore semaphore = new Semaphore(0); final int numStreamThreads = 2; - final NamedTopologyStreamsBuilder builder1A = new NamedTopologyStreamsBuilder("topology-A"); + final NamedTopologyBuilder builder1A = new NamedTopologyBuilder("topology-A"); getStreamsBuilderWithTopology(builder1A, semaphore); - final NamedTopologyStreamsBuilder builder2A = new NamedTopologyStreamsBuilder("topology-A"); + final NamedTopologyBuilder builder2A = new NamedTopologyBuilder("topology-A"); getStreamsBuilderWithTopology(builder2A, semaphore); final Properties streamsConfiguration1 = streamsConfiguration(); @@ -567,7 +567,7 @@ private KafkaStreams createKafkaStreams(final StreamsBuilder builder, final Prop return streams; } - private KafkaStreamsNamedTopologyWrapper createNamedTopologyKafkaStreams(final NamedTopologyStreamsBuilder builder, final Properties config) { + private KafkaStreamsNamedTopologyWrapper createNamedTopologyKafkaStreams(final NamedTopologyBuilder builder, final Properties config) { final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(builder.buildNamedTopology(config), config); streamsToCleanup.add(streams); return streams; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 35dda2e7ff4e5..80ea99f3177c4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -948,7 +948,7 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); - builder.setNamedTopology(new NamedTopology("test-topology")); + builder.setNamedTopology(new NamedTopology("test-topology", new Properties())); builder.setTopologyOverrides(topologyOverrides); final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java index 59b3eda727d0f..e6e9318f50fb0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java @@ -22,7 +22,7 @@ import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; -import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyStreamsBuilder; +import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.test.TestUtils; @@ -41,9 +41,9 @@ public class NamedTopologyTest { final KafkaClientSupplier clientSupplier = new DefaultKafkaClientSupplier(); final Properties props = configProps(); - final NamedTopologyStreamsBuilder builder1 = new NamedTopologyStreamsBuilder("topology-1"); - final NamedTopologyStreamsBuilder builder2 = new NamedTopologyStreamsBuilder("topology-2"); - final NamedTopologyStreamsBuilder builder3 = new NamedTopologyStreamsBuilder("topology-3"); + final NamedTopologyBuilder builder1 = new NamedTopologyBuilder("topology-1"); + final NamedTopologyBuilder builder2 = new NamedTopologyBuilder("topology-2"); + final NamedTopologyBuilder builder3 = new NamedTopologyBuilder("topology-3"); KafkaStreamsNamedTopologyWrapper streams; @@ -71,7 +71,7 @@ private static Properties configProps() { @Test public void shouldThrowIllegalArgumentOnIllegalName() { - assertThrows(IllegalArgumentException.class, () -> new NamedTopologyStreamsBuilder("__not-allowed__")); + assertThrows(IllegalArgumentException.class, () -> new NamedTopologyBuilder("__not-allowed__")); } @Test @@ -103,15 +103,15 @@ public void shouldReturnTopologyByName() { final NamedTopology topology2 = builder2.buildNamedTopology(props); final NamedTopology topology3 = builder3.buildNamedTopology(props); streams = new KafkaStreamsNamedTopologyWrapper(asList(topology1, topology2, topology3), props, clientSupplier); - assertThat(streams.getTopologyByName("topology-1").get(), equalTo(topology1)); - assertThat(streams.getTopologyByName("topology-2").get(), equalTo(topology2)); - assertThat(streams.getTopologyByName("topology-3").get(), equalTo(topology3)); + assertThat(streams.lookupTopologyByName("topology-1").get(), equalTo(topology1)); + assertThat(streams.lookupTopologyByName("topology-2").get(), equalTo(topology2)); + assertThat(streams.lookupTopologyByName("topology-3").get(), equalTo(topology3)); } @Test public void shouldReturnEmptyWhenLookingUpNonExistentTopologyByName() { streams = new KafkaStreamsNamedTopologyWrapper(builder1.buildNamedTopology(props), props, clientSupplier); - assertThat(streams.getTopologyByName("non-existent-topology").isPresent(), equalTo(false)); + assertThat(streams.lookupTopologyByName("non-existent-topology").isPresent(), equalTo(false)); } @Test From 3c4d8eb63bf49afb0079b79365127ca5672b2f0f Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Sun, 19 Sep 2021 03:46:57 -0700 Subject: [PATCH 10/18] latest feedback --- .../internals/InternalTopologyBuilder.java | 8 ++----- .../KafkaStreamsNamedTopologyWrapper.java | 22 +++---------------- .../namedtopology/NamedTopology.java | 14 ++++++++---- .../InternalTopologyBuilderTest.java | 3 --- 4 files changed, 15 insertions(+), 32 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index dc0d7481f4322..243a450650ce4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -71,10 +71,10 @@ public InternalTopologyBuilder() { this.namedTopology = null; } - public InternalTopologyBuilder(final NamedTopology namedTopology, final TopologyConfig topologyConfigs) { + public InternalTopologyBuilder(final NamedTopology namedTopology, final TopologyConfig topologyOverrides) { this.topologyName = namedTopology.name(); this.namedTopology = namedTopology; - this.topologyConfigs = topologyConfigs; + this.topologyConfigs = topologyOverrides; } private static final Logger log = LoggerFactory.getLogger(InternalTopologyBuilder.class); @@ -363,10 +363,6 @@ public final InternalTopologyBuilder setApplicationId(final String applicationId return this; } - public synchronized final void setTopologyOverrides(final Properties props) { - this.topologyOverrides = props; - } - public synchronized final void setStreamsConfig(final StreamsConfig applicationConfig) { Objects.requireNonNull(applicationConfig, "config can't be null"); topologyConfigs = new TopologyConfig( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java index 1736c663241bc..2eeb5b9e7260b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java @@ -118,31 +118,15 @@ private KafkaStreamsNamedTopologyWrapper(final Collection topolog * independent topology that can be executed by this {@link KafkaStreams}. * * @param topologyName The name for this topology - * @param topologyConfigs The properties and any config overrides for this topology + * @param topologyOverrides The properties and any config overrides for this topology * * @throws IllegalArgumentException if the name contains the character sequence "__" */ - public NamedTopologyBuilder newNamedTopologyBuilder(final String topologyName, final Properties topologyConfigs) { + public NamedTopologyBuilder newNamedTopologyBuilder(final String topologyName, final Properties topologyOverrides) { if (topologyName.contains(TaskId.NAMED_TOPOLOGY_DELIMITER)) { throw new IllegalArgumentException("The character sequence '__' is not allowed in a NamedTopology, please select a new name"); } - return new NamedTopologyBuilder(topologyName, applicationConfigs, topologyConfigs); - } - - /** - * Returns an empty topology for full control over the graph of streams and processor nodes that define the processing - * logic to be executed by this {@link KafkaStreams}. - * - * @param topologyName The name for this topology - * @param topologyConfigs The properties and any config overrides for this topology - * - * @throws IllegalArgumentException if the name contains the character sequence "__" - */ - public NamedTopology newNamedTopology(final String topologyName, final Properties topologyConfigs) { - if (topologyName.contains(TaskId.NAMED_TOPOLOGY_DELIMITER)) { - throw new IllegalArgumentException("The character sequence '__' is not allowed in a NamedTopology, please select a new name"); - } - return new NamedTopology(topologyName, applicationConfigs, topologyConfigs); + return new NamedTopologyBuilder(topologyName, applicationConfigs, topologyOverrides); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java index 37500fa15a5e1..503afedd59eec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java @@ -26,17 +26,23 @@ public class NamedTopology extends Topology { private final String name; - private final TopologyConfig topologyConfigs; + private final TopologyConfig topologyOverrides; NamedTopology(final String topologyName, final StreamsConfig applicationConfigs, final Properties topologyOverrides) { name = topologyName; - topologyConfigs = new TopologyConfig(topologyName, applicationConfigs, topologyOverrides); + this.topologyOverrides = new TopologyConfig(topologyName, applicationConfigs, topologyOverrides); } + /** + * @return the name of this topology + */ public String name() { return name; } + /** + * @return the list of all source topics this topology is subscribed to + */ public List sourceTopics() { return super.internalTopologyBuilder.fullSourceTopicNames(); } @@ -46,11 +52,11 @@ InternalTopologyBuilder internalTopologyBuilder() { } TopologyConfig topologyConfigs() { - return topologyConfigs; + return topologyOverrides; } @Override protected InternalTopologyBuilder newInternalTopologyBuilder() { - return new InternalTopologyBuilder(this, topologyConfigs); + return new InternalTopologyBuilder(this, topologyOverrides); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 80ea99f3177c4..22f37a8dc621a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -948,8 +948,6 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); - builder.setNamedTopology(new NamedTopology("test-topology", new Properties())); - builder.setTopologyOverrides(topologyOverrides); final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); @@ -969,7 +967,6 @@ public void shouldNotOverrideGlobalStreamsConfigWhenGivenUnnamedTopologyProps() topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); - builder.setTopologyOverrides(topologyOverrides); final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); From d8d0ac2bfced08e039c64dde8ef62650f21d11d7 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Tue, 28 Sep 2021 17:32:42 -0700 Subject: [PATCH 11/18] Refactor builder API: move NamedTopologies from KS constructor to KS#start, use KS to get builder --- .../apache/kafka/streams/StreamsBuilder.java | 19 ++++-- .../org/apache/kafka/streams/Topology.java | 10 +++- .../internals/InternalStreamsBuilder.java | 19 +++--- .../internals/InternalTopologyBuilder.java | 33 ++++------ .../processor/internals/TopologyMetadata.java | 2 +- .../KafkaStreamsNamedTopologyWrapper.java | 60 +++---------------- .../namedtopology/NamedTopology.java | 19 ++---- .../namedtopology/NamedTopologyBuilder.java | 17 +++--- .../namedtopology/TopologyConfig.java | 23 ++++--- .../InternalTopologyBuilderTest.java | 25 ++++---- .../kafka/streams/TopologyTestDriver.java | 2 +- 11 files changed, 94 insertions(+), 135 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index e5befd2b3b3a9..5f5d0b7853aac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -38,6 +38,7 @@ import org.apache.kafka.streams.processor.internals.ProcessorAdapter; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.processor.internals.SourceNode; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; @@ -75,15 +76,21 @@ public class StreamsBuilder { protected final InternalStreamsBuilder internalStreamsBuilder; public StreamsBuilder() { - this(new Topology()); + topology = new Topology(); + internalTopologyBuilder = topology.internalTopologyBuilder; + internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); } - protected StreamsBuilder(final Topology topology) { - this.topology = topology; + protected StreamsBuilder(final TopologyConfig topologyConfigs) { + topology = getNewTopology(topologyConfigs); internalTopologyBuilder = topology.internalTopologyBuilder; internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); } + protected Topology getNewTopology(final TopologyConfig topologyConfigs) { + return new Topology(); + } + /** * Create a {@link KStream} from the specified topic. * The default {@code "auto.offset.reset"} strategy, default {@link TimestampExtractor}, and default key and value @@ -610,7 +617,11 @@ public synchronized Topology build() { * @return the {@link Topology} that represents the specified processing logic */ public synchronized Topology build(final Properties props) { - internalStreamsBuilder.buildAndOptimizeTopology(props); + final boolean optimizeTopology = + props != null && + StreamsConfig.OPTIMIZE.equals(props.getProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG)); + + internalStreamsBuilder.buildAndOptimizeTopology(optimizeTopology); return topology; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/Topology.java b/streams/src/main/java/org/apache/kafka/streams/Topology.java index 821ab6377fcf8..0eb5e2b3861b4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/Topology.java +++ b/streams/src/main/java/org/apache/kafka/streams/Topology.java @@ -54,10 +54,14 @@ */ public class Topology { - protected final InternalTopologyBuilder internalTopologyBuilder = newInternalTopologyBuilder(); + protected final InternalTopologyBuilder internalTopologyBuilder; - protected InternalTopologyBuilder newInternalTopologyBuilder() { - return new InternalTopologyBuilder(); + public Topology() { + this(new InternalTopologyBuilder()); + } + + protected Topology(final InternalTopologyBuilder internalTopologyBuilder) { + this.internalTopologyBuilder = internalTopologyBuilder; } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java index 7f75f72a6584d..8d79e3847479c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java @@ -270,13 +270,17 @@ private void maybeAddNodeForOptimizationMetadata(final GraphNode node) { // use this method for testing only public void buildAndOptimizeTopology() { - buildAndOptimizeTopology(null); + buildAndOptimizeTopology(false); } - public void buildAndOptimizeTopology(final Properties props) { + public void buildAndOptimizeTopology(final boolean optimizeTopology) { mergeDuplicateSourceNodes(); - maybePerformOptimizations(props); + if (optimizeTopology) { + LOG.debug("Optimizing the Kafka Streams graph for repartition nodes"); + optimizeKTableSourceTopics(); + maybeOptimizeRepartitionOperations(); + } final PriorityQueue graphNodePriorityQueue = new PriorityQueue<>(5, Comparator.comparing(GraphNode::buildPriority)); @@ -346,15 +350,6 @@ private void mergeDuplicateSourceNodes() { } } - private void maybePerformOptimizations(final Properties props) { - - if (props != null && StreamsConfig.OPTIMIZE.equals(props.getProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG))) { - LOG.debug("Optimizing the Kafka Streams graph for repartition nodes"); - optimizeKTableSourceTopics(); - maybeOptimizeRepartitionOperations(); - } - } - private void optimizeKTableSourceTopics() { LOG.debug("Marking KTable source nodes to optimize using source topic for changelogs "); tableSourceNodes.forEach(node -> ((TableSourceNode) node).reuseSourceTopicForChangeLog(true)); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 243a450650ce4..3d5ba3e9cb60b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -68,13 +68,11 @@ public class InternalTopologyBuilder { public InternalTopologyBuilder() { this.topologyName = null; - this.namedTopology = null; } - public InternalTopologyBuilder(final NamedTopology namedTopology, final TopologyConfig topologyOverrides) { - this.topologyName = namedTopology.name(); - this.namedTopology = namedTopology; - this.topologyConfigs = topologyOverrides; + public InternalTopologyBuilder(final TopologyConfig topologyConfigs) { + this.topologyConfigs = topologyConfigs; + this.topologyName = topologyConfigs.topologyName; } private static final Logger log = LoggerFactory.getLogger(InternalTopologyBuilder.class); @@ -146,15 +144,15 @@ public InternalTopologyBuilder(final NamedTopology namedTopology, final Topology private Map> nodeGroups = null; - // The name of the topology this builder belongs to, or null if none + // The name of the topology this builder belongs to, or null if this is not a NamedTopology private final String topologyName; - private final NamedTopology namedTopology; + // TODO KAFKA-13336: we can remove this referance once we make the Topology/NamedTopology class into an interface and implement it + private NamedTopology namedTopology; // TODO KAFKA-13283: we still need to save the topology overrides here since we don't get the app configs until we // get passed in to the KafkaStreams, once we enforce all configs be passed in when constructing the topology then // we can make topologyConfigs final and drop the other private TopologyConfig topologyConfigs; // the configs for this topology, including overrides and global defaults - private Properties topologyOverrides = null; private boolean hasPersistentStores = false; @@ -365,16 +363,14 @@ public final InternalTopologyBuilder setApplicationId(final String applicationId public synchronized final void setStreamsConfig(final StreamsConfig applicationConfig) { Objects.requireNonNull(applicationConfig, "config can't be null"); - topologyConfigs = new TopologyConfig( - topologyName, - applicationConfig, - topologyOverrides == null ? - new Properties() : - topologyOverrides - ); + topologyConfigs = new TopologyConfig(applicationConfig); + } + + public synchronized final void setNamedTopology(final NamedTopology namedTopology) { + this.namedTopology = namedTopology; } - public synchronized final TopologyConfig topologyConfig() { + public synchronized final TopologyConfig topologyConfigs() { return topologyConfigs; } @@ -389,8 +385,8 @@ public NamedTopology namedTopology() { public synchronized final InternalTopologyBuilder rewriteTopology(final StreamsConfig config) { Objects.requireNonNull(config, "config can't be null"); - // set application id setApplicationId(config.getString(StreamsConfig.APPLICATION_ID_CONFIG)); + setStreamsConfig(config); // maybe strip out caching layers if (config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG) == 0L) { @@ -408,9 +404,6 @@ public synchronized final InternalTopologyBuilder rewriteTopology(final StreamsC globalStateStores.put(storeBuilder.name(), storeBuilder.build()); } - // set streams config - setStreamsConfig(config); - return this; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java index 23a2d073f7653..c7bb2ce228de6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java @@ -163,7 +163,7 @@ public void unregisterTopology(final String topologyName) { } public TaskConfig getTaskConfigFor(final TaskId taskId) { - return lookupBuilderForTask(taskId).topologyConfig().getTaskConfig(); + return lookupBuilderForTask(taskId).topologyConfigs().getTaskConfig(); } public void buildAndRewriteTopology() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java index 2eeb5b9e7260b..ae23a1e787c53 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java @@ -49,68 +49,26 @@ @Unstable public class KafkaStreamsNamedTopologyWrapper extends KafkaStreams { - /** - * A Kafka Streams application with a single initial NamedTopology - */ - public KafkaStreamsNamedTopologyWrapper(final NamedTopology topology, final Properties props) { - this(Collections.singleton(topology), new StreamsConfig(props), new DefaultKafkaClientSupplier()); - } - - /** - * A Kafka Streams application with a single initial NamedTopology - */ - public KafkaStreamsNamedTopologyWrapper(final NamedTopology topology, final Properties props, final KafkaClientSupplier clientSupplier) { - this(Collections.singleton(topology), new StreamsConfig(props), clientSupplier); - } - /** * An empty Kafka Streams application that allows NamedTopologies to be added at a later point */ public KafkaStreamsNamedTopologyWrapper(final Properties props) { - this(Collections.emptyList(), new StreamsConfig(props), new DefaultKafkaClientSupplier()); + this(new StreamsConfig(props), new DefaultKafkaClientSupplier()); } - /** - * An empty Kafka Streams application that allows NamedTopologies to be added at a later point - */ public KafkaStreamsNamedTopologyWrapper(final Properties props, final KafkaClientSupplier clientSupplier) { - this(Collections.emptyList(), new StreamsConfig(props), clientSupplier); - } - - /** - * A Kafka Streams application with a multiple initial NamedTopologies - * - * @throws IllegalArgumentException if any of the named topologies have the same name - * @throws TopologyException if multiple NamedTopologies subscribe to the same input topics or pattern - */ - public KafkaStreamsNamedTopologyWrapper(final Collection topologies, final Properties props) { - this(topologies, new StreamsConfig(props), new DefaultKafkaClientSupplier()); + this(new StreamsConfig(props), clientSupplier); } - /** - * A Kafka Streams application with a multiple initial NamedTopologies - * - * @throws IllegalArgumentException if any of the named topologies have the same name - * @throws TopologyException if multiple NamedTopologies subscribe to the same input topics or pattern - */ - public KafkaStreamsNamedTopologyWrapper(final Collection topologies, final Properties props, final KafkaClientSupplier clientSupplier) { - this(topologies, new StreamsConfig(props), clientSupplier); + private KafkaStreamsNamedTopologyWrapper(final StreamsConfig config, final KafkaClientSupplier clientSupplier) { + super(new TopologyMetadata(new ConcurrentSkipListMap<>(), config), config, clientSupplier); } - private KafkaStreamsNamedTopologyWrapper(final Collection topologies, final StreamsConfig config, final KafkaClientSupplier clientSupplier) { - super( - new TopologyMetadata( - topologies.stream().collect(Collectors.toMap( - NamedTopology::name, - NamedTopology::internalTopologyBuilder, - (v1, v2) -> { - throw new IllegalArgumentException("Topology names must be unique"); - }, - () -> new ConcurrentSkipListMap<>())), - config), - config, - clientSupplier - ); + public void start(final NamedTopology... initialTopologies) { + for (final NamedTopology topology : initialTopologies) { + addNamedTopology(topology); + } + super.start(); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java index 503afedd59eec..21b9d7ce56cc4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java @@ -16,28 +16,22 @@ */ package org.apache.kafka.streams.processor.internals.namedtopology; -import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import java.util.List; -import java.util.Properties; public class NamedTopology extends Topology { - private final String name; - private final TopologyConfig topologyOverrides; - - NamedTopology(final String topologyName, final StreamsConfig applicationConfigs, final Properties topologyOverrides) { - name = topologyName; - this.topologyOverrides = new TopologyConfig(topologyName, applicationConfigs, topologyOverrides); + public NamedTopology(final InternalTopologyBuilder internalTopologyBuilder) { + super(internalTopologyBuilder); } /** * @return the name of this topology */ public String name() { - return name; + return internalTopologyBuilder.topologyName(); } /** @@ -52,11 +46,6 @@ InternalTopologyBuilder internalTopologyBuilder() { } TopologyConfig topologyConfigs() { - return topologyOverrides; - } - - @Override - protected InternalTopologyBuilder newInternalTopologyBuilder() { - return new InternalTopologyBuilder(this, topologyOverrides); + return internalTopologyBuilder.topologyConfigs(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java index fc703f493f6e2..bf506e6cafd5e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java @@ -18,22 +18,25 @@ import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import java.util.Properties; public class NamedTopologyBuilder extends StreamsBuilder { - NamedTopologyBuilder(final String topologyName, final StreamsConfig applicationConfigs, final Properties topologyOverrides) { - super(new NamedTopology(topologyName, applicationConfigs, topologyOverrides)); - } + NamedTopologyBuilder(final String topologyName, final StreamsConfig applicationConfigs, final Properties topologyOverrides) { + super(new TopologyConfig(topologyName, applicationConfigs, topologyOverrides)); + internalTopologyBuilder.setNamedTopology((NamedTopology) topology); + } @Override public synchronized NamedTopology build() { - super.build(namedTopology().topologyConfigs().topologyOverrides); - return namedTopology(); + super.build(internalTopologyBuilder.topologyConfigs().topologyOverrides); + return (NamedTopology) topology; } - private NamedTopology namedTopology() { - return (NamedTopology) topology; + @Override + public NamedTopology getNewTopology(final TopologyConfig topologyConfigs) { + return new NamedTopology(new InternalTopologyBuilder(topologyConfigs)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java index fa12055ad3938..55e9861d8224f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -45,8 +45,7 @@ * Streams configs that apply at the topology level. The values in the {@link StreamsConfig} parameter of the * {@link org.apache.kafka.streams.KafkaStreams} or {@link KafkaStreamsNamedTopologyWrapper} constructors will * determine the defaults, which can then be overridden for specific topologies by passing them in when creating the - * topology via the {@link org.apache.kafka.streams.StreamsBuilder#build(Properties)} or - * {@link NamedTopologyBuilder#buildNamedTopology(Properties)} methods. + * topology builders via the {@link org.apache.kafka.streams.StreamsBuilder()} method. */ public class TopologyConfig extends AbstractConfig { private static final ConfigDef CONFIG; @@ -86,11 +85,15 @@ public class TopologyConfig extends AbstractConfig { public final StreamsConfig applicationConfigs; public final Properties topologyOverrides; - final long maxTaskIdleMs; - final long taskTimeoutMs; - final int maxBufferedSize; - final Supplier timestampExtractorSupplier; - final Supplier deserializationExceptionHandlerSupplier; + public final long maxTaskIdleMs; + public final long taskTimeoutMs; + public final int maxBufferedSize; + public final Supplier timestampExtractorSupplier; + public final Supplier deserializationExceptionHandlerSupplier; + + public TopologyConfig(final StreamsConfig globalAppConfigs) { + this(null, globalAppConfigs, new Properties()); + } public TopologyConfig(final String topologyName, final StreamsConfig globalAppConfigs, final Properties topologyOverrides) { super(CONFIG, topologyOverrides, false); @@ -103,7 +106,7 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo if (isTopologyOverride(MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) { maxTaskIdleMs = getLong(MAX_TASK_IDLE_MS_CONFIG); - log.info("Topology {} is overriding {} to {}", topologyName, MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs); + log.info("Topology {} is overridding {} to {}", topologyName, MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs); } else { maxTaskIdleMs = globalAppConfigs.getLong(MAX_TASK_IDLE_MS_CONFIG); } @@ -137,6 +140,10 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo } } + public boolean isNamedTopology() { + return topologyName != null; + } + /** * @return true if there is an override for this config in the properties of this NamedTopology. Applications that * don't use named topologies will just refer to the global defaults regardless of the topology properties diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 22f37a8dc621a..6eb289647f6cd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -30,7 +30,6 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; -import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; @@ -936,8 +935,8 @@ public void shouldSetTopologyConfigOnRewriteTopology() { globalProps.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 100L); final StreamsConfig globalStreamsConfig = new StreamsConfig(globalProps); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(globalStreamsConfig); - assertThat(topologyBuilder.topologyConfig(), equalTo(new TopologyConfig(null, globalStreamsConfig, new Properties()))); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxTaskIdleMs, equalTo(100L)); + assertThat(topologyBuilder.topologyConfigs(), equalTo(new TopologyConfig(null, globalStreamsConfig, new Properties()))); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxTaskIdleMs, equalTo(100L)); } @Test @@ -952,11 +951,11 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxTaskIdleMs, equalTo(500L)); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().taskTimeoutMs, equalTo(1000L)); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxBufferedSize, equalTo(15)); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().timestampExtractor.getClass(), equalTo(MockTimestampExtractor.class)); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().deserializationExceptionHandler.getClass(), equalTo(LogAndContinueExceptionHandler.class)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxTaskIdleMs, equalTo(500L)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().taskTimeoutMs, equalTo(1000L)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxBufferedSize, equalTo(15)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().timestampExtractor.getClass(), equalTo(MockTimestampExtractor.class)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().deserializationExceptionHandler.getClass(), equalTo(LogAndContinueExceptionHandler.class)); } @Test @@ -971,11 +970,11 @@ public void shouldNotOverrideGlobalStreamsConfigWhenGivenUnnamedTopologyProps() final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxTaskIdleMs, not(500L)); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().taskTimeoutMs, not(1000L)); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().maxBufferedSize, not(15)); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().timestampExtractor.getClass(), not(MockTimestampExtractor.class)); - assertThat(topologyBuilder.topologyConfig().getTaskConfig().deserializationExceptionHandler.getClass(), not(LogAndContinueExceptionHandler.class)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxTaskIdleMs, not(500L)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().taskTimeoutMs, not(1000L)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxBufferedSize, not(15)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().timestampExtractor.getClass(), not(MockTimestampExtractor.class)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().deserializationExceptionHandler.getClass(), not(LogAndContinueExceptionHandler.class)); } @Test diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index 651c5311d509e..2dc4026f40b52 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -368,7 +368,7 @@ public Consumer getGlobalConsumer(final Map conf ); setupGlobalTask(mockWallClockTime, streamsConfig, streamsMetrics, cache); - setupTask(streamsConfig, streamsMetrics, cache, internalTopologyBuilder.topologyConfig().getTaskConfig()); + setupTask(streamsConfig, streamsMetrics, cache, internalTopologyBuilder.topologyConfigs().getTaskConfig()); } private static void logIfTaskIdleEnabled(final StreamsConfig streamsConfig) { From d66e329d495236b0707777943eabd584d911b5a0 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 30 Sep 2021 15:35:52 -0700 Subject: [PATCH 12/18] tests adapted to new API --- .../internals/InternalStreamsBuilder.java | 2 - .../internals/InternalTopologyBuilder.java | 1 - .../KafkaStreamsNamedTopologyWrapper.java | 36 ++++- .../namedtopology/NamedTopology.java | 2 +- .../namedtopology/NamedTopologyBuilder.java | 2 +- .../NamedTopologyIntegrationTest.java | 127 +++++++++--------- .../StoreQueryIntegrationTest.java | 26 ++-- .../internals/NamedTopologyTest.java | 111 ++++++--------- 8 files changed, 154 insertions(+), 153 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java index 8d79e3847479c..f5046b352f02c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java @@ -19,7 +19,6 @@ import java.util.TreeMap; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.kstream.GlobalKTable; @@ -51,7 +50,6 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.PriorityQueue; -import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 3d5ba3e9cb60b..b5e1d50c23260 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -53,7 +53,6 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Properties; import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java index ae23a1e787c53..661e8aa846d17 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java @@ -33,7 +33,6 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.ConcurrentSkipListMap; -import java.util.stream.Collectors; /** * This is currently an internal and experimental feature for enabling certain kinds of topology upgrades. Use at @@ -64,7 +63,17 @@ private KafkaStreamsNamedTopologyWrapper(final StreamsConfig config, final Kafka super(new TopologyMetadata(new ConcurrentSkipListMap<>(), config), config, clientSupplier); } - public void start(final NamedTopology... initialTopologies) { + /** + * Start up Streams with a single initial NamedTopology + */ + public void start(final NamedTopology initialTopology) { + start(Collections.singleton(initialTopology)); + } + + /** + * Start up Streams with a collection of initial NamedTopologies + */ + public void start(final Collection initialTopologies) { for (final NamedTopology topology : initialTopologies) { addNamedTopology(topology); } @@ -76,7 +85,7 @@ public void start(final NamedTopology... initialTopologies) { * independent topology that can be executed by this {@link KafkaStreams}. * * @param topologyName The name for this topology - * @param topologyOverrides The properties and any config overrides for this topology + * @param topologyOverrides The properties and any config overrides for this topology * * @throws IllegalArgumentException if the name contains the character sequence "__" */ @@ -87,10 +96,23 @@ public NamedTopologyBuilder newNamedTopologyBuilder(final String topologyName, f return new NamedTopologyBuilder(topologyName, applicationConfigs, topologyOverrides); } + /** + * Provides a high-level DSL for specifying the processing logic of your application and building it into an + * independent topology that can be executed by this {@link KafkaStreams}. This method will use the global + * application {@link StreamsConfig} passed in to the constructor for all topology-level configs. To override + * any of these for this specific Topology, use {@link #newNamedTopologyBuilder(String, Properties)}. + * @param topologyName The name for this topology + * + * @throws IllegalArgumentException if the name contains the character sequence "__" + */ + public NamedTopologyBuilder newNamedTopologyBuilder(final String topologyName) { + return newNamedTopologyBuilder(topologyName, new Properties()); + } + /** * @return the NamedTopology for the specific name, or Optional.empty() if the application has no NamedTopology of that name */ - public Optional lookupTopologyByName(final String name) { + public Optional getTopologyByName(final String name) { return Optional.ofNullable(topologyMetadata.lookupBuilderForNamedTopology(name)).map(InternalTopologyBuilder::namedTopology); } @@ -106,7 +128,7 @@ public Optional lookupTopologyByName(final String name) { public void addNamedTopology(final NamedTopology newTopology) { if (hasStartedOrFinishedShuttingDown()) { throw new IllegalStateException("Cannot add a NamedTopology while the state is " + super.state); - } else if (lookupTopologyByName(newTopology.name()).isPresent()) { + } else if (getTopologyByName(newTopology.name()).isPresent()) { throw new IllegalArgumentException("Unable to add the new NamedTopology " + newTopology.name() + " as another of the same name already exists"); } @@ -125,7 +147,7 @@ public void addNamedTopology(final NamedTopology newTopology) { public void removeNamedTopology(final String topologyToRemove) { if (!isRunningOrRebalancing()) { throw new IllegalStateException("Cannot remove a NamedTopology while the state is " + super.state); - } else if (!lookupTopologyByName(topologyToRemove).isPresent()) { + } else if (!getTopologyByName(topologyToRemove).isPresent()) { throw new IllegalArgumentException("Unable to locate for removal a NamedTopology called " + topologyToRemove); } @@ -146,7 +168,7 @@ public void removeNamedTopology(final String topologyToRemove) { * @throws StreamsException if cleanup failed */ public void cleanUpNamedTopology(final String name) { - if (lookupTopologyByName(name).isPresent()) { + if (getTopologyByName(name).isPresent()) { throw new IllegalStateException("Can't clean up local state for an active NamedTopology: " + name); } stateDirectory.clearLocalStateForNamedTopology(name); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java index 21b9d7ce56cc4..a1debbe4b5620 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java @@ -23,7 +23,7 @@ public class NamedTopology extends Topology { - public NamedTopology(final InternalTopologyBuilder internalTopologyBuilder) { + NamedTopology(final InternalTopologyBuilder internalTopologyBuilder) { super(internalTopologyBuilder); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java index bf506e6cafd5e..ddd9192d53e55 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyBuilder.java @@ -36,7 +36,7 @@ public synchronized NamedTopology build() { } @Override - public NamedTopology getNewTopology(final TopologyConfig topologyConfigs) { + protected NamedTopology getNewTopology(final TopologyConfig topologyConfigs) { return new NamedTopology(new InternalTopologyBuilder(topologyConfigs)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java index 6828822a25a4e..ff3f39b8b22e6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/NamedTopologyIntegrationTest.java @@ -35,7 +35,6 @@ import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; -import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; @@ -51,7 +50,6 @@ import org.junit.Test; import org.junit.rules.TestName; import java.time.Duration; -import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Properties; @@ -62,6 +60,7 @@ import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.streams.KeyValue.pair; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived; import static org.hamcrest.CoreMatchers.equalTo; @@ -73,6 +72,10 @@ public class NamedTopologyIntegrationTest { public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + private static final String TOPOLOGY_1 = "topology-1"; + private static final String TOPOLOGY_2 = "topology-2"; + private static final String TOPOLOGY_3 = "topology-3"; + // TODO KAFKA-12648: // 1) full test coverage for add/removeNamedTopology, covering: // - the "last topology removed" case @@ -90,7 +93,6 @@ public class NamedTopologyIntegrationTest { private final static String SUM_OUTPUT = "sum"; private final static String COUNT_OUTPUT = "count"; - // "delayed" input topics which are empty at start to allow control over when input data appears private final static String DELAYED_INPUT_STREAM_1 = "delayed-input-stream-1"; private final static String DELAYED_INPUT_STREAM_2 = "delayed-input-stream-2"; @@ -141,23 +143,23 @@ public static void closeCluster() { private final KafkaClientSupplier clientSupplier = new DefaultKafkaClientSupplier(); - // builders for the 1st Streams instance (default) - private final NamedTopologyBuilder topology1Builder = new NamedTopologyBuilder("topology-1"); - private final NamedTopologyBuilder topology2Builder = new NamedTopologyBuilder("topology-2"); - private final NamedTopologyBuilder topology3Builder = new NamedTopologyBuilder("topology-3"); - - // builders for the 2nd Streams instance - private final NamedTopologyBuilder topology1Builder2 = new NamedTopologyBuilder("topology-1"); - private final NamedTopologyBuilder topology2Builder2 = new NamedTopologyBuilder("topology-2"); - private final NamedTopologyBuilder topology3Builder2 = new NamedTopologyBuilder("topology-3"); - private Properties props; private Properties props2; private KafkaStreamsNamedTopologyWrapper streams; private KafkaStreamsNamedTopologyWrapper streams2; - private Properties configProps() { + // builders for the 1st Streams instance (default) + private NamedTopologyBuilder topology1Builder; + private NamedTopologyBuilder topology2Builder; + private NamedTopologyBuilder topology3Builder; + + // builders for the 2nd Streams instance + private NamedTopologyBuilder topology1Builder2; + private NamedTopologyBuilder topology2Builder2; + private NamedTopologyBuilder topology3Builder2; + + private Properties configProps(final String appId) { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); @@ -174,11 +176,16 @@ private Properties configProps() { @Before public void setup() throws Exception { appId = safeUniqueTestName(NamedTopologyIntegrationTest.class, testName); - changelog1 = appId + "-topology-1-store-changelog"; - changelog2 = appId + "-topology-2-store-changelog"; - changelog3 = appId + "-topology-3-store-changelog"; - props = configProps(); - props2 = configProps(); + changelog1 = appId + "-" + TOPOLOGY_1 + "-store-changelog"; + changelog2 = appId + "-" + TOPOLOGY_2 + "-store-changelog"; + changelog3 = appId + "-" + TOPOLOGY_3 + "-store-changelog"; + props = configProps(appId); + + streams = new KafkaStreamsNamedTopologyWrapper(props, clientSupplier); + + topology1Builder = streams.newNamedTopologyBuilder(TOPOLOGY_1); + topology2Builder = streams.newNamedTopologyBuilder(TOPOLOGY_2); + topology3Builder = streams.newNamedTopologyBuilder(TOPOLOGY_3); // TODO KAFKA-12648: refactor to avoid deleting & (re)creating outputs topics for each test CLUSTER.createTopic(OUTPUT_STREAM_1, 2, 1); @@ -186,6 +193,14 @@ public void setup() throws Exception { CLUSTER.createTopic(OUTPUT_STREAM_3, 2, 1); } + private void setupSecondKafkaStreams() { + props2 = configProps(appId); + streams2 = new KafkaStreamsNamedTopologyWrapper(props2, clientSupplier); + topology1Builder2 = streams2.newNamedTopologyBuilder(TOPOLOGY_1); + topology2Builder2 = streams2.newNamedTopologyBuilder(TOPOLOGY_2); + topology3Builder2 = streams2.newNamedTopologyBuilder(TOPOLOGY_3); + } + @After public void shutdown() throws Exception { if (streams != null) { @@ -203,10 +218,10 @@ public void shouldPrefixAllInternalTopicNamesWithNamedTopology() throws Exceptio final String countTopologyName = "count-topology"; final String fkjTopologyName = "FKJ-topology"; - final NamedTopologyBuilder countBuilder = new NamedTopologyBuilder(countTopologyName); + final NamedTopologyBuilder countBuilder = streams.newNamedTopologyBuilder(countTopologyName); countBuilder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(); - final NamedTopologyBuilder fkjBuilder = new NamedTopologyBuilder(fkjTopologyName); + final NamedTopologyBuilder fkjBuilder = streams.newNamedTopologyBuilder(fkjTopologyName); final UniqueTopicSerdeScope serdeScope = new UniqueTopicSerdeScope(); final KTable left = fkjBuilder.table( @@ -225,8 +240,8 @@ public void shouldPrefixAllInternalTopicNamesWithNamedTopology() throws Exceptio (value1, value2) -> String.valueOf(value1 + value2), Materialized.with(null, serdeScope.decorateSerde(Serdes.String(), props, false))); - streams = new KafkaStreamsNamedTopologyWrapper(buildNamedTopologies(fkjBuilder, countBuilder), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); + streams.start(asList(fkjBuilder.build(), countBuilder.build())); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(60)); final String countTopicPrefix = appId + "-" + countTopologyName; final String fkjTopicPrefix = appId + "-" + fkjTopologyName; @@ -253,8 +268,8 @@ public void shouldProcessSingleNamedTopologyAndPrefixInternalTopics() throws Exc .groupByKey() .count(ROCKSDB_STORE) .toStream().to(OUTPUT_STREAM_1); - streams = new KafkaStreamsNamedTopologyWrapper(topology1Builder.buildNamedTopology(props), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); + streams.start(topology1Builder.build()); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(30)); final List> results = waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3); assertThat(results, equalTo(COUNT_OUTPUT_DATA)); @@ -268,8 +283,8 @@ public void shouldProcessMultipleIdenticalNamedTopologiesWithInMemoryAndPersiste topology1Builder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(ROCKSDB_STORE).toStream().to(OUTPUT_STREAM_1); topology2Builder.stream(INPUT_STREAM_2).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2); topology3Builder.stream(INPUT_STREAM_3).groupBy((k, v) -> k).count(ROCKSDB_STORE).toStream().to(OUTPUT_STREAM_3); - streams = new KafkaStreamsNamedTopologyWrapper(buildNamedTopologies(topology1Builder, topology2Builder, topology3Builder), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); + streams.start(asList(topology1Builder.build(), topology2Builder.build(), topology3Builder.build())); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(30)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), equalTo(COUNT_OUTPUT_DATA)); @@ -281,8 +296,7 @@ public void shouldProcessMultipleIdenticalNamedTopologiesWithInMemoryAndPersiste @Test public void shouldAddNamedTopologyToUnstartedApplicationWithEmptyInitialTopology() throws Exception { topology1Builder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_1); - streams = new KafkaStreamsNamedTopologyWrapper(props, clientSupplier); - streams.addNamedTopology(topology1Builder.buildNamedTopology(props)); + streams.addNamedTopology(topology1Builder.build()); IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); @@ -291,10 +305,9 @@ public void shouldAddNamedTopologyToUnstartedApplicationWithEmptyInitialTopology @Test public void shouldAddNamedTopologyToRunningApplicationWithEmptyInitialTopology() throws Exception { topology1Builder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_1); - streams = new KafkaStreamsNamedTopologyWrapper(props, clientSupplier); streams.start(); - streams.addNamedTopology(topology1Builder.buildNamedTopology(props)); + streams.addNamedTopology(topology1Builder.build()); IntegrationTestUtils.waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(15)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); @@ -304,10 +317,9 @@ public void shouldAddNamedTopologyToRunningApplicationWithEmptyInitialTopology() public void shouldAddNamedTopologyToRunningApplicationWithSingleInitialNamedTopology() throws Exception { topology1Builder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_1); topology2Builder.stream(INPUT_STREAM_2).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2); - streams = new KafkaStreamsNamedTopologyWrapper(topology1Builder.buildNamedTopology(props), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); - - streams.addNamedTopology(topology2Builder.buildNamedTopology(props)); + streams.start(topology1Builder.build()); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(30)); + streams.addNamedTopology(topology2Builder.build()); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), equalTo(COUNT_OUTPUT_DATA)); @@ -318,10 +330,10 @@ public void shouldAddNamedTopologyToRunningApplicationWithMultipleInitialNamedTo topology1Builder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(ROCKSDB_STORE).toStream().to(OUTPUT_STREAM_1); topology2Builder.stream(INPUT_STREAM_2).groupBy((k, v) -> k).count(ROCKSDB_STORE).toStream().to(OUTPUT_STREAM_2); topology3Builder.stream(INPUT_STREAM_3).groupBy((k, v) -> k).count(ROCKSDB_STORE).toStream().to(OUTPUT_STREAM_3); - streams = new KafkaStreamsNamedTopologyWrapper(buildNamedTopologies(topology1Builder, topology2Builder), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); + streams.start(asList(topology1Builder.build(), topology2Builder.build())); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(30)); - streams.addNamedTopology(topology3Builder.buildNamedTopology(props)); + streams.addNamedTopology(topology3Builder.build()); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), equalTo(COUNT_OUTPUT_DATA)); @@ -330,18 +342,19 @@ public void shouldAddNamedTopologyToRunningApplicationWithMultipleInitialNamedTo @Test public void shouldAddNamedTopologyToRunningApplicationWithMultipleNodes() throws Exception { + setupSecondKafkaStreams(); topology1Builder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_1); topology1Builder2.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_1); topology2Builder.stream(INPUT_STREAM_2).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2); topology2Builder2.stream(INPUT_STREAM_2).groupBy((k, v) -> k).count(IN_MEMORY_STORE).toStream().to(OUTPUT_STREAM_2); - streams = new KafkaStreamsNamedTopologyWrapper(topology1Builder.buildNamedTopology(props), props, clientSupplier); - streams2 = new KafkaStreamsNamedTopologyWrapper(topology1Builder2.buildNamedTopology(props2), props2, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(asList(streams, streams2), Duration.ofSeconds(15)); + streams.start(topology1Builder.build()); + streams2.start(topology1Builder2.build()); + waitForApplicationState(asList(streams, streams2), State.RUNNING, Duration.ofSeconds(30)); - streams.addNamedTopology(topology2Builder.buildNamedTopology(props)); - streams2.addNamedTopology(topology2Builder2.buildNamedTopology(props2)); + streams.addNamedTopology(topology2Builder.build()); + streams2.addNamedTopology(topology2Builder2.build()); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), equalTo(COUNT_OUTPUT_DATA)); @@ -359,8 +372,8 @@ public void shouldRemoveOneNamedTopologyWhileAnotherContinuesProcessing() throws topology2Builder.stream(DELAYED_INPUT_STREAM_2).map((k, v) -> { throw new IllegalStateException("Should not process any records for removed topology-2"); }); - streams = new KafkaStreamsNamedTopologyWrapper(buildNamedTopologies(topology1Builder, topology2Builder), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); + streams.start(asList(topology1Builder.build(), topology2Builder.build())); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(30)); streams.removeNamedTopology("topology-2"); @@ -378,8 +391,9 @@ public void shouldRemoveAndReplaceTopologicallyIncompatibleNamedTopology() throw final KStream inputStream1 = topology1Builder.stream(INPUT_STREAM_1); inputStream1.groupByKey().count().toStream().to(COUNT_OUTPUT); inputStream1.groupByKey().reduce(Long::sum).toStream().to(SUM_OUTPUT); - streams = new KafkaStreamsNamedTopologyWrapper(buildNamedTopologies(topology1Builder), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); + streams = new KafkaStreamsNamedTopologyWrapper(props, clientSupplier); + streams.start(singletonList(topology1Builder.build())); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(30)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, COUNT_OUTPUT, 3), equalTo(COUNT_OUTPUT_DATA)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, SUM_OUTPUT, 3), equalTo(SUM_OUTPUT_DATA)); @@ -387,12 +401,13 @@ public void shouldRemoveAndReplaceTopologicallyIncompatibleNamedTopology() throw streams.cleanUpNamedTopology("topology-1"); // Prepare a new named topology with the same name but an incompatible topology (stateful subtopologies swap order) + final NamedTopologyBuilder topology1Builder2 = streams.newNamedTopologyBuilder(TOPOLOGY_1); final KStream inputStream2 = topology1Builder2.stream(DELAYED_INPUT_STREAM_1); inputStream2.groupByKey().reduce(Long::sum).toStream().to(SUM_OUTPUT); inputStream2.groupByKey().count().toStream().to(COUNT_OUTPUT); produceToInputTopics(DELAYED_INPUT_STREAM_1, STANDARD_INPUT_DATA); - streams.addNamedTopology(topology1Builder2.buildNamedTopology(props)); + streams.addNamedTopology(topology1Builder2.build()); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, COUNT_OUTPUT, 3), equalTo(COUNT_OUTPUT_DATA)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, SUM_OUTPUT, 3), equalTo(SUM_OUTPUT_DATA)); @@ -404,8 +419,8 @@ public void shouldAllowPatternSubscriptionWithMultipleNamedTopologies() throws E topology1Builder.stream(Pattern.compile(INPUT_STREAM_1)).groupBy((k, v) -> k).count().toStream().to(OUTPUT_STREAM_1); topology2Builder.stream(Pattern.compile(INPUT_STREAM_2)).groupBy((k, v) -> k).count().toStream().to(OUTPUT_STREAM_2); topology3Builder.stream(Pattern.compile(INPUT_STREAM_3)).groupBy((k, v) -> k).count().toStream().to(OUTPUT_STREAM_3); - streams = new KafkaStreamsNamedTopologyWrapper(buildNamedTopologies(topology1Builder, topology2Builder, topology3Builder), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); + streams.start(asList(topology1Builder.build(), topology2Builder.build(), topology3Builder.build())); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(30)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), equalTo(COUNT_OUTPUT_DATA)); @@ -417,8 +432,8 @@ public void shouldAllowMixedCollectionAndPatternSubscriptionWithMultipleNamedTop topology1Builder.stream(INPUT_STREAM_1).groupBy((k, v) -> k).count().toStream().to(OUTPUT_STREAM_1); topology2Builder.stream(Pattern.compile(INPUT_STREAM_2)).groupBy((k, v) -> k).count().toStream().to(OUTPUT_STREAM_2); topology3Builder.stream(Pattern.compile(INPUT_STREAM_3)).groupBy((k, v) -> k).count().toStream().to(OUTPUT_STREAM_3); - streams = new KafkaStreamsNamedTopologyWrapper(buildNamedTopologies(topology1Builder, topology2Builder, topology3Builder), props, clientSupplier); - IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(15)); + streams.start(asList(topology1Builder.build(), topology2Builder.build(), topology3Builder.build())); + waitForApplicationState(singletonList(streams), State.RUNNING, Duration.ofSeconds(30)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_1, 3), equalTo(COUNT_OUTPUT_DATA)); assertThat(waitUntilMinKeyValueRecordsReceived(consumerConfig, OUTPUT_STREAM_2, 3), equalTo(COUNT_OUTPUT_DATA)); @@ -433,12 +448,4 @@ private static void produceToInputTopics(final String topic, final Collection buildNamedTopologies(final NamedTopologyBuilder... builders) { - final List topologies = new ArrayList<>(); - for (final NamedTopologyBuilder builder : builders) { - topologies.add(builder.buildNamedTopology(props)); - } - return topologies; - } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java index cbf28bff2ba33..f9d2708e98b63 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KafkaStreams.State; import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StoreQueryParameters; @@ -66,6 +67,7 @@ import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.getStore; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; import static org.apache.kafka.streams.state.QueryableStoreTypes.keyValueStore; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; @@ -397,23 +399,25 @@ public void shouldQuerySpecificStalePartitionStoresMultiStreamThreadsNamedTopolo final Semaphore semaphore = new Semaphore(0); final int numStreamThreads = 2; - final NamedTopologyBuilder builder1A = new NamedTopologyBuilder("topology-A"); - getStreamsBuilderWithTopology(builder1A, semaphore); - - final NamedTopologyBuilder builder2A = new NamedTopologyBuilder("topology-A"); - getStreamsBuilderWithTopology(builder2A, semaphore); - final Properties streamsConfiguration1 = streamsConfiguration(); streamsConfiguration1.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, numStreamThreads); final Properties streamsConfiguration2 = streamsConfiguration(); streamsConfiguration2.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, numStreamThreads); - final KafkaStreamsNamedTopologyWrapper kafkaStreams1 = createNamedTopologyKafkaStreams(builder1A, streamsConfiguration1); - final KafkaStreamsNamedTopologyWrapper kafkaStreams2 = createNamedTopologyKafkaStreams(builder2A, streamsConfiguration2); + final KafkaStreamsNamedTopologyWrapper kafkaStreams1 = createNamedTopologyKafkaStreams(streamsConfiguration1); + final KafkaStreamsNamedTopologyWrapper kafkaStreams2 = createNamedTopologyKafkaStreams(streamsConfiguration2); final List kafkaStreamsList = Arrays.asList(kafkaStreams1, kafkaStreams2); - startApplicationAndWaitUntilRunning(kafkaStreamsList, Duration.ofSeconds(60)); + final NamedTopologyBuilder builder1A = kafkaStreams1.newNamedTopologyBuilder("topology-A", streamsConfiguration1); + getStreamsBuilderWithTopology(builder1A, semaphore); + + final NamedTopologyBuilder builder2A = kafkaStreams2.newNamedTopologyBuilder("topology-A", streamsConfiguration2); + getStreamsBuilderWithTopology(builder2A, semaphore); + + kafkaStreams1.start(builder1A.build()); + kafkaStreams2.start(builder2A.build()); + waitForApplicationState(kafkaStreamsList, State.RUNNING, Duration.ofSeconds(60)); assertTrue(kafkaStreams1.metadataForLocalThreads().size() > 1); assertTrue(kafkaStreams2.metadataForLocalThreads().size() > 1); @@ -567,8 +571,8 @@ private KafkaStreams createKafkaStreams(final StreamsBuilder builder, final Prop return streams; } - private KafkaStreamsNamedTopologyWrapper createNamedTopologyKafkaStreams(final NamedTopologyBuilder builder, final Properties config) { - final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(builder.buildNamedTopology(config), config); + private KafkaStreamsNamedTopologyWrapper createNamedTopologyKafkaStreams(final Properties config) { + final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(config); streamsToCleanup.add(streams); return streams; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java index e6e9318f50fb0..5253a1d708bec 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java @@ -41,11 +41,11 @@ public class NamedTopologyTest { final KafkaClientSupplier clientSupplier = new DefaultKafkaClientSupplier(); final Properties props = configProps(); - final NamedTopologyBuilder builder1 = new NamedTopologyBuilder("topology-1"); - final NamedTopologyBuilder builder2 = new NamedTopologyBuilder("topology-2"); - final NamedTopologyBuilder builder3 = new NamedTopologyBuilder("topology-3"); + private final KafkaStreamsNamedTopologyWrapper streams = new KafkaStreamsNamedTopologyWrapper(props); - KafkaStreamsNamedTopologyWrapper streams; + private final NamedTopologyBuilder builder1 = streams.newNamedTopologyBuilder("topology-1"); + private final NamedTopologyBuilder builder2 = streams.newNamedTopologyBuilder("topology-2"); + private final NamedTopologyBuilder builder3 = streams.newNamedTopologyBuilder("topology-3"); @Before public void setup() { @@ -56,9 +56,7 @@ public void setup() { @After public void cleanup() { - if (streams != null) { - streams.close(); - } + streams.close(); } private static Properties configProps() { @@ -71,14 +69,13 @@ private static Properties configProps() { @Test public void shouldThrowIllegalArgumentOnIllegalName() { - assertThrows(IllegalArgumentException.class, () -> new NamedTopologyBuilder("__not-allowed__")); + assertThrows(IllegalArgumentException.class, () -> streams.newNamedTopologyBuilder("__not-allowed__")); } @Test public void shouldBuildSingleNamedTopology() { builder1.stream("stream-1").filter((k, v) -> !k.equals(v)).to("output-1"); - - streams = new KafkaStreamsNamedTopologyWrapper(builder1.buildNamedTopology(props), props, clientSupplier); + streams.start(builder1.build()); } @Test @@ -87,31 +84,29 @@ public void shouldBuildMultipleIdenticalNamedTopologyWithRepartition() { builder2.stream("stream-2").selectKey((k, v) -> v).groupByKey().count().toStream().to("output-2"); builder3.stream("stream-3").selectKey((k, v) -> v).groupByKey().count().toStream().to("output-3"); - streams = new KafkaStreamsNamedTopologyWrapper( + streams.start( asList( - builder1.buildNamedTopology(props), - builder2.buildNamedTopology(props), - builder3.buildNamedTopology(props)), - props, - clientSupplier + builder1.build(), + builder2.build(), + builder3.build()) ); } @Test public void shouldReturnTopologyByName() { - final NamedTopology topology1 = builder1.buildNamedTopology(props); - final NamedTopology topology2 = builder2.buildNamedTopology(props); - final NamedTopology topology3 = builder3.buildNamedTopology(props); - streams = new KafkaStreamsNamedTopologyWrapper(asList(topology1, topology2, topology3), props, clientSupplier); - assertThat(streams.lookupTopologyByName("topology-1").get(), equalTo(topology1)); - assertThat(streams.lookupTopologyByName("topology-2").get(), equalTo(topology2)); - assertThat(streams.lookupTopologyByName("topology-3").get(), equalTo(topology3)); + final NamedTopology topology1 = builder1.build(); + final NamedTopology topology2 = builder2.build(); + final NamedTopology topology3 = builder3.build(); + streams.start(asList(topology1, topology2, topology3)); + assertThat(streams.getTopologyByName("topology-1").get(), equalTo(topology1)); + assertThat(streams.getTopologyByName("topology-2").get(), equalTo(topology2)); + assertThat(streams.getTopologyByName("topology-3").get(), equalTo(topology3)); } @Test public void shouldReturnEmptyWhenLookingUpNonExistentTopologyByName() { - streams = new KafkaStreamsNamedTopologyWrapper(builder1.buildNamedTopology(props), props, clientSupplier); - assertThat(streams.lookupTopologyByName("non-existent-topology").isPresent(), equalTo(false)); + streams.start(builder1.build()); + assertThat(streams.getTopologyByName("non-existent-topology").isPresent(), equalTo(false)); } @Test @@ -119,12 +114,7 @@ public void shouldAllowSameStoreNameToBeUsedByMultipleNamedTopologies() { builder1.stream("stream-1").selectKey((k, v) -> v).groupByKey().count(Materialized.as(Stores.inMemoryKeyValueStore("store"))); builder2.stream("stream-2").selectKey((k, v) -> v).groupByKey().count(Materialized.as(Stores.inMemoryKeyValueStore("store"))); - streams = new KafkaStreamsNamedTopologyWrapper(asList( - builder1.buildNamedTopology(props), - builder2.buildNamedTopology(props)), - props, - clientSupplier - ); + streams.start(asList(builder1.build(), builder2.build())); } @Test @@ -134,12 +124,9 @@ public void shouldThrowTopologyExceptionWhenMultipleNamedTopologiesCreateStreamF assertThrows( TopologyException.class, - () -> streams = new KafkaStreamsNamedTopologyWrapper( - asList( - builder1.buildNamedTopology(props), - builder2.buildNamedTopology(props)), - props, - clientSupplier) + () -> streams.start(asList( + builder1.build(), + builder2.build())) ); } @@ -150,12 +137,9 @@ public void shouldThrowTopologyExceptionWhenMultipleNamedTopologiesCreateTableFr assertThrows( TopologyException.class, - () -> streams = new KafkaStreamsNamedTopologyWrapper( - asList( - builder1.buildNamedTopology(props), - builder2.buildNamedTopology(props)), - props, - clientSupplier) + () -> streams.start(asList( + builder1.build(), + builder2.build())) ); } @@ -166,12 +150,9 @@ public void shouldThrowTopologyExceptionWhenMultipleNamedTopologiesCreateStreamA assertThrows( TopologyException.class, - () -> streams = new KafkaStreamsNamedTopologyWrapper( - asList( - builder1.buildNamedTopology(props), - builder2.buildNamedTopology(props)), - props, - clientSupplier) + () -> streams.start(asList( + builder1.build(), + builder2.build())) ); } @@ -182,12 +163,9 @@ public void shouldThrowTopologyExceptionWhenMultipleNamedTopologiesCreateStreamF assertThrows( TopologyException.class, - () -> streams = new KafkaStreamsNamedTopologyWrapper( - asList( - builder1.buildNamedTopology(props), - builder2.buildNamedTopology(props)), - props, - clientSupplier) + () -> streams.start(asList( + builder1.build(), + builder2.build())) ); } @@ -198,19 +176,16 @@ public void shouldThrowTopologyExceptionWhenMultipleNamedTopologiesCreateStreamF assertThrows( TopologyException.class, - () -> streams = new KafkaStreamsNamedTopologyWrapper( - asList( - builder1.buildNamedTopology(props), - builder2.buildNamedTopology(props)), - props, - clientSupplier) + () -> streams.start(asList( + builder1.build(), + builder2.build())) ); } @Test public void shouldDescribeWithSingleNamedTopology() { builder1.stream("input").filter((k, v) -> !k.equals(v)).to("output"); - streams = new KafkaStreamsNamedTopologyWrapper(builder1.buildNamedTopology(props), props, clientSupplier); + streams.start(builder1.build()); assertThat( streams.getFullTopologyDescription(), @@ -237,13 +212,11 @@ public void shouldDescribeWithMultipleNamedTopologies() { builder2.stream("stream-2").filter((k, v) -> !k.equals(v)).to("output-2"); builder3.stream("stream-3").filter((k, v) -> !k.equals(v)).to("output-3"); - streams = new KafkaStreamsNamedTopologyWrapper( + streams.start( asList( - builder1.buildNamedTopology(props), - builder2.buildNamedTopology(props), - builder3.buildNamedTopology(props)), - props, - clientSupplier + builder1.build(), + builder2.build(), + builder3.build()) ); assertThat( @@ -295,8 +268,6 @@ public void shouldDescribeWithMultipleNamedTopologies() { @Test public void shouldDescribeWithEmptyNamedTopology() { - streams = new KafkaStreamsNamedTopologyWrapper(props, clientSupplier); - assertThat(streams.getFullTopologyDescription(), equalTo("")); } } From b8583b306760e91b6a2874c5195e6419853566b6 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 30 Sep 2021 20:29:38 -0700 Subject: [PATCH 13/18] fixin' up the tests --- .../processor/internals/TopologyMetadata.java | 3 +- .../processor/internals/StreamThreadTest.java | 62 +++++++++++++------ 2 files changed, 45 insertions(+), 20 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java index c7bb2ce228de6..7714d3bba1140 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java @@ -163,7 +163,8 @@ public void unregisterTopology(final String topologyName) { } public TaskConfig getTaskConfigFor(final TaskId taskId) { - return lookupBuilderForTask(taskId).topologyConfigs().getTaskConfig(); + final InternalTopologyBuilder builder = lookupBuilderForTask(taskId); + return builder.topologyConfigs().getTaskConfig(); } public void buildAndRewriteTopology() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index c735978978119..3723a58801f0d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -227,10 +227,11 @@ private StreamThread createStreamThread(@SuppressWarnings("SameParameterValue") mockTime ); - internalTopologyBuilder.buildTopology(); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); return StreamThread.create( - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, config, clientSupplier, clientSupplier.getAdmin(config.getAdminConfigs(clientId)), @@ -443,6 +444,7 @@ public void shouldNotCommitBeforeTheCommitInterval() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final StreamThread thread = new StreamThread( mockTime, config, @@ -453,7 +455,7 @@ public void shouldNotCommitBeforeTheCommitInterval() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -492,8 +494,10 @@ public void shouldEnforceRebalanceAfterNextScheduledProbingRebalanceTime() throw mockClientSupplier.setCluster(createCluster()); EasyMock.replay(mockConsumer); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = StreamThread.create( - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, config, mockClientSupplier, mockClientSupplier.getAdmin(config.getAdminConfigs(CLIENT_ID)), @@ -692,6 +696,8 @@ public void shouldNotCauseExceptionIfNothingCommitted() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, config, @@ -702,7 +708,7 @@ public void shouldNotCauseExceptionIfNothingCommitted() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -762,6 +768,8 @@ int commit(final Collection tasksToCommit) { } }; + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, config, @@ -772,7 +780,7 @@ int commit(final Collection tasksToCommit) { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -827,6 +835,8 @@ public void shouldRecordCommitLatency() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final TaskManager taskManager = new TaskManager( null, @@ -836,7 +846,7 @@ public void shouldRecordCommitLatency() { null, activeTaskCreator, standbyTaskCreator, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, null, null, null @@ -859,7 +869,7 @@ int commit(final Collection tasksToCommit) { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -1168,7 +1178,6 @@ public void shouldNotReturnDataAfterTaskMigrated() { final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class); expect(taskManager.producerClientIds()).andStubReturn(Collections.emptySet()); final InternalTopologyBuilder internalTopologyBuilder = EasyMock.createNiceMock(InternalTopologyBuilder.class); - expect(internalTopologyBuilder.sourceTopicCollection()).andReturn(Collections.singletonList(topic1)).times(2); final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.LATEST); @@ -1190,7 +1199,6 @@ public void restore(final Map tasks) { }; taskManager.handleLostAll(); - EasyMock.replay(taskManager, internalTopologyBuilder); final StreamsMetricsImpl streamsMetrics = @@ -1241,6 +1249,8 @@ public void shouldShutdownTaskManagerOnCloseWithoutStart() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, config, @@ -1251,7 +1261,7 @@ public void shouldShutdownTaskManagerOnCloseWithoutStart() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -1279,6 +1289,8 @@ public void shouldOnlyShutdownOnce() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, config, @@ -1289,7 +1301,7 @@ public void shouldOnlyShutdownOnce() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -1615,10 +1627,11 @@ public void shouldReturnActiveTaskMetadataWhileRunningState() { mockTime ); - internalTopologyBuilder.buildTopology(); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = StreamThread.create( - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, config, clientSupplier, clientSupplier.getAdmin(config.getAdminConfigs(CLIENT_ID)), @@ -2186,6 +2199,8 @@ public void shouldThrowTaskMigratedExceptionHandlingTaskLost() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, config, @@ -2196,7 +2211,7 @@ public void shouldThrowTaskMigratedExceptionHandlingTaskLost() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -2234,6 +2249,8 @@ public void shouldThrowTaskMigratedExceptionHandlingRevocation() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, config, @@ -2244,7 +2261,7 @@ public void shouldThrowTaskMigratedExceptionHandlingRevocation() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -2640,6 +2657,8 @@ public void shouldNotCommitNonRunningNonRestoringTasks() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, config, @@ -2650,7 +2669,7 @@ public void shouldNotCommitNonRunningNonRestoringTasks() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -2782,6 +2801,8 @@ public void shouldTransmitTaskManagerMetrics() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, new StreamsConfig(configProps(true)), @@ -2792,7 +2813,7 @@ public void shouldTransmitTaskManagerMetrics() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -2823,6 +2844,8 @@ public void shouldConstructAdminMetrics() { final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST, mockTime); + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); final StreamThread thread = new StreamThread( mockTime, config, @@ -2833,7 +2856,7 @@ public void shouldConstructAdminMetrics() { null, taskManager, streamsMetrics, - new TopologyMetadata(internalTopologyBuilder, config), + topologyMetadata, CLIENT_ID, new LogContext(""), new AtomicInteger(), @@ -2938,6 +2961,7 @@ private void setupInternalTopologyWithoutState() { (ProcessorSupplier) MockApiProcessor::new, "source1" ); + internalTopologyBuilder.setStreamsConfig(config); } // TODO: change return type to `StandbyTask` From e2c8208487496e8dc8085fd73c1b82eb01af42ea Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Sat, 9 Oct 2021 03:38:14 -0500 Subject: [PATCH 14/18] test fixes and cache 0 case --- .../apache/kafka/streams/KafkaStreams.java | 1 + .../apache/kafka/streams/StreamsConfig.java | 2 +- .../internals/InternalTopologyBuilder.java | 9 ++-- .../processor/internals/StandbyTask.java | 2 +- .../namedtopology/TopologyConfig.java | 31 +++++++++---- .../internals/ActiveTaskCreatorTest.java | 4 +- .../InternalTopologyBuilderTest.java | 45 ++++++++++++------- .../internals/NamedTopologyTest.java | 8 ++-- .../processor/internals/StandbyTaskTest.java | 4 +- 9 files changed, 67 insertions(+), 39 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index a44a6c41748b6..bb0c40a83dc05 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -964,6 +964,7 @@ private StreamThread createAndAddStreamThread(final long cacheSizePerThread, fin time, streamsMetadataState, cacheSizePerThread, + stateDirectory, delegatingStateRestoreListener, threadIdx, diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 7a0b420910d2e..2ebd5543ad0bc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -358,7 +358,7 @@ public class StreamsConfig extends AbstractConfig { /** {@code cache.max.bytes.buffering} */ @SuppressWarnings("WeakerAccess") public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering"; - private 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"; /** {@code client.id} */ @SuppressWarnings("WeakerAccess") diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index b5e1d50c23260..fd982f71078c5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -148,9 +148,8 @@ public InternalTopologyBuilder(final TopologyConfig topologyConfigs) { // TODO KAFKA-13336: we can remove this referance once we make the Topology/NamedTopology class into an interface and implement it private NamedTopology namedTopology; - // TODO KAFKA-13283: we still need to save the topology overrides here since we don't get the app configs until we - // get passed in to the KafkaStreams, once we enforce all configs be passed in when constructing the topology then - // we can make topologyConfigs final and drop the other + // TODO KAFKA-13283: once we enforce all configs be passed in when constructing the topology builder then we can set + // this up front and make it final, but for now we have to wait for the global app configs passed in to rewriteTopology private TopologyConfig topologyConfigs; // the configs for this topology, including overrides and global defaults private boolean hasPersistentStores = false; @@ -369,7 +368,7 @@ public synchronized final void setNamedTopology(final NamedTopology namedTopolo this.namedTopology = namedTopology; } - public synchronized final TopologyConfig topologyConfigs() { + public synchronized TopologyConfig topologyConfigs() { return topologyConfigs; } @@ -388,7 +387,7 @@ public synchronized final InternalTopologyBuilder rewriteTopology(final StreamsC setStreamsConfig(config); // maybe strip out caching layers - if (config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG) == 0L) { + if (topologyConfigs.cacheSize == 0L) { for (final StateStoreFactory storeFactory : stateFactories.values()) { storeFactory.builder.withCachingDisabled(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index 352b4b7980789..6b3d6794e18d1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -68,7 +68,7 @@ public class StandbyTask extends AbstractTask implements Task { stateDirectory, stateMgr, inputPartitions, - config.maxTaskIdleMs, + config.taskTimeoutMs, "standby-task", StandbyTask.class ); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java index 55e9861d8224f..1fe2b852ab1c6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java @@ -32,6 +32,8 @@ 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.CACHE_MAX_BYTES_BUFFERING_CONFIG; +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_DOC; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG; @@ -56,6 +58,11 @@ public class TopologyConfig extends AbstractConfig { null, Importance.LOW, BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, + Type.LONG, + null, + Importance.MEDIUM, + CACHE_MAX_BYTES_BUFFERING_DOC) .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, Type.CLASS, null, @@ -85,9 +92,10 @@ public class TopologyConfig extends AbstractConfig { public final StreamsConfig applicationConfigs; public final Properties topologyOverrides; + public final int maxBufferedSize; + public final long cacheSize; public final long maxTaskIdleMs; public final long taskTimeoutMs; - public final int maxBufferedSize; public final Supplier timestampExtractorSupplier; public final Supplier deserializationExceptionHandlerSupplier; @@ -104,6 +112,20 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo this.applicationConfigs = globalAppConfigs; this.topologyOverrides = topologyOverrides; + if (isTopologyOverride(BUFFERED_RECORDS_PER_PARTITION_CONFIG, topologyOverrides)) { + maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize); + } else { + maxBufferedSize = globalAppConfigs.getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); + } + + if (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); + } else { + cacheSize = globalAppConfigs.getLong(CACHE_MAX_BYTES_BUFFERING_CONFIG); + } + if (isTopologyOverride(MAX_TASK_IDLE_MS_CONFIG, topologyOverrides)) { maxTaskIdleMs = getLong(MAX_TASK_IDLE_MS_CONFIG); log.info("Topology {} is overridding {} to {}", topologyName, MAX_TASK_IDLE_MS_CONFIG, maxTaskIdleMs); @@ -118,13 +140,6 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo taskTimeoutMs = globalAppConfigs.getLong(TASK_TIMEOUT_MS_CONFIG); } - if (isTopologyOverride(BUFFERED_RECORDS_PER_PARTITION_CONFIG, topologyOverrides)) { - maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); - log.info("Topology {} is overriding {} to {}", topologyName, BUFFERED_RECORDS_PER_PARTITION_CONFIG, maxBufferedSize); - } else { - maxBufferedSize = globalAppConfigs.getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); - } - if (isTopologyOverride(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, topologyOverrides)) { timestampExtractorSupplier = () -> getConfiguredInstance(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, getClass(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG)); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java index 74d81bd7f3a37..4b4ff571673f6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.namedtopology.TopologyConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockClientSupplier; import org.easymock.EasyMockRunner; @@ -80,8 +81,6 @@ public class ActiveTaskCreatorTest { private ActiveTaskCreator activeTaskCreator; - - // non-EOS test // functional test @@ -477,6 +476,7 @@ private void createTasks() { final SourceNode sourceNode = mock(SourceNode.class); reset(builder, stateDirectory); + expect(builder.topologyConfigs()).andStubReturn(new TopologyConfig(new StreamsConfig(properties))); expect(builder.buildSubtopology(0)).andReturn(topology).anyTimes(); expect(stateDirectory.getOrCreateDirectoryForTask(task00)).andReturn(mock(File.class)); expect(stateDirectory.checkpointFileFor(task00)).andReturn(mock(File.class)); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 6eb289647f6cd..8d4cb46b8864d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -942,6 +942,7 @@ public void shouldSetTopologyConfigOnRewriteTopology() { @Test public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { final Properties topologyOverrides = new Properties(); + topologyOverrides.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 12345L); topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); topologyOverrides.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L); topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); @@ -949,8 +950,14 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); - final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); + final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( + new TopologyConfig( + "my-topology", + config, + topologyOverrides) + ); + assertThat(topologyBuilder.topologyConfigs().cacheSize, is(12345L)); assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxTaskIdleMs, equalTo(500L)); assertThat(topologyBuilder.topologyConfigs().getTaskConfig().taskTimeoutMs, equalTo(1000L)); assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxBufferedSize, equalTo(15)); @@ -960,21 +967,27 @@ public void shouldOverrideGlobalStreamsConfigWhenGivenNamedTopologyProps() { @Test public void shouldNotOverrideGlobalStreamsConfigWhenGivenUnnamedTopologyProps() { - final Properties topologyOverrides = new Properties(); - topologyOverrides.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); - topologyOverrides.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L); - topologyOverrides.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); - topologyOverrides.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); - topologyOverrides.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); - - final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); - final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); - - assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxTaskIdleMs, not(500L)); - assertThat(topologyBuilder.topologyConfigs().getTaskConfig().taskTimeoutMs, not(1000L)); - assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxBufferedSize, not(15)); - assertThat(topologyBuilder.topologyConfigs().getTaskConfig().timestampExtractor.getClass(), not(MockTimestampExtractor.class)); - assertThat(topologyBuilder.topologyConfigs().getTaskConfig().deserializationExceptionHandler.getClass(), not(LogAndContinueExceptionHandler.class)); + final Properties streamsProps = StreamsTestUtils.getStreamsConfig(); + streamsProps.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 12345L); + streamsProps.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L); + streamsProps.put(StreamsConfig.TASK_TIMEOUT_MS_CONFIG, 1000L); + streamsProps.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 15); + streamsProps.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); + streamsProps.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class); + + final StreamsConfig config = new StreamsConfig(streamsProps); + final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( + new TopologyConfig( + "my-topology", + config, + new Properties()) + ); + assertThat(topologyBuilder.topologyConfigs().cacheSize, is(12345L)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxTaskIdleMs, is(500L)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().taskTimeoutMs, is(1000L)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().maxBufferedSize, is(15)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().timestampExtractor.getClass(), is(MockTimestampExtractor.class)); + assertThat(topologyBuilder.topologyConfigs().getTaskConfig().deserializationExceptionHandler.getClass(), is(LogAndContinueExceptionHandler.class)); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java index 5253a1d708bec..761fa07101931 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/NamedTopologyTest.java @@ -190,7 +190,7 @@ public void shouldDescribeWithSingleNamedTopology() { assertThat( streams.getFullTopologyDescription(), equalTo( - "Topology - topology-1:\n" + "Topology: topology-1:\n" + " Sub-topology: 0\n" + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-1])\n" + " --> none\n" @@ -222,7 +222,7 @@ public void shouldDescribeWithMultipleNamedTopologies() { assertThat( streams.getFullTopologyDescription(), equalTo( - "Topology - topology-1:\n" + "Topology: topology-1:\n" + " Sub-topology: 0\n" + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-1])\n" + " --> none\n" @@ -236,7 +236,7 @@ public void shouldDescribeWithMultipleNamedTopologies() { + " Sink: KSTREAM-SINK-0000000003 (topic: output-1)\n" + " <-- KSTREAM-FILTER-0000000002\n" + "\n" - + "Topology - topology-2:\n" + + "Topology: topology-2:\n" + " Sub-topology: 0\n" + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-2])\n" + " --> none\n" @@ -250,7 +250,7 @@ public void shouldDescribeWithMultipleNamedTopologies() { + " Sink: KSTREAM-SINK-0000000003 (topic: output-2)\n" + " <-- KSTREAM-FILTER-0000000002\n" + "\n" - + "Topology - topology-3:\n" + + "Topology: topology-3:\n" + " Sub-topology: 0\n" + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-3])\n" + " --> none\n" diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 9c366fbeeb9d1..58f870da64a24 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -583,7 +583,7 @@ public void shouldInitTaskTimeoutAndEventuallyThrow() { } @Test - public void shouldCLearTaskTimeout() { + public void shouldClearTaskTimeout() { EasyMock.replay(stateManager); task = createStandbyTask(); @@ -613,7 +613,7 @@ private StandbyTask createStandbyTask() { taskId, Collections.singleton(partition), topology, - new TopologyConfig(null, config, new Properties()).getTaskConfig(), + new TopologyConfig(config).getTaskConfig(), streamsMetrics, stateManager, stateDirectory, From b95ce3152bc33b536c45c004afd68a67b2d168d5 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Sat, 9 Oct 2021 03:40:43 -0500 Subject: [PATCH 15/18] checlkstyle --- .../kafka/streams/processor/internals/StandbyTaskTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 58f870da64a24..00daaa6e9d82b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -57,7 +57,6 @@ import java.time.Duration; import java.util.Collections; import java.util.List; -import java.util.Properties; import java.util.stream.Collectors; import static java.util.Arrays.asList; From 2bcff16dfdc00efb7264c2f1e90393e97b6b1957 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Fri, 15 Oct 2021 21:34:53 -0700 Subject: [PATCH 16/18] fix tests --- .../streams/processor/internals/InternalTopologyBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index fd982f71078c5..99d3bdf59b0c0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -1986,7 +1986,7 @@ public String toString() { if (namedTopology == null) { sb.append("Topologies:\n "); } else { - sb.append("Topology - ").append(namedTopology).append(":\n "); + sb.append("Topology: ").append(namedTopology).append(":\n "); } final TopologyDescription.Subtopology[] sortedSubtopologies = subtopologies.descendingSet().toArray(new TopologyDescription.Subtopology[0]); From b86e44cdacf32547274521455a2f50b3ee0a9bc2 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Mon, 1 Nov 2021 18:12:48 -0700 Subject: [PATCH 17/18] skip initial subscription if starting up empty --- .../streams/processor/internals/StreamThread.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 88814b0c93475..d106257e1011f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -714,10 +714,12 @@ private void handleTaskMigrated(final TaskMigratedException e) { } private void subscribeConsumer() { - if (topologyMetadata.usesPatternSubscription()) { - mainConsumer.subscribe(topologyMetadata.sourceTopicPattern(), rebalanceListener); - } else { - mainConsumer.subscribe(topologyMetadata.sourceTopicCollection(), rebalanceListener); + if (!topologyMetadata.isEmpty()) { + if (topologyMetadata.usesPatternSubscription()) { + mainConsumer.subscribe(topologyMetadata.sourceTopicPattern(), rebalanceListener); + } else { + mainConsumer.subscribe(topologyMetadata.sourceTopicCollection(), rebalanceListener); + } } } From ffec3d780c60ca978d997ca0e33acad9f2407cdd Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Tue, 9 Nov 2021 18:09:50 -0800 Subject: [PATCH 18/18] fix out of date comment, tighten up topology update handling --- .../processor/internals/StreamThread.java | 20 +++++++++++-------- .../processor/internals/TaskManager.java | 2 +- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index d106257e1011f..1390cdf2ed667 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -596,6 +596,14 @@ boolean runLoop() { // until the rebalance is completed before we close and commit the tasks while (isRunning() || taskManager.isRebalanceInProgress()) { try { + checkForTopologyUpdates(); + // If we received the shutdown signal while waiting for a topology to be added, we can + // stop polling regardless of the rebalance status since we know there are no tasks left + if (!isRunning() && topologyMetadata.isEmpty()) { + log.info("Shutting down thread with empty topology."); + break; + } + maybeSendShutdown(); final long size = cacheResizeSize.getAndSet(-1L); if (size != -1L) { @@ -714,12 +722,10 @@ private void handleTaskMigrated(final TaskMigratedException e) { } private void subscribeConsumer() { - if (!topologyMetadata.isEmpty()) { - if (topologyMetadata.usesPatternSubscription()) { - mainConsumer.subscribe(topologyMetadata.sourceTopicPattern(), rebalanceListener); - } else { - mainConsumer.subscribe(topologyMetadata.sourceTopicCollection(), rebalanceListener); - } + if (topologyMetadata.usesPatternSubscription()) { + mainConsumer.subscribe(topologyMetadata.sourceTopicPattern(), rebalanceListener); + } else { + mainConsumer.subscribe(topologyMetadata.sourceTopicCollection(), rebalanceListener); } } @@ -916,8 +922,6 @@ private void checkForTopologyUpdates() { } private long pollPhase() { - checkForTopologyUpdates(); - final ConsumerRecords records; log.debug("Invoking poll on main Consumer"); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index c5ba0c183a2f7..d861ef16db8e2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -1250,7 +1250,7 @@ public void updateTaskEndMetadata(final TopicPartition topicPartition, final Lon /** * Handle any added or removed NamedTopologies. Check if any uncreated assigned tasks belong to a newly - * added NamedTopology and create them if so, then freeze any tasks whose named topology no longer exists + * added NamedTopology and create them if so, then close any tasks whose named topology no longer exists */ void handleTopologyUpdates() { tasks.maybeCreateTasksFromNewTopologies();