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/12/09 19:49:55 UTC

[GitHub] [kafka] cadonna commented on a change in pull request #10851: KAFKA-6718 / Rack aware standby task assignor

cadonna commented on a change in pull request #10851:
URL: https://github.com/apache/kafka/pull/10851#discussion_r765909088



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.computeTasksToRemainingStandbys;
+
+/**
+ * Distributes standby tasks over different tag dimensions.
+ * Only tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags} are taken into account.
+ * Standby task distribution is on a best-effort basis. For example, if there are not enough clients available
+ * on different tag dimensions compared to an active and corresponding standby task,
+ * in that case, the algorithm will fall back to distributing tasks on least-loaded clients.
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    @Override
+    public boolean assign(final Map<UUID, ClientState> clients,
+                          final Set<TaskId> allTaskIds,
+                          final Set<TaskId> statefulTaskIds,
+                          final AssignorConfiguration.AssignmentConfigs configs) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToValues = new HashMap<>();
+        final Map<TagEntry, Set<UUID>> tagEntryToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagEntryToClients, tagKeyToValues);
+
+        for (final TaskId statefulTaskId : statefulTaskIds) {
+            for (final Map.Entry<UUID, ClientState> entry : clients.entrySet()) {
+                final UUID clientId = entry.getKey();
+                final ClientState clientState = entry.getValue();
+
+                if (clientState.activeTasks().contains(statefulTaskId)) {
+                    assignStandbyTasksForActiveTask(
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+                }
+            }
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    @Override
+    public boolean isAllowedTaskMovement(final ClientState source, final ClientState destination) {
+        final Map<String, String> sourceClientTags = source.clientTags();
+        final Map<String, String> destinationClientTags = destination.clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                 final Map<String, Set<String>> tagKeyToValues) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue);
+                tagEntryToClients.computeIfAbsent(new TagEntry(tagKey, tagValue), ignored -> new HashSet<>()).add(clientId);
+            });
+        }
+    }
+
+    private static void assignStandbyTasksForActiveTask(final int numStandbyReplicas,
+                                                        final TaskId activeTaskId,
+                                                        final UUID activeTaskClient,
+                                                        final Set<String> rackAwareAssignmentTags,
+                                                        final Map<UUID, ClientState> clientStates,
+                                                        final Map<TaskId, Integer> tasksToRemainingStandbys,
+                                                        final Map<String, Set<String>> tagKeyToValues,
+                                                        final Map<TagEntry, Set<UUID>> tagEntryToClients) {
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clientStates.get(client).hasAssignedTask(t),
+            client -> clientStates.get(client).assignedTaskLoad()
+        );
+
+        final Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        while (numRemainingStandbys > 0) {
+            final Set<UUID> clientsOnAlreadyUsedTagDimensions = findClientsOnUsedTagDimensions(

Review comment:
       For each standby of a single active task the set `clientsOnAlreadyUsedTagDimensions` is computed from scratch. I think this is not necessary since the clients on already used tag dimensions that we found for the first standby are still valid for the second standby and the clients on already used tag dimensions found for the second standby are still valid for the third standby and so on. This is true because we only add clients to the set `usedClients` but we never remove any. I think we can compute `clientsOnAlreadyUsedTagDimensions` incrementally for each standby of a single active task instead of computing it from scratch each time.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.computeTasksToRemainingStandbys;
+
+/**
+ * Distributes standby tasks over different tag dimensions.
+ * Only tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags} are taken into account.
+ * Standby task distribution is on a best-effort basis. For example, if there are not enough clients available
+ * on different tag dimensions compared to an active and corresponding standby task,
+ * in that case, the algorithm will fall back to distributing tasks on least-loaded clients.
+ */

Review comment:
       Something does not work as expected in this algorithm. According to this doc, the assignor should fall back to distributing tasks on least-loaded clients. However, the following test case fails:
   ```
       @Test
       public void shouldDistributeTasksOnLeastLoadedClientsWhenThereAreNoEnoughUniqueTagDimensions() {
           final Map<UUID, ClientState> clientStates = mkMap(
               mkEntry(UUID_1, createClientStateWithCapacity(3, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_1)), TASK_0_0)),
               mkEntry(UUID_2, createClientStateWithCapacity(3, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_2)), TASK_0_1)),
               mkEntry(UUID_3, createClientStateWithCapacity(3, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_1)), TASK_0_2)),
               mkEntry(UUID_4, createClientStateWithCapacity(3, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_1)), TASK_1_0))
           );
   
           final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
           final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(1, CLUSTER_TAG, ZONE_TAG);
   
           new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
   
           assertEquals(1, clientStates.get(UUID_1).standbyTaskCount());
           assertEquals(1, clientStates.get(UUID_2).standbyTaskCount());
           assertEquals(1, clientStates.get(UUID_3).standbyTaskCount());
           assertEquals(1, clientStates.get(UUID_4).standbyTaskCount());
       }
   ``` 
   The standby task for active task 0_0 can be put on client UUID_2 and the standby task for active task 0_1 can be put on client UUID_1 without breaking rack awareness constraints. Standby tasks for active tasks 0_2 and 1_0 cannot be put on any client without breaking rack awareness, so they should be distributed on least-loaded clients. However, that does apparently not happen, because client UUID_3 and UUID_4 are not assigned any standby.    
   




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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