-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-12648: fix bug where thread is re-added to TopologyMetadata when shutting down #11857
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1129,13 +1129,25 @@ public void updateTaskEndMetadata(final TopicPartition topicPartition, final Lon | |
| * added NamedTopology and create them if so, then close any tasks whose named topology no longer exists | ||
| */ | ||
| void handleTopologyUpdates() { | ||
| tasks.maybeCreateTasksFromNewTopologies(); | ||
| final Set<String> currentNamedTopologies = topologyMetadata.updateThreadTopologyVersion(Thread.currentThread().getName()); | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This isn't the main fix, but we were playing a little fast and loose with the topology version we were reporting having ack'ed -- tightened this up by first atomically updating the topology version and saving the set of current named topologies, then doing the actual update handling, and then checking the listeners and completing any finished add/remove topology requests |
||
|
|
||
| tasks.maybeCreateTasksFromNewTopologies(currentNamedTopologies); | ||
| maybeCloseTasksFromRemovedTopologies(currentNamedTopologies); | ||
|
|
||
| if (topologyMetadata.isEmpty()) { | ||
| log.info("Proactively unsubscribing from all topics due to empty topology"); | ||
| mainConsumer.unsubscribe(); | ||
| } | ||
|
|
||
| topologyMetadata.maybeNotifyTopologyVersionListeners(); | ||
| } | ||
|
|
||
| void maybeCloseTasksFromRemovedTopologies(final Set<String> currentNamedTopologies) { | ||
| try { | ||
| final Set<Task> activeTasksToRemove = new HashSet<>(); | ||
| final Set<Task> standbyTasksToRemove = new HashSet<>(); | ||
| for (final Task task : tasks.allTasks()) { | ||
| if (!topologyMetadata.namedTopologiesView().contains(task.id().topologyName())) { | ||
| if (!currentNamedTopologies.contains(task.id().topologyName())) { | ||
| if (task.isActive()) { | ||
| activeTasksToRemove.add(task); | ||
| } else { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -84,14 +84,14 @@ public static class TopologyVersion { | |
| public AtomicLong topologyVersion = new AtomicLong(0L); // the local topology version | ||
| public ReentrantLock topologyLock = new ReentrantLock(); | ||
| public Condition topologyCV = topologyLock.newCondition(); | ||
| public List<TopologyVersionWaiters> activeTopologyWaiters = new LinkedList<>(); | ||
| public List<TopologyVersionListener> activeTopologyUpdateListeners = new LinkedList<>(); | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just renamed from
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Also another quick question regarding why we need to keep
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good find, yeah I believe it no longer needs to be an AtomicLong, I'll change back to
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh right actually no, we. do still need it to be an AtomicLong as we check it in the StreamThread main loop when looking for topology updates. And obviously we don't want to have to grab the full lock for that |
||
| } | ||
|
|
||
| public static class TopologyVersionWaiters { | ||
| public static class TopologyVersionListener { | ||
| final long topologyVersion; // the (minimum) version to wait for these threads to cross | ||
| final KafkaFutureImpl<Void> future; // the future waiting on all threads to be updated | ||
|
|
||
| public TopologyVersionWaiters(final long topologyVersion, final KafkaFutureImpl<Void> future) { | ||
| public TopologyVersionListener(final long topologyVersion, final KafkaFutureImpl<Void> future) { | ||
| this.topologyVersion = topologyVersion; | ||
| this.future = future; | ||
| } | ||
|
|
@@ -161,35 +161,47 @@ public void registerThread(final String threadName) { | |
|
|
||
| public void unregisterThread(final String threadName) { | ||
| threadVersions.remove(threadName); | ||
| maybeNotifyTopologyVersionWaitersAndUpdateThreadsTopologyVersion(threadName); | ||
| maybeNotifyTopologyVersionListeners(); | ||
| } | ||
|
|
||
| public TaskExecutionMetadata taskExecutionMetadata() { | ||
| return taskExecutionMetadata; | ||
| } | ||
|
|
||
| public void maybeNotifyTopologyVersionWaitersAndUpdateThreadsTopologyVersion(final String threadName) { | ||
| public Set<String> updateThreadTopologyVersion(final String threadName) { | ||
| try { | ||
| lock(); | ||
| final Iterator<TopologyVersionWaiters> iterator = version.activeTopologyWaiters.listIterator(); | ||
| TopologyVersionWaiters topologyVersionWaiters; | ||
| version.topologyLock.lock(); | ||
| threadVersions.put(threadName, topologyVersion()); | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @wcarlson5 / @guozhangwang / @vvcephei This is the main fix -- we need to split out the version update where we add the current thread with the latest topology version to this The other function of the method was to check whether we could complete any of the queued listeners, which is why we were invoking this when shutting down a thread. Splitting this out into a separate method avoids ghost threads being left behind in the |
||
| return namedTopologiesView(); | ||
| } finally { | ||
| version.topologyLock.unlock(); | ||
| } | ||
| } | ||
|
|
||
| public void maybeNotifyTopologyVersionListeners() { | ||
| try { | ||
| lock(); | ||
| final long minThreadVersion = getMinimumThreadVersion(); | ||
| final Iterator<TopologyVersionListener> iterator = version.activeTopologyUpdateListeners.listIterator(); | ||
| TopologyVersionListener topologyVersionListener; | ||
ableegoldman marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| while (iterator.hasNext()) { | ||
| topologyVersionWaiters = iterator.next(); | ||
| final long topologyVersionWaitersVersion = topologyVersionWaiters.topologyVersion; | ||
| if (topologyVersionWaitersVersion <= threadVersions.get(threadName)) { | ||
| if (threadVersions.values().stream().allMatch(t -> t >= topologyVersionWaitersVersion)) { | ||
| topologyVersionWaiters.future.complete(null); | ||
| iterator.remove(); | ||
| log.info("All threads are now on topology version {}", topologyVersionWaiters.topologyVersion); | ||
| } | ||
| topologyVersionListener = iterator.next(); | ||
| final long topologyVersionWaitersVersion = topologyVersionListener.topologyVersion; | ||
| if (minThreadVersion >= topologyVersionWaitersVersion) { | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I also refactored this slightly to optimize/clean up this method. It's less about the optimization as we should generally not have too many threads per KafkaStreams runtime, but I found it much easier to follow the logic by computing the minimum version across all threads and then completing all futures listening for the topology to be updated up to that version
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we also want to remove the listeners for threads that were removed as well right?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The listeners are for the caller threads not stream threads right? I thought since the thread is removed, it would not be counted in the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No, the listeners are for stream threads. They get added in the task manager. Once all threads are at the version the future blocking the calling thread is completed.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm... just to make sure we are talking about Anyways, my understanding is that when a thread is removed, the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah @guozhangwang yeah the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah I think this was already resolved but just to clarify for anyone else reading this/ourselves in the future, yes, the listeners are for the callers of add/removeNamedTopology 👍 |
||
| topologyVersionListener.future.complete(null); | ||
| iterator.remove(); | ||
| log.info("All threads are now on topology version {}", topologyVersionListener.topologyVersion); | ||
| } | ||
| } | ||
| } finally { | ||
| unlock(); | ||
| } | ||
| } | ||
|
|
||
| private long getMinimumThreadVersion() { | ||
| return threadVersions.values().stream().min(Long::compare).get(); | ||
| } | ||
|
|
||
| public void wakeupThreads() { | ||
| try { | ||
| lock(); | ||
|
|
@@ -224,9 +236,9 @@ public void registerAndBuildNewTopology(final KafkaFutureImpl<Void> future, fina | |
| try { | ||
| lock(); | ||
| buildAndVerifyTopology(newTopologyBuilder); | ||
| log.info("New NamedTopology passed validation and will be added {}, old topology version is {}", newTopologyBuilder.topologyName(), version.topologyVersion.get()); | ||
| log.info("New NamedTopology {} passed validation and will be added, old topology version is {}", newTopologyBuilder.topologyName(), version.topologyVersion.get()); | ||
| version.topologyVersion.incrementAndGet(); | ||
| version.activeTopologyWaiters.add(new TopologyVersionWaiters(topologyVersion(), future)); | ||
| version.activeTopologyUpdateListeners.add(new TopologyVersionListener(topologyVersion(), future)); | ||
| builders.put(newTopologyBuilder.topologyName(), newTopologyBuilder); | ||
| wakeupThreads(); | ||
| log.info("Added NamedTopology {} and updated topology version to {}", newTopologyBuilder.topologyName(), version.topologyVersion.get()); | ||
|
|
@@ -247,7 +259,7 @@ public KafkaFuture<Void> unregisterTopology(final KafkaFutureImpl<Void> removeTo | |
| lock(); | ||
| log.info("Beginning removal of NamedTopology {}, old topology version is {}", topologyName, version.topologyVersion.get()); | ||
| version.topologyVersion.incrementAndGet(); | ||
| version.activeTopologyWaiters.add(new TopologyVersionWaiters(topologyVersion(), removeTopologyFuture)); | ||
| version.activeTopologyUpdateListeners.add(new TopologyVersionListener(topologyVersion(), removeTopologyFuture)); | ||
| final InternalTopologyBuilder removedBuilder = builders.remove(topologyName); | ||
| removedBuilder.fullSourceTopicNames().forEach(allInputTopics::remove); | ||
| removedBuilder.allSourcePatternStrings().forEach(allInputTopics::remove); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
All of this has been moved into
taskManager#handleTopologyUpdates