You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/02/19 16:54:15 UTC

[GitHub] [kafka] cadonna opened a new pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

cadonna opened a new pull request #10163:
URL: https://github.com/apache/kafka/pull/10163


   To implement the explicit user initialization of Kafka Streams as
   described in KIP-698, we first need to extract the code for the
   setup of the changelog topics from the Streams partition assignor
   so that it can also be called outside of a rebalance.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r585546908



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -741,41 +666,35 @@ private boolean populateClientStatesMap(final Map<UUID, ClientState> clientState
     }
 
     /**
-     * @param changelogsByStatefulTask map from stateful task to its set of changelog topic partitions
      * @param endOffsets the listOffsets result from the adminClient
      * @param sourceChangelogEndOffsets the end (committed) offsets of optimized source changelogs
-     * @param newlyCreatedChangelogPartitions any changelogs that were just created duringthis assignment
+     * @param changelogTopics object that manages changelog topics
      *
      * @return Map from stateful task to its total end offset summed across all changelog partitions
      */
-    private Map<TaskId, Long> computeEndOffsetSumsByTask(final Map<TaskId, Set<TopicPartition>> changelogsByStatefulTask,
-                                                         final Map<TopicPartition, ListOffsetsResultInfo> endOffsets,
+    private Map<TaskId, Long> computeEndOffsetSumsByTask(final Map<TopicPartition, ListOffsetsResultInfo> endOffsets,
                                                          final Map<TopicPartition, Long> sourceChangelogEndOffsets,
-                                                         final Collection<TopicPartition> newlyCreatedChangelogPartitions) {
+                                                         final ChangelogTopics changelogTopics) {
+
         final Map<TaskId, Long> taskEndOffsetSums = new HashMap<>();
-        for (final Map.Entry<TaskId, Set<TopicPartition>> taskEntry : changelogsByStatefulTask.entrySet()) {
-            final TaskId task = taskEntry.getKey();
-            final Set<TopicPartition> changelogs = taskEntry.getValue();
-
-            taskEndOffsetSums.put(task, 0L);
-            for (final TopicPartition changelog : changelogs) {
-                final long changelogEndOffset;
-                if (newlyCreatedChangelogPartitions.contains(changelog)) {
-                    changelogEndOffset = 0L;
-                } else if (sourceChangelogEndOffsets.containsKey(changelog)) {
-                    changelogEndOffset = sourceChangelogEndOffsets.get(changelog);
-                } else if (endOffsets.containsKey(changelog)) {
-                    changelogEndOffset = endOffsets.get(changelog).offset();
+        for (final TaskId taskId : changelogTopics.taskIds()) {

Review comment:
       Will do!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r583907570



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();
+    private final Map<TaskId, Set<TopicPartition>> preExistingChangelogPartitionsForTask = new HashMap<>();
+    private final Set<TopicPartition> preExistingNonSourceTopicBasedChangelogPartitions = new HashSet<>();
+    private final Set<String> sourceTopicBasedChangelogTopics = new HashSet<>();
+    private final Set<TopicPartition> sourceTopicBasedChangelogTopicPartitions = new HashSet<>();
+    private final Logger log;
+
+    public ChangelogTopics(final InternalTopicManager internalTopicManager,
+                           final Map<Integer, TopicsInfo> topicGroups,
+                           final Map<Integer, Set<TaskId>> tasksForTopicGroup,
+                           final String logPrefix) {
+        this.internalTopicManager = internalTopicManager;
+        this.topicGroups = topicGroups;
+        this.tasksForTopicGroup = tasksForTopicGroup;
+        final LogContext logContext = new LogContext(logPrefix);
+        log = logContext.logger(getClass());
+    }
+
+    public void setup() {

Review comment:
       Is there a specific reason we need to make an explicit call to `setup()` rather than just doing this in the constructor? I'm always worried we'll end up forgetting to call `setup` again after some refactoring and someone will waste a day debugging their code because they tried to use a `Changelogs` object before/without first calling `setup()`

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();

Review comment:
        nit: clarify that this contains only stateful tasks (eg `changelogPartitionsForStatefulTask` -- for this and any other data structure which only contains stateful partitions/tasks

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();
+    private final Map<TaskId, Set<TopicPartition>> preExistingChangelogPartitionsForTask = new HashMap<>();
+    private final Set<TopicPartition> preExistingNonSourceTopicBasedChangelogPartitions = new HashSet<>();
+    private final Set<String> sourceTopicBasedChangelogTopics = new HashSet<>();
+    private final Set<TopicPartition> sourceTopicBasedChangelogTopicPartitions = new HashSet<>();
+    private final Logger log;
+
+    public ChangelogTopics(final InternalTopicManager internalTopicManager,
+                           final Map<Integer, TopicsInfo> topicGroups,
+                           final Map<Integer, Set<TaskId>> tasksForTopicGroup,
+                           final String logPrefix) {
+        this.internalTopicManager = internalTopicManager;
+        this.topicGroups = topicGroups;
+        this.tasksForTopicGroup = tasksForTopicGroup;
+        final LogContext logContext = new LogContext(logPrefix);
+        log = logContext.logger(getClass());
+    }
+
+    public void setup() {
+        // add tasks to state change log topic subscribers
+        final Map<String, InternalTopicConfig> changelogTopicMetadata = new HashMap<>();
+        for (final Map.Entry<Integer, TopicsInfo> entry : topicGroups.entrySet()) {
+            final int topicGroupId = entry.getKey();
+            final TopicsInfo topicsInfo = entry.getValue();
+
+            final Set<TaskId> topicGroupTasks = tasksForTopicGroup.get(topicGroupId);
+            if (topicGroupTasks == null) {
+                log.debug("No tasks found for topic group {}", topicGroupId);
+                continue;
+            } else if (topicsInfo.stateChangelogTopics.isEmpty()) {
+                continue;
+            }
+
+            for (final TaskId task : topicGroupTasks) {
+                final Set<TopicPartition> changelogTopicPartitions = topicsInfo.stateChangelogTopics
+                    .keySet()
+                    .stream()
+                    .map(topic -> new TopicPartition(topic, task.partition))
+                    .collect(Collectors.toSet());
+                changelogPartitionsForTask.put(task, changelogTopicPartitions);
+            }
+
+            for (final InternalTopicConfig topicConfig : topicsInfo.nonSourceChangelogTopics()) {
+                // the expected number of partitions is the max value of TaskId.partition + 1
+                int numPartitions = UNKNOWN;
+                for (final TaskId task : topicGroupTasks) {
+                    if (numPartitions < task.partition + 1) {
+                        numPartitions = task.partition + 1;
+                    }
+                }
+                topicConfig.setNumberOfPartitions(numPartitions);
+                changelogTopicMetadata.put(topicConfig.name(), topicConfig);
+            }
+            sourceTopicBasedChangelogTopics.addAll(topicsInfo.sourceTopicChangelogs());
+        }
+
+        final Set<String> newlyCreatedChangelogTopics = internalTopicManager.makeReady(changelogTopicMetadata);
+        log.debug("Created state changelog topics {} from the parsed topology.", changelogTopicMetadata.values());
+
+        for (final Map.Entry<TaskId, Set<TopicPartition>> entry : changelogPartitionsForTask.entrySet()) {
+            final TaskId taskId = entry.getKey();
+            final Set<TopicPartition> topicPartitions = entry.getValue();
+            for (final TopicPartition topicPartition : topicPartitions) {
+                if (!newlyCreatedChangelogTopics.contains(topicPartition.topic())) {
+                    preExistingChangelogPartitionsForTask.computeIfAbsent(taskId, task -> new HashSet<>()).add(topicPartition);
+                    if (!sourceTopicBasedChangelogTopics.contains(topicPartition.topic())) {
+                        preExistingNonSourceTopicBasedChangelogPartitions.add(topicPartition);
+                    } else {
+                        sourceTopicBasedChangelogTopicPartitions.add(topicPartition);

Review comment:
       So technically this only contains pre-existing source topic based changelogs -- can we specify so in the set name? It's already pretty long but maybe we can drop the `Topic` part to help somewhat

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -741,41 +666,35 @@ private boolean populateClientStatesMap(final Map<UUID, ClientState> clientState
     }
 
     /**
-     * @param changelogsByStatefulTask map from stateful task to its set of changelog topic partitions
      * @param endOffsets the listOffsets result from the adminClient
      * @param sourceChangelogEndOffsets the end (committed) offsets of optimized source changelogs
-     * @param newlyCreatedChangelogPartitions any changelogs that were just created duringthis assignment
+     * @param changelogTopics object that manages changelog topics
      *
      * @return Map from stateful task to its total end offset summed across all changelog partitions
      */
-    private Map<TaskId, Long> computeEndOffsetSumsByTask(final Map<TaskId, Set<TopicPartition>> changelogsByStatefulTask,
-                                                         final Map<TopicPartition, ListOffsetsResultInfo> endOffsets,
+    private Map<TaskId, Long> computeEndOffsetSumsByTask(final Map<TopicPartition, ListOffsetsResultInfo> endOffsets,
                                                          final Map<TopicPartition, Long> sourceChangelogEndOffsets,
-                                                         final Collection<TopicPartition> newlyCreatedChangelogPartitions) {
+                                                         final ChangelogTopics changelogTopics) {
+
         final Map<TaskId, Long> taskEndOffsetSums = new HashMap<>();
-        for (final Map.Entry<TaskId, Set<TopicPartition>> taskEntry : changelogsByStatefulTask.entrySet()) {
-            final TaskId task = taskEntry.getKey();
-            final Set<TopicPartition> changelogs = taskEntry.getValue();
-
-            taskEndOffsetSums.put(task, 0L);
-            for (final TopicPartition changelog : changelogs) {
-                final long changelogEndOffset;
-                if (newlyCreatedChangelogPartitions.contains(changelog)) {
-                    changelogEndOffset = 0L;
-                } else if (sourceChangelogEndOffsets.containsKey(changelog)) {
-                    changelogEndOffset = sourceChangelogEndOffsets.get(changelog);
-                } else if (endOffsets.containsKey(changelog)) {
-                    changelogEndOffset = endOffsets.get(changelog).offset();
+        for (final TaskId taskId : changelogTopics.taskIds()) {

Review comment:
       Just to clarify, `changelogTopics.taskIds()` will only return stateful tasks -- again I think we should clarify that in the getter method name. I had to trace back through the code to be sure




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r585863461



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();
+    private final Map<TaskId, Set<TopicPartition>> preExistingChangelogPartitionsForTask = new HashMap<>();
+    private final Set<TopicPartition> preExistingNonSourceTopicBasedChangelogPartitions = new HashSet<>();
+    private final Set<String> sourceTopicBasedChangelogTopics = new HashSet<>();
+    private final Set<TopicPartition> sourceTopicBasedChangelogTopicPartitions = new HashSet<>();
+    private final Logger log;
+
+    public ChangelogTopics(final InternalTopicManager internalTopicManager,
+                           final Map<Integer, TopicsInfo> topicGroups,
+                           final Map<Integer, Set<TaskId>> tasksForTopicGroup,
+                           final String logPrefix) {
+        this.internalTopicManager = internalTopicManager;
+        this.topicGroups = topicGroups;
+        this.tasksForTopicGroup = tasksForTopicGroup;
+        final LogContext logContext = new LogContext(logPrefix);
+        log = logContext.logger(getClass());
+    }
+
+    public void setup() {

Review comment:
       Sounds good 🙂 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
cadonna commented on pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#issuecomment-782208101


   Call for review: @ableegoldman @guozhangwang @stevenpyzhang


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ableegoldman merged pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
ableegoldman merged pull request #10163:
URL: https://github.com/apache/kafka/pull/10163


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r585537160



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();

Review comment:
       I also thought about this and came to the conclusion that it would be clear that the task is stateful since it has at least one changelog. But I am fine with renaming it as you proposed.  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r585537160



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();

Review comment:
       I also thought about this and came to the conclusion that it would be clear that the task is stateful since it has at least one changelog. But I am fine with renaming it as you proposed. Better clear! 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r579336874



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -741,41 +666,35 @@ private boolean populateClientStatesMap(final Map<UUID, ClientState> clientState
     }
 
     /**
-     * @param changelogsByStatefulTask map from stateful task to its set of changelog topic partitions
      * @param endOffsets the listOffsets result from the adminClient
      * @param sourceChangelogEndOffsets the end (committed) offsets of optimized source changelogs
-     * @param newlyCreatedChangelogPartitions any changelogs that were just created duringthis assignment
+     * @param changelogTopics object that manages changelog topics
      *
      * @return Map from stateful task to its total end offset summed across all changelog partitions
      */
-    private Map<TaskId, Long> computeEndOffsetSumsByTask(final Map<TaskId, Set<TopicPartition>> changelogsByStatefulTask,
-                                                         final Map<TopicPartition, ListOffsetsResultInfo> endOffsets,
+    private Map<TaskId, Long> computeEndOffsetSumsByTask(final Map<TopicPartition, ListOffsetsResultInfo> endOffsets,
                                                          final Map<TopicPartition, Long> sourceChangelogEndOffsets,
-                                                         final Collection<TopicPartition> newlyCreatedChangelogPartitions) {

Review comment:
       I removed the newly created changelog partitions because they do actually not contribute to the task end offset sums.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#issuecomment-789174791


   Java11 failed with unrelated flaky `kafka.server.ListOffsetsRequestTest.testResponseIncludesLeaderEpoch`, Java15 failed with `Execution failed for task ':streams:test-utils:unitTest'` (this PR does not touch on the test-utils so this failure should be unrelated, however I ran them locally to verify)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r585544243



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();
+    private final Map<TaskId, Set<TopicPartition>> preExistingChangelogPartitionsForTask = new HashMap<>();
+    private final Set<TopicPartition> preExistingNonSourceTopicBasedChangelogPartitions = new HashSet<>();
+    private final Set<String> sourceTopicBasedChangelogTopics = new HashSet<>();
+    private final Set<TopicPartition> sourceTopicBasedChangelogTopicPartitions = new HashSet<>();
+    private final Logger log;
+
+    public ChangelogTopics(final InternalTopicManager internalTopicManager,
+                           final Map<Integer, TopicsInfo> topicGroups,
+                           final Map<Integer, Set<TaskId>> tasksForTopicGroup,
+                           final String logPrefix) {
+        this.internalTopicManager = internalTopicManager;
+        this.topicGroups = topicGroups;
+        this.tasksForTopicGroup = tasksForTopicGroup;
+        final LogContext logContext = new LogContext(logPrefix);
+        log = logContext.logger(getClass());
+    }
+
+    public void setup() {
+        // add tasks to state change log topic subscribers
+        final Map<String, InternalTopicConfig> changelogTopicMetadata = new HashMap<>();
+        for (final Map.Entry<Integer, TopicsInfo> entry : topicGroups.entrySet()) {
+            final int topicGroupId = entry.getKey();
+            final TopicsInfo topicsInfo = entry.getValue();
+
+            final Set<TaskId> topicGroupTasks = tasksForTopicGroup.get(topicGroupId);
+            if (topicGroupTasks == null) {
+                log.debug("No tasks found for topic group {}", topicGroupId);
+                continue;
+            } else if (topicsInfo.stateChangelogTopics.isEmpty()) {
+                continue;
+            }
+
+            for (final TaskId task : topicGroupTasks) {
+                final Set<TopicPartition> changelogTopicPartitions = topicsInfo.stateChangelogTopics
+                    .keySet()
+                    .stream()
+                    .map(topic -> new TopicPartition(topic, task.partition))
+                    .collect(Collectors.toSet());
+                changelogPartitionsForTask.put(task, changelogTopicPartitions);
+            }
+
+            for (final InternalTopicConfig topicConfig : topicsInfo.nonSourceChangelogTopics()) {
+                // the expected number of partitions is the max value of TaskId.partition + 1
+                int numPartitions = UNKNOWN;
+                for (final TaskId task : topicGroupTasks) {
+                    if (numPartitions < task.partition + 1) {
+                        numPartitions = task.partition + 1;
+                    }
+                }
+                topicConfig.setNumberOfPartitions(numPartitions);
+                changelogTopicMetadata.put(topicConfig.name(), topicConfig);
+            }
+            sourceTopicBasedChangelogTopics.addAll(topicsInfo.sourceTopicChangelogs());
+        }
+
+        final Set<String> newlyCreatedChangelogTopics = internalTopicManager.makeReady(changelogTopicMetadata);
+        log.debug("Created state changelog topics {} from the parsed topology.", changelogTopicMetadata.values());
+
+        for (final Map.Entry<TaskId, Set<TopicPartition>> entry : changelogPartitionsForTask.entrySet()) {
+            final TaskId taskId = entry.getKey();
+            final Set<TopicPartition> topicPartitions = entry.getValue();
+            for (final TopicPartition topicPartition : topicPartitions) {
+                if (!newlyCreatedChangelogTopics.contains(topicPartition.topic())) {
+                    preExistingChangelogPartitionsForTask.computeIfAbsent(taskId, task -> new HashSet<>()).add(topicPartition);
+                    if (!sourceTopicBasedChangelogTopics.contains(topicPartition.topic())) {
+                        preExistingNonSourceTopicBasedChangelogPartitions.add(topicPartition);
+                    } else {
+                        sourceTopicBasedChangelogTopicPartitions.add(topicPartition);

Review comment:
       Also here, if a changelog is source based it has to be pre-existing, because the `InternalTopicManager` will not setup source topics. However, making this clearer in the name does not harm.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r585542433



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();
+    private final Map<TaskId, Set<TopicPartition>> preExistingChangelogPartitionsForTask = new HashMap<>();
+    private final Set<TopicPartition> preExistingNonSourceTopicBasedChangelogPartitions = new HashSet<>();
+    private final Set<String> sourceTopicBasedChangelogTopics = new HashSet<>();
+    private final Set<TopicPartition> sourceTopicBasedChangelogTopicPartitions = new HashSet<>();
+    private final Logger log;
+
+    public ChangelogTopics(final InternalTopicManager internalTopicManager,
+                           final Map<Integer, TopicsInfo> topicGroups,
+                           final Map<Integer, Set<TaskId>> tasksForTopicGroup,
+                           final String logPrefix) {
+        this.internalTopicManager = internalTopicManager;
+        this.topicGroups = topicGroups;
+        this.tasksForTopicGroup = tasksForTopicGroup;
+        final LogContext logContext = new LogContext(logPrefix);
+        log = logContext.logger(getClass());
+    }
+
+    public void setup() {

Review comment:
       I agree with you and if I remember correctly you brought this up also on my change that introduces `RepartitionTopics`. Once we have the explicit user initialisation as described in KIP-698, we need to distinguish between setting up the broker-side state for Streams and verifying the the broker-side change. That requires changes in the `InternalTopicManager`. Currently, I could not separate this two concerns and left both in the setup method. Moving forward I will put everything that can be initialized in the constructor to avoid the mistakes you refer to. Consider the state of this class a temporary situation that will be soon fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #10163: KAFKA-10357: Extract setup of changelog from Streams partition assignor

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #10163:
URL: https://github.com/apache/kafka/pull/10163#discussion_r579332437



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN;
+
+public class ChangelogTopics {
+
+    private final InternalTopicManager internalTopicManager;
+    private final Map<Integer, TopicsInfo> topicGroups;
+    private final Map<Integer, Set<TaskId>> tasksForTopicGroup;
+    private final Map<TaskId, Set<TopicPartition>> changelogPartitionsForTask = new HashMap<>();
+    private final Map<TaskId, Set<TopicPartition>> preExistingChangelogPartitionsForTask = new HashMap<>();
+    private final Set<TopicPartition> preExistingNonSourceTopicBasedChangelogPartitions = new HashSet<>();
+    private final Set<String> sourceTopicBasedChangelogTopics = new HashSet<>();
+    private final Set<TopicPartition> sourceTopicBasedChangelogTopicPartitions = new HashSet<>();
+    private final Logger log;
+
+    public ChangelogTopics(final InternalTopicManager internalTopicManager,
+                           final Map<Integer, TopicsInfo> topicGroups,
+                           final Map<Integer, Set<TaskId>> tasksForTopicGroup,
+                           final String logPrefix) {
+        this.internalTopicManager = internalTopicManager;
+        this.topicGroups = topicGroups;
+        this.tasksForTopicGroup = tasksForTopicGroup;
+        final LogContext logContext = new LogContext(logPrefix);
+        log = logContext.logger(getClass());
+    }
+
+    public void setup() {
+        // add tasks to state change log topic subscribers
+        final Map<String, InternalTopicConfig> changelogTopicMetadata = new HashMap<>();
+        for (final Map.Entry<Integer, TopicsInfo> entry : topicGroups.entrySet()) {
+            final int topicGroupId = entry.getKey();
+            final TopicsInfo topicsInfo = entry.getValue();
+
+            final Set<TaskId> topicGroupTasks = tasksForTopicGroup.get(topicGroupId);
+            if (topicGroupTasks == null) {
+                log.debug("No tasks found for topic group {}", topicGroupId);
+                continue;
+            } else if (topicsInfo.stateChangelogTopics.isEmpty()) {
+                continue;
+            }
+
+            for (final TaskId task : topicGroupTasks) {
+                final Set<TopicPartition> changelogTopicPartitions = topicsInfo.stateChangelogTopics
+                    .keySet()
+                    .stream()
+                    .map(topic -> new TopicPartition(topic, task.partition))
+                    .collect(Collectors.toSet());
+                changelogPartitionsForTask.put(task, changelogTopicPartitions);
+            }
+
+            for (final InternalTopicConfig topicConfig : topicsInfo.nonSourceChangelogTopics()) {
+                // the expected number of partitions is the max value of TaskId.partition + 1
+                int numPartitions = UNKNOWN;
+                for (final TaskId task : topicGroupTasks) {
+                    if (numPartitions < task.partition + 1) {
+                        numPartitions = task.partition + 1;
+                    }
+                }
+                topicConfig.setNumberOfPartitions(numPartitions);
+                changelogTopicMetadata.put(topicConfig.name(), topicConfig);
+            }
+            sourceTopicBasedChangelogTopics.addAll(topicsInfo.sourceTopicChangelogs());
+        }
+
+        final Set<String> newlyCreatedChangelogTopics = internalTopicManager.makeReady(changelogTopicMetadata);
+        log.debug("Created state changelog topics {} from the parsed topology.", changelogTopicMetadata.values());
+
+        for (final Map.Entry<TaskId, Set<TopicPartition>> entry : changelogPartitionsForTask.entrySet()) {
+            final TaskId taskId = entry.getKey();
+            final Set<TopicPartition> topicPartitions = entry.getValue();
+            for (final TopicPartition topicPartition : topicPartitions) {
+                if (!newlyCreatedChangelogTopics.contains(topicPartition.topic())) {
+                    preExistingChangelogPartitionsForTask.computeIfAbsent(taskId, task -> new HashSet<>()).add(topicPartition);
+                    if (!sourceTopicBasedChangelogTopics.contains(topicPartition.topic())) {
+                        preExistingNonSourceTopicBasedChangelogPartitions.add(topicPartition);
+                    } else {
+                        sourceTopicBasedChangelogTopicPartitions.add(topicPartition);
+                    }
+                }
+            }
+        }
+    }
+
+    public Set<TopicPartition> preExistingNonSourceTopicBasedPartitions() {
+        return Collections.unmodifiableSet(preExistingNonSourceTopicBasedChangelogPartitions);
+    }
+
+    public Set<TopicPartition> preExistingPartitionsFor(final TaskId taskId) {
+        if (preExistingChangelogPartitionsForTask.containsKey(taskId)) {
+            return Collections.unmodifiableSet(preExistingChangelogPartitionsForTask.get(taskId));
+        }
+        return Collections.emptySet();
+    }
+
+    public Set<TopicPartition> sourceTopicBasedPartitions() {
+        return Collections.unmodifiableSet(sourceTopicBasedChangelogTopicPartitions);
+    }
+
+    public Set<TaskId> taskIds() {
+        return Collections.unmodifiableSet(changelogPartitionsForTask.keySet());
+    }

Review comment:
       I added some methods to simplify some code in the assignor.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org