ableegoldman commented on a change in pull request #9835:
URL: https://github.com/apache/kafka/pull/9835#discussion_r561340839



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.clients.consumer.Consumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+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.metrics.StreamsMetricsImpl;
+import org.slf4j.Logger;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+class Tasks {
+    private final Logger log;
+    private final InternalTopologyBuilder builder;
+    private final StreamsMetricsImpl streamsMetrics;
+
+    private final Map<TaskId, Task> allTasksPerId = new TreeMap<>();
+    private final Map<TaskId, Task> readOnlyTasksPerId = 
Collections.unmodifiableMap(allTasksPerId);
+    private final Collection<Task> readOnlyTasks = 
Collections.unmodifiableCollection(allTasksPerId.values());
+
+    // TODO: change type to `StreamTask`
+    private final Map<TaskId, Task> activeTasksPerId = new TreeMap<>();
+    // TODO: change type to `StreamTask`
+    private final Map<TopicPartition, Task> activeTasksPerPartition = new 
HashMap<>();
+    // TODO: change type to `StreamTask`
+    private final Map<TaskId, Task> readOnlyActiveTasksPerId = 
Collections.unmodifiableMap(activeTasksPerId);
+    private final Set<TaskId> readOnlyActiveTaskIds = 
Collections.unmodifiableSet(activeTasksPerId.keySet());
+    // TODO: change type to `StreamTask`
+    private final Collection<Task> readOnlyActiveTasks = 
Collections.unmodifiableCollection(activeTasksPerId.values());
+
+    // TODO: change type to `StandbyTask`
+    private final Map<TaskId, Task> standbyTasksPerId = new TreeMap<>();
+    // TODO: change type to `StandbyTask`
+    private final Map<TaskId, Task> readOnlyStandbyTasksPerId = 
Collections.unmodifiableMap(standbyTasksPerId);
+    private final Set<TaskId> readOnlyStandbyTaskIds = 
Collections.unmodifiableSet(standbyTasksPerId.keySet());
+
+    private final ActiveTaskCreator activeTaskCreator;
+    private final StandbyTaskCreator standbyTaskCreator;
+
+    private Consumer<byte[], byte[]> mainConsumer;
+
+    Tasks(final String logPrefix,
+          final InternalTopologyBuilder builder,
+          final StreamsMetricsImpl streamsMetrics,
+          final ActiveTaskCreator activeTaskCreator,
+          final StandbyTaskCreator standbyTaskCreator) {
+
+        final LogContext logContext = new LogContext(logPrefix);
+        log = logContext.logger(getClass());
+
+        this.builder = builder;
+        this.streamsMetrics = streamsMetrics;
+        this.activeTaskCreator = activeTaskCreator;
+        this.standbyTaskCreator = standbyTaskCreator;
+    }
+
+    void setMainConsumer(final Consumer<byte[], byte[]> mainConsumer) {
+        this.mainConsumer = mainConsumer;
+    }
+
+    void createTasks(final Map<TaskId, Set<TopicPartition>> 
activeTasksToCreate,
+                     final Map<TaskId, Set<TopicPartition>> 
standbyTasksToCreate) {
+        for (final Map.Entry<TaskId, Set<TopicPartition>> taskToBeCreated : 
activeTasksToCreate.entrySet()) {
+            final TaskId taskId = taskToBeCreated.getKey();
+
+            if (activeTasksPerId.containsKey(taskId)) {
+                throw new IllegalStateException("Attempted to create an active 
task that we already own: " + taskId);
+            }
+        }
+
+        for (final Map.Entry<TaskId, Set<TopicPartition>> taskToBeCreated : 
standbyTasksToCreate.entrySet()) {
+            final TaskId taskId = taskToBeCreated.getKey();
+
+            if (standbyTasksPerId.containsKey(taskId)) {
+                throw new IllegalStateException("Attempted to create a standby 
task that we already own: " + taskId);
+            }
+        }
+
+        // keep this check to simplify testing (ie, no need to mock 
`activeTaskCreator`)
+        if (!activeTasksToCreate.isEmpty()) {
+            // TODO: change type to `StreamTask`
+            for (final Task activeTask : 
activeTaskCreator.createTasks(mainConsumer, activeTasksToCreate)) {
+                activeTasksPerId.put(activeTask.id(), activeTask);
+                allTasksPerId.put(activeTask.id(), activeTask);
+                for (final TopicPartition topicPartition : 
activeTask.inputPartitions()) {
+                    activeTasksPerPartition.put(topicPartition, activeTask);
+                }
+            }
+        }
+
+        // keep this check to simplify testing (ie, no need to mock 
`standbyTaskCreator`)
+        if (!standbyTasksToCreate.isEmpty()) {
+            // TODO: change type to `StandbyTask`
+            for (final Task standbyTask : 
standbyTaskCreator.createTasks(standbyTasksToCreate)) {
+                standbyTasksPerId.put(standbyTask.id(), standbyTask);
+                allTasksPerId.put(standbyTask.id(), standbyTask);
+            }
+        }
+    }
+
+    void convertActiveToStandby(final StreamTask activeTask,
+                                final Set<TopicPartition> partitions,
+                                final Map<TaskId, RuntimeException> 
taskCloseExceptions) {
+        if (activeTasksPerId.remove(activeTask.id()) == null) {
+            throw new IllegalStateException("Attempted to convert unknown 
active task to standby task: " + activeTask.id());
+        }
+        activeTasksPerPartition.entrySet().stream()
+            .filter(e -> e.getValue().id().equals(activeTask.id()))
+            .forEach(e -> activeTasksPerPartition.remove(e.getKey()));

Review comment:
       @mjsax this seems to be throwing a ConcurrentModificationException




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to