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 2020/04/24 02:34:39 UTC

[GitHub] [kafka] ableegoldman commented on a change in pull request #8541: KAFKA-6145: KIP-441: Add TaskAssignor class config

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -713,23 +713,18 @@ private boolean assignTasksToClients(final Set<String> allSourceTopics,
             allTasks, clientStates, numStandbyReplicas());
 
         final TaskAssignor taskAssignor;
-        if (highAvailabilityEnabled) {
-            if (lagComputationSuccessful) {
-                taskAssignor = new HighAvailabilityTaskAssignor(
-                    clientStates,
-                    allTasks,
-                    statefulTasks,
-                    assignmentConfigs);
-            } else {
-                log.info("Failed to fetch end offsets for changelogs, will return previous assignment to clients and "
-                             + "trigger another rebalance to retry.");
-                setAssignmentErrorCode(AssignorError.REBALANCE_NEEDED.code());
-                taskAssignor = new StickyTaskAssignor(clientStates, allTasks, statefulTasks, assignmentConfigs, true);
-            }
+        if (!lagComputationSuccessful) {

Review comment:
       Fine with me (although it does slightly detract from the opt-out possibility). WDYT about adding a retry backoff though? I'm a bit concerned we might just end up stuck in a loop of useless rebalancing, and waiting the full `probing.rebalance.interval` doesn't feel right either

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java
##########
@@ -41,8 +42,8 @@
 import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION;
 
 public final class AssignorConfiguration {
-    public static final String HIGH_AVAILABILITY_ENABLED_CONFIG = "internal.high.availability.enabled";
-    private final boolean highAvailabilityEnabled;
+    public static final String INTERNAL_TASK_ASSIGNOR_CLASS = "internal.task.assignor.class";

Review comment:
       Should we put this with the other Streams internal configs? And/or follow the pattern of prefix+suffixing with `__` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -95,6 +94,11 @@ public boolean assign() {
 
         assignStatelessActiveTasks();
 
+        log.info("Decided on assignment: " +

Review comment:
       I'm all for useful logging 👍 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/PriorTaskAssignor.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.assignment;
+
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentConfigs;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+public class PriorTaskAssignor implements TaskAssignor {
+    private final StickyTaskAssignor delegate;
+
+    public PriorTaskAssignor() {
+        delegate = new StickyTaskAssignor(true);

Review comment:
       Thanks for the improvement, this feels a lot nicer




----------------------------------------------------------------
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