Skip to content
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

KAFKA-3443 [Kafka Stream] support for adding sources to KafkaStreams via Pattern #1477

Conversation

bbejeck
Copy link
Contributor

@bbejeck bbejeck commented Jun 7, 2016

This PR is the follow on to the closed PR #1410.

@bbejeck
Copy link
Contributor Author

bbejeck commented Jun 7, 2016

@guozhangwang
I needed to clean up the last PR ( #1410 ), seemed best to close and re-open a new one. I believe this PR covers all issued raised via comments:

  1. Support for multiple sources via Patterns and/or concrete topic lists.
  2. Includes integration tests.
  3. Does not allow for "overlapping" regex patterns.

Two remaining issues per your suggestions for creating integration tests:

  1. Deleting a topic will causes all stream tasks to be removed, and a rebalance is not triggered. EDIT: clarification - a rebalance is triggered, but all stream tasks are removed, no further messages processed from topics still still available.
  2. Adding a topic does not seem to trigger a rebalance.

Should these be included in this PR or addressed as a separate issue?

@bbejeck
Copy link
Contributor Author

bbejeck commented Jun 7, 2016

Tests run fine locally. Looks like the build stalled for some reason:

Build timed out (after 120 minutes). Marking the build as failed.
Build was aborted

@guozhangwang
Copy link
Contributor

@ijuma @hachikuji Do you know if this test could be transiently stalled?

kafka.api.QuotasTest > testProducerConsumerOverrideUnthrottled STARTED
Build timed out (after 120 minutes). Marking the build as failed.

@guozhangwang
Copy link
Contributor

@bbejeck Thanks! I will review the patch soon.

@bbejeck
Copy link
Contributor Author

bbejeck commented Jun 7, 2016

Thoughts on the topic removal - store TopicPartitions when assigned. On partitionsRevoked compare what's being revoked vs what was assigned, if any partitions left, then don't remove stream tasks?

@guozhangwang
Copy link
Contributor

@bbejeck

  1. Adding a topic does not seem to trigger a rebalance: hmm, this is not expected and may reveal a bug in the current implementation. My understanding is that, when a new topic is created, the underlying consumer will detect these new topics from refreshed metadata and send a new JoinGroup request to the coordinator, causing rebalance.
  2. Upon a rebalance is triggered, upon onPartitionRevoked we will close all the tasks; and upon onPartitionAssigned we will create new tasks. So for both topic creation / deletion, the StreamPartitionAssignor should be triggered, which in turn triggers PartitionGrouper. I will look at your comments in KAFKA-3443 [Kafka Streams] added support for subscribing to topics via regex  #1410 trying to find out what was the issue.

@guozhangwang
Copy link
Contributor

I cannot find anything from #1410's logs you posted.

As I mentioned both creating / deleting a topic should trigger a rebalance, which triggers the StreamPartitionAssignor, and although all tasks are removed upon partition revoked, they should be re-created / resumed upon partition assigned. So I guess there is something wrong in Kafka Streams code. If you could post some more logs from your local integration test run, I'd love to investigate further.


public String[] getTopics(Collection<String> subscribedTopics) {
List<String> matchedTopics = new ArrayList<>();
for (String update : subscribedTopics) {
Copy link
Contributor

@guozhangwang guozhangwang Jun 9, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we do the following instead?

for (String update : subscribedTopics) {
                // already matched from previous rebalances
                if (this.pattern == topicToPatterns.get(update)) {
                    matchedTopics.add(update);
                } else if (topicToPatterns.get(update) != null) {
                    throw new TopologyBuilderException("Topic " + update + " already matched with another regex pattern");
                } else if (this.pattern.matcher(update).matches()) {
                    topicToPatterns.put(update, this.pattern);
                    matchedTopics.add(update);
                }
            }

@guozhangwang
Copy link
Contributor

I looked through the diff but cannot fine obvious bugs that could cause your integration tests execute unexpectedly. One thing we would suggest is to print out the topicGroups after the this.topicGroups = streamThread.builder.topicGroups(streamThread.applicationId); call, and check if the topics reflect the changes to the subscribed topics.

@guozhangwang
Copy link
Contributor

@bbejeck I think I get the root cause of integration test issues (kudos to @hachikuji): as mentioned the consumer depend on metadata refresh to detect newly added topics / deleted topics, and by default this ConsumerConfig.METADATA_MAX_AGE_CONFIG is set to 5 minutes, so newly added topics will not be detected in time; as for topic deletion, the consumer will get a TopicsUnknowException immediately and hence re-join the group, causing the rebalance to happen much sooner.

Now about why topic deletion does not cause newly created tasks in onPartitionAssigned, that is still mystery to me. I think maybe it is just that the resuming of the tasks takes longer time and you did not wait enough time?

@bbejeck
Copy link
Contributor Author

bbejeck commented Jun 9, 2016

@bbejeck I think I get the root cause of integration test issues.

I observed this behavior as well when running a KStreams program locally. After deleting a topic, I let the program continue to run for some time (30+ minutes) and the remaining topics were never re-assigned. As for adding a new topic I did the same thing, add a topic and let the program run for a while and a new assignment never happened. I can see the new topic in the KafkaServer logs, but the changes don't propagate over to the Streams program. I'm thinking this could have something to do with my environment and there is a silent failure somewhere. I'll add some detailed logging and see if I can get some more information.

EDIT: Fixed the solution was to update the ConsumerConfig.METADATA_MAX_AGE_CONFIG to a smaller value. Additionally, I was running the code from inside Intellij, the console log shows no action after an update, but tailing the log file revealed the appropriate behavior. Sorry for the hassle.

…new topic after streams started, fixed potential NullPointerException that can occur if a matching topic is removed while a KStreams program is running.
@bbejeck
Copy link
Contributor Author

bbejeck commented Jun 10, 2016

@guozhangwang
Changes made per comments, added an integration test demonstrating picking up a matching topic that is added after the streams instance has started. Also updated the Metadata object, if a matching topic is deleted while KStreams is running a NullPointerException is thrown during the rebalance.

I need some guidance for a counter test - removing a matching topic once a KStreams program has started. Very tough to get to the subscription updates without resorting to some messy reflection work.

EDIT: To be more clear, I'm looking for another way to prove the regex subscriptions are correct after a rebalance, the case of adding a topic was easier, just confirm a message recieved. Would this be a test for the StreamsPartitionAssignor?

@@ -292,7 +292,10 @@ private Cluster getClusterForCurrentTopics(Cluster cluster) {
unauthorizedTopics.retainAll(this.topics.keySet());

for (String topic : this.topics.keySet()) {
partitionInfos.addAll(cluster.partitionsForTopic(topic));
List<PartitionInfo> partitionInfoList = cluster.partitionsForTopic(topic);
if (partitionInfoList != null) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This was necessary as deleting a matching topic causes a NullPointerException when KStreams is running.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice catch!

@guozhangwang
Copy link
Contributor

@bbejeck I think one way to validate that added / removed topics are handled is to check the removed / created tasks instead of checking the processed messages; i.e. if a new topic is added, the new tasks corresponding to this new topic should be created in the rebalance callback (i.e. addStreamTasks), and similarly for deleted topics, we should see that first all tasks are removed, and they are re-created in the rebalance again but without the deleted topic partitions assigned any more.

We can override the removeStreamTasks and createStreamTask in StreamThread to do this validation. And let me know if you still cannot observe the rebalance happening after topics deleted.

…ics added and/or deleted when a streams instance is running.
@bbejeck
Copy link
Contributor Author

bbejeck commented Jun 13, 2016

@guozhangwang
I think I've addressed all comments so far and I added two new integration tests per suggestions for validating topics being added/removed.

Rebalancing when deleting works fine, just needed to set delete.topic.enable to true.

pattern1Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC);

// Remove any state from previous test runs
IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of purge for previous runs, could you use @After to call this function at the end of each test case?

@guozhangwang
Copy link
Contributor

Thanks @bbejeck ! LGTM overall, just left some minor comments about unit tests.

…ed integration test, removed Thread.sleep calls
@bbejeck
Copy link
Contributor Author

bbejeck commented Jun 15, 2016

@guozhangwang
Updated the unit tests per suggestions, created JIRA ticket KAFKA-3842 for moving helper functions into test utils.

EDIT: I realized that my replacement for Thread.sleep needs a timeout added, I'll update that.


void waitUntilTasksUpdated() {
while (!streamTaskUpdated) {
//empty loop just waiting for update
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Needs a check for total time elapsed here, default to 30 seconds then return.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good.

@bbejeck
Copy link
Contributor Author

bbejeck commented Jun 16, 2016

@guozhangwang changes made per suggestions.

@@ -249,7 +249,7 @@ public void testShouldReadFromRegexAndNamedTopics() throws Exception {
assertThat(actualValues, equalTo(expectedReceivedValues));
}

//TODO should be updated to expected = TopologyBuilderException after KAFKA-3708
// todo should be updated to expected = TopologyBuilderException after KAFKA-3708
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm actually more considering about the comment in line 264-267, and probably line 276 for de-capitalizing. As for TODO it is usually treated as a marker in most IDEs for easy finding and we'd keep it that way. For example:

https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L972

@guozhangwang
Copy link
Contributor

One minor comment regarding comments, otherwise LGTM, running unit / integration test locally now.

@asfgit asfgit closed this in fb42558 Jun 16, 2016
@guozhangwang
Copy link
Contributor

LGTM and merged to trunk. Thanks @bbejeck for this patch! It's a big contribution which drags long :)

efeg added a commit to efeg/kafka that referenced this pull request May 29, 2024
@bbejeck bbejeck deleted the KAFKA-3443_streams_support_for_regex_sources branch July 10, 2024 12:55
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants