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/06/09 10:03:04 UTC

[GitHub] [kafka] lkokhreidze opened a new pull request #10851: KAFKA-6718 / Rack aware standby task assignor

lkokhreidze opened a new pull request #10851:
URL: https://github.com/apache/kafka/pull/10851


   This PR is part of [KIP-708](https://cwiki.apache.org/confluence/display/KAFKA/KIP-708%3A+Rack+awareness+for+Kafka+Streams) and adds rack aware standby task assignment logic.
   
   Rack aware standby task assignment won't be functional until all part of this KIP gets merged.
   
   Splitting PRs into three smaller PRs to make the review process easier to follow. Overall plan is the following:
   
   👉  Rack aware standby task assignment logic.
   ⏭️  Protocol change, add clientTags to SubscriptionInfoData https://github.com/apache/kafka/pull/10802
   ⏭️  Add required configurations to StreamsConfig (public API change, at this point we should have full functionality)
   
   This PR implements first point of the above mentioned plan.
   
   ### 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] lkokhreidze commented on pull request #10851: KAFKA-6718 / Rack aware standby task assignor

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


   Call for review @cadonna @vvcephei @ableegoldman


-- 
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] lkokhreidze commented on a change in pull request #10851: KAFKA-6718 / Rack aware standby task assignor

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorInitializer.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.internals.assignment.AssignorConfiguration.AssignmentConfigs;
+
+/**
+ * Decides which {@link TaskAssignor} implementation to use for standby task assignment based on {@link AssignmentConfigs}.
+ */
+class StandbyTaskAssignorInitializer {

Review comment:
       Moved factory method in `StandbyTaskAssignor` interface itself. Hope it addresses your comment.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));
+
+        return true;
+    }
+
+    @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<String, Set<UUID>> tagValueToClients,
+                                                 final Map<String, Set<String>> tagKeyToTagValues) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue);
+                tagValueToClients.computeIfAbsent(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>> tagKeyToTagValues,
+                                                        final Map<String, Set<UUID>> tagValueToClients) {
+        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(
+                usedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagValueToClients,
+                tagKeyToTagValues
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            standbyTaskClientsByTaskLoad.offerAll(clientsOnAlreadyUsedTagDimensions);

Review comment:
       I've removed this line and pushed the changes.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.Set;
+import java.util.SortedMap;
+import java.util.UUID;
+
+interface StandbyTaskAssignor {
+
+    void assignStandbyTasks(final SortedMap<UUID, ClientState> clientStates, final Set<TaskId> statefulTasks);
+
+    default boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        return true;
+    }
+
+    static StandbyTaskAssignor init(final AssignmentConfigs configs) {
+        if (!configs.rackAwareAssignmentTags.isEmpty()) {
+            return new ClientTagAwareStandbyTaskAssignor(configs);
+        } else {
+            return new DefaultStandbyTaskAssignor(configs);
+        }
+    }

Review comment:
       I would prefer to make this interface independent of its implementations. If you put the factory method here, the interface is not independent anymore. I would prefer a factory method named `createStandbyTaskAssignor()` in `HighAvailabilityTaskAssignor` similar to the existing factory method `createTaskAssignor()` in `StreamsPartitionAssignor`. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementAttempt.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+class TaskMovementAttempt {

Review comment:
       I would prefer to just add two parameters -- `source` and `destination` -- to the `isValidMovement()` method in `StandbyTaskAssignor` and get rid of this class. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
##########
@@ -64,6 +66,7 @@ public ClientState() {
         previousStandbyTasks.taskIds(new TreeSet<>());
         previousActiveTasks.taskIds(new TreeSet<>());
 
+        clientTags = new HashMap<>();

Review comment:
       I guess the initialization in the constructor on line 79 is only temporary. This will change in one of the next PRs. Nevertheless, I agree that it would also be fine to move the initialization to the field declaration for now.
   
   I would even propose to pass the client tags to the constructor, since those are kind of constants coming from the config.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.Set;
+import java.util.SortedMap;
+import java.util.UUID;
+
+interface StandbyTaskAssignor {
+
+    void assignStandbyTasks(final SortedMap<UUID, ClientState> clientStates, final Set<TaskId> statefulTasks);
+
+    default boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        return true;
+    }

Review comment:
       nit: I think `isAllowedTaskMovement()` reflects better the meaning of this method.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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 extends StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    ClientTagAwareStandbyTaskAssignor(final AssignmentConfigs configs) {
+        super(configs);
+    }
+
+    @Override
+    public void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                   final TreeMap<UUID, ClientState> clientStates) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+
+        final StandbyTaskDistributor standbyTaskDistributor = new StandbyTaskDistributor(
+            numStandbyReplicas,
+            clientStates,
+            rackAwareAssignmentTags,
+            statefulTasksWithClients
+        );
+
+        statefulTasksWithClients.forEach(standbyTaskDistributor::assignStandbyTasksForActiveTask);
+    }
+
+    @Override
+    public boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        final Map<String, String> sourceClientTags = taskMovementAttempt.sourceClient().clientTags();
+        final Map<String, String> destinationClientTags = taskMovementAttempt.destinationClient().clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static final class StandbyTaskDistributor {

Review comment:
       I see your point, but I do also not see the need for an internal state for which we need to avoid invalidation. Variables `numStandbyReplicas` and `numStandbyReplicas` are configs that can be stored as member fields of `ClientTagAwareStandbyTaskAssignor` or passed along to the methods that need them. Variables `tagKeyToTagValuesMapping`, `clientsPerTagValue`, `standbyTaskClientsByTaskLoad`, and `clientStates` can also be passed to the methods that need them. Avoiding state makes reasoning about code simpler and here it seems possible to avoid state. See `HighAvailabilityTaskAssignor`, it does not have any state.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.Set;
+import java.util.SortedMap;
+import java.util.UUID;
+
+interface StandbyTaskAssignor {

Review comment:
       Minor: You could extend interface `TaskAssignor` and remove `assignStandbyTasks()` from this interface since `assign()` in `TaskAssignor` has almost the same signature. The difference is parameters `configs` and `allTaskIds`. You will need `configs` if you will not keep the config as a member variable as mentioned in my other comment. You will not need `allTaskIds`, but that would be OK, I guess.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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);
+
+    private final AssignmentConfigs configs;
+
+    ClientTagAwareStandbyTaskAssignor(final AssignmentConfigs configs) {
+        this.configs = configs;
+    }
+
+    @Override
+    public void assignStandbyTasks(final SortedMap<UUID, ClientState> clientStates, final Set<TaskId> statefulTasks) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+        final Map<TaskId, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTasks.forEach(statefulTaskId -> clientStates.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final StandbyTaskDistributor standbyTaskDistributor = new StandbyTaskDistributor(
+            numStandbyReplicas,
+            clientStates,
+            rackAwareAssignmentTags,
+            statefulTasksWithClients
+        );
+
+        statefulTasksWithClients.forEach(standbyTaskDistributor::assignStandbyTasksForActiveTask);
+    }
+
+    @Override
+    public boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        final Map<String, String> sourceClientTags = taskMovementAttempt.sourceClient().clientTags();
+        final Map<String, String> destinationClientTags = taskMovementAttempt.destinationClient().clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static final class StandbyTaskDistributor {
+        private final int numStandbyReplicas;
+        private final SortedMap<UUID, ClientState> clientStates;
+        private final Set<String> rackAwareAssignmentTags;
+        private final Map<String, Set<String>> tagKeyToTagValuesMapping;
+        private final Map<String, Set<UUID>> clientsPerTagValue;
+        private final ConstrainedPrioritySet standbyTaskClientsByTaskLoad;
+
+        private final Set<UUID> usedClients = new HashSet<>();
+        private final Map<TaskId, Integer> tasksToRemainingStandbys;
+
+        StandbyTaskDistributor(final int numStandbyReplicas,
+                               final SortedMap<UUID, ClientState> clientStates,
+                               final Set<String> rackAwareAssignmentTags,
+                               final Map<TaskId, UUID> statefulTasksWithClients) {
+            tagKeyToTagValuesMapping = new HashMap<>();
+            clientsPerTagValue = new HashMap<>();
+            standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+                (client, t) -> !clientStates.get(client).hasAssignedTask(t),
+                client -> clientStates.get(client).assignedTaskLoad()
+            );

Review comment:
       Would it be possible to integrate the tag constraint as part of the constraint on the priority queue?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -183,7 +164,7 @@ private static void balanceTasksOverThreads(final SortedMap<UUID, ClientState> c
                     while (shouldMoveATask(sourceClientState, destinationClientState) && sourceIterator.hasNext()) {
                         final TaskId taskToMove = sourceIterator.next();
                         final boolean canMove = !destinationClientState.hasAssignedTask(taskToMove);
-                        if (canMove) {
+                        if (canMove && taskMovementAttemptPredicate.test(new TaskMovementAttempt(taskToMove, sourceClientState, destinationClientState))) {

Review comment:
       Could you move the second condition to the `canMove` assignment on line 166? I think the condition is logically a part of `canMove`. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementAttempt.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+class TaskMovementAttempt {
+    private final TaskId taskId;
+    private final ClientState sourceClient;
+    private final ClientState destinationClient;
+
+    TaskMovementAttempt(final TaskId taskId,
+                        final ClientState sourceClient,
+                        final ClientState destinationClient) {
+        this.taskId = taskId;
+        this.sourceClient = sourceClient;
+        this.destinationClient = destinationClient;
+    }
+
+    public TaskId taskId() {
+        return taskId;
+    }

Review comment:
       The task ID is never used. Could we remove it?




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



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

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


   Thanks @cadonna for the feedback.
   I've replied/addressed all of your comments.


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



[GitHub] [kafka] lkokhreidze edited a comment on pull request #10851: KAFKA-6718 / Rack aware standby task assignor

Posted by GitBox <gi...@apache.org>.
lkokhreidze edited a comment on pull request #10851:
URL: https://github.com/apache/kafka/pull/10851#issuecomment-905514160


   The general thought on the implementation. 
   As of now, we choose concrete `StandbyTaskAssignor` implementation based on passed `AssignmentConfigs` value.
   Instead, an alternative approach would be to have a chained standby task assignment based on multiple implementations. For instance, when have required client tag configuration is present, we can try to assign the standby tasks based on `List.of(new ClientTagAwareStandbyTaskAssignor(), new LeastLoadedClientStandbyTaskAssignor())`. This way, `ClientTagAwareStandbyTaskAssignor` can try to distribute the standby tasks based on tags dimensions. If there are no more client tag dimensions available, and we still have `any(tasksToRemainingStandbys) > 0`, the next implementation in the chain (in this case `LeastLoadedClientStandbyTaskAssignor`) will be called. With this, `LeastLoadedClientStandbyTaskAssignor` can default to assigning the remaining standbys to the least loaded clients. The benefit of this approach is that, right now, `ClientTagAwareStandbyTaskAssignor` does both assignments based on available dimensions and fallback to the least loaded if there are no enough tag dimensions
 . Current implementation leads to more complex code. While with the approach above, we can clearly separate the implementations without duplication (there's no code duplication, rather "logic" duplication).
   For now, I've chosen to go with the simplest approach - having two independent implementations and selecting an appropriate one based on passed `AssignmentConfigs.` Still, I wanted to share this idea here, just in case. 


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });
+
+                clientsPerTagValue.compute(tagValue, (key, currentValue) -> {
+                    final Set<UUID> clientIdsForTagValue = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    clientIdsForTagValue.add(clientId);
+                    return clientIdsForTagValue;
+                });
+            });
+        }
+    }
+
+    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>> tagKeyToTagValuesMapping = new HashMap<>();
+        final Map<String, Set<UUID>> clientsPerTagValue = new HashMap<>();
+
+        fillClientsTagStatistics(clientStates, clientsPerTagValue, tagKeyToTagValuesMapping);

Review comment:
       Pushed changes

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });
+
+                clientsPerTagValue.compute(tagValue, (key, currentValue) -> {
+                    final Set<UUID> clientIdsForTagValue = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    clientIdsForTagValue.add(clientId);
+                    return clientIdsForTagValue;
+                });

Review comment:
       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.

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

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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.TreeMap;
+import java.util.UUID;
+
+abstract class StandbyTaskAssignor {
+    protected final AssignmentConfigs configs;
+
+    StandbyTaskAssignor(final AssignmentConfigs configs) {
+        this.configs = configs;
+    }
+
+    abstract void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                     final TreeMap<UUID, ClientState> clientStates);

Review comment:
       I didn't give it much thought to be honest.
   
   `TreeMap` for the `clientStates` was already used in the `HighAvailabilityTaskAssignor` and went with the same signature here. I think it makes sense to change the contract to be a `SortedMap`. Will do that.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtils.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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 java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+final class StandbyTaskAssignmentUtils {
+    private StandbyTaskAssignmentUtils() {
+    }

Review comment:
       nit: close this bracket in the same line. That is: `private StandbyTaskAssignmentUtils() {}`

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultStandbyTaskAssignor.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Default standby task assignor that distributes standby tasks to the least loaded clients.
+ *
+ * @see org.apache.kafka.streams.processor.internals.assignment.ClientTagAwareStandbyTaskAssignor

Review comment:
       nit: `@see ClientTagAwareStandbyTaskAssignor` (no need the package name)

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java
##########
@@ -490,8 +498,8 @@ public void shouldNotAssignMoreThanMaxWarmupReplicas() {
                 /*acceptableRecoveryLag*/ 100L,
                 /*maxWarmupReplicas*/ 1,
                 /*numStandbyReplicas*/ 0,
-                /*probingRebalanceIntervalMs*/ 60 * 1000L
-            )
+                /*probingRebalanceIntervalMs*/ 60 * 1000L,
+                                          EMPTY_RACK_AWARE_ASSIGNMENT_TAGS)

Review comment:
       nit: indent issue. Could we add comment in front of `EMPTY_RACK_AWARE_ASSIGNMENT_TAGS`?

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java
##########
@@ -518,8 +526,8 @@ public void shouldNotAssignWarmupAndStandbyToTheSameClient() {
                 /*acceptableRecoveryLag*/ 100L,
                 /*maxWarmupReplicas*/ 1,
                 /*numStandbyReplicas*/ 1,
-                /*probingRebalanceIntervalMs*/ 60 * 1000L
-            )
+                /*probingRebalanceIntervalMs*/ 60 * 1000L,
+                                          EMPTY_RACK_AWARE_ASSIGNMENT_TAGS)

Review comment:
       same as above.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedClientTagDimensions(final UUID usedClient,
+                                                                  final int countOfUsedClients,
+                                                                  final Set<String> rackAwareAssignmentTags,
+                                                                  final Map<UUID, ClientState> clientStates,
+                                                                  final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                                  final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+            final String tagValue = usedClientTagEntry.getValue();
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag.
+            if (allTagValues.size() <= countOfUsedClients) {
+                continue;

Review comment:
       Consider the following example
   
   ```
   # Kafka Streams Client 1
   client.tag.zone: eu-central-1a
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
   
   # Kafka Streams Client 2
   client.tag.zone: eu-central-1b
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 3
   client.tag.zone: eu-central-1c
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
   
   # Kafka Streams Client 4
   client.tag.zone: eu-central-1a
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 5
   client.tag.zone: eu-central-1b
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 6
   client.tag.zone: eu-central-1c
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
   ```
   
   With the above we have following number of unique tag values:
   
   ```
   zone: 3 (eu-central-1a, eu-central-1b, eu-central-1c)
   cluster: 2 (k8s-cluster1, k8s-cluster2)
   ```
   
   Now lets say we have standby replica count as `2` and we want to active task is located on `Client 1`
   
   `usedClients=1` (because of the active task)
   
   ### 1s standby assignment
   
   During 1st standby takes assignment, we will exclude clients on following dimensions:
   
   ```
   zone: eu-central-1a
   cluster: k8s-cluster1
   ```
   
   Used clients will get incremented since we can allocate the client on different `zone` and `cluster`.
   `usedClients=2`
   
   ### 2nd standby assignment
   
   We will have to exclude `zone: eu-central-1a and (eu-central-1b || eu-central-1c)` tag values. We can do that, because
   after we exclude clients on the new tag, we still have clients on the one free tag value we can assign the next standby
   to.
   
   We can't exclude `cluster` because we have already used two clients, and we just have two unique values for
   the `cluster`
   tag, so it's impossible to get the ideal distribution with this configuration and number of instances. So we can only
   achieve partial distribution.
   
   So idea of this check is to ignore tags where we have less unique values than the clients we have already used. If we
   don't have this check, for the 2nd standby task assignment we would have excluded all the clients located
   on `k8s-cluster1` and `k8s-cluster2`, and there wouldn't be any client left to assign the standby task to. We would fall
   back to the least loaded client, BUT there will be no guarantee that least loaded client assignment would honor partial
   rack awareness.
   
   Hope this makes sense.
   




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            if (shouldBalanceLoad(clientStates.values(), standbyTaskClient)) {

Review comment:
       Answering why do we need this:
   I think with client tag aware standby task assignment, there's a much higher chance that we will overload some clients without this check. I think it's better to not to overload the clients and instead log the warning so users can do the needful of increasing the capacity in order to satisfy the rack awareness.




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



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

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






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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            if (shouldBalanceLoad(clientStates.values(), standbyTaskClient)) {
+                final Map<String, String> standbyClientTags = standbyTaskClient.clientTags();
+                log.warn("Can't assign {} of {} standby task for task [{}]. " +
+                         "There is not enough capacity on client(s) with {} tag dimensions. " +
+                         "To have a proper distribution of standby tasks across different tag dimensions, " +
+                         "increase the number of application instances with [{}] tag dimensions.",
+                         numRemainingStandbys, numStandbyReplicas, activeTaskId, standbyClientTags, standbyClientTags);
+                break;
+            }
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedTagDimensions(final Set<UUID> usedClients,
+                                                            final Set<String> rackAwareAssignmentTags,
+                                                            final Map<UUID, ClientState> clientStates,
+                                                            final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                            final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        for (final UUID usedClientId : usedClients) {
+            final Map<String, String> usedClientTags = clientStates.get(usedClientId).clientTags();
+
+            for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+                final String tagKey = usedClientTagEntry.getKey();
+
+                if (!rackAwareAssignmentTags.contains(tagKey)) {
+                    continue;

Review comment:
       Is it normal when this happened? Should we do anything to it? Or at least log something here?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)

Review comment:
       The variable name `polledClient` is unreadable. I think the variable is the client not having the same tag key/value, right? Could we give it a more meaningful name, ex: `clientUUIDNotOnUsedTagDimension`, or other better one if you have. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+

Review comment:
       When reaching this point, we have tried our best to assign standby tasks with rack awareness to all clients. I think we should have a debug log here, to log some current status, like current assignment, `pendingStandbyTaskToNumberRemainingStandbys`, `pendingStandbyTaskToClientId`, and mention we're going to distribute the remaining tasks with least loaded assignor...etc, for better troubleshooting.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -182,7 +174,8 @@ private static void balanceTasksOverThreads(final SortedMap<UUID, ClientState> c
                     final Iterator<TaskId> sourceIterator = sourceTasks.iterator();
                     while (shouldMoveATask(sourceClientState, destinationClientState) && sourceIterator.hasNext()) {
                         final TaskId taskToMove = sourceIterator.next();
-                        final boolean canMove = !destinationClientState.hasAssignedTask(taskToMove);
+                        final boolean canMove = !destinationClientState.hasAssignedTask(taskToMove)
+                                                && taskMovementAttemptPredicate.test(sourceClientState, destinationClientState);

Review comment:
       maybe add a comment here to mention we need to make sure the sourceClient tag matches to destinationClient tag if rack tag is enabled...something like that. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,

Review comment:
       TBH, I don't understand this method well before I read into the implementation. I think the method is trying to assign standby tasks to those clients without using the same tag key/value, right? If so, maybe we can change the name to `assignStandbyTasksToClientsWithoutSameTag`, or others you can think of. WDYT?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );

Review comment:
       Looks like the `findClientsOnUsedTagDimensions` method keeps finding duplicated `usedClients`. That is, if we have 10 `numRemainingStandbys`, we'll run `findClientsOnUsedTagDimensions` with 1 `usedClients` at first. And then, add one more, to have 2 `usedClients` at 2nd run, and add one to 3, 4, 5, ... 10. Is my understanding correct? If so, could we improve it? 




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultStandbyTaskAssignor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+class DefaultStandbyTaskAssignor implements StandbyTaskAssignor {

Review comment:
       I know there was no any java doc for the default least load assignor. But do you think we can add some comments to it, just like in `ClientTagAwareStandbyTaskAssignor`? I believe not everyone knows default assignor algorithm is least loaded assignor.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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,

Review comment:
       Could we add some java doc to this assign to briefly mention about the algorithm used in the assignor? Thanks.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
##########
@@ -59,29 +60,31 @@
 
     private int capacity;
 
-    public ClientState() {
-        this(0);
+    public ClientState(final Map<String, String> clientTags) {
+        this(0, clientTags);
     }
 
-    ClientState(final int capacity) {
+    ClientState(final int capacity, final Map<String, String> clientTags) {

Review comment:
       I'm wondering could we keep the original constructor and pass empty map into the new one? So that we don't have to make changes to the old caller. That is:
   
   ```
   public ClientState() {
       this(0, Collections.emptyMap());
   }
   
   ClientState(final int capacity) {
       this(capacity, Collections.emptyMap());
   }
   ```
   WDYT?




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);

Review comment:
       Good call. I don't know how I missed that...




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,337 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.createLeastLoadedPrioritySetConstrainedByAssignedTask;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = createLeastLoadedPrioritySetConstrainedByAssignedTask(clients);
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients,
+                        pendingStandbyTasksToClientId
+                    );
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +

Review comment:
       I think info log would also be OK here. I imagine users that are wondering why their standbys are not distributed as they would expect. With this information they can at least try to fix it on the config level. This log message should only happen at rebalance time, which should usually be rather seldom.
   If we decide to put the log message on info level, you should also change a bit the wording and not use variable names in it. Maybe some hints what the users can do to fix this would also be nice.
   
   Is it possible to separate the concerns of this log message and the one on line 135? Something along the lines of here the rack-aware standby assignment did not work due the tag config and on line 135 the assignment did not work due to too low number of instances. We can then put both on warn or info (do not forget to also check the related log message in the default standby assignor).   

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -115,55 +116,46 @@ private static void assignActiveStatefulTasks(final SortedMap<UUID, ClientState>
             clientStates,
             ClientState::activeTasks,
             ClientState::unassignActive,
-            ClientState::assignActive
+            ClientState::assignActive,
+            (source, destination) -> true
         );
     }
 
-    private static void assignStandbyReplicaTasks(final TreeMap<UUID, ClientState> clientStates,
-                                                  final Set<TaskId> statefulTasks,
-                                                  final int numStandbyReplicas) {
-        final Map<TaskId, Integer> tasksToRemainingStandbys =
-            statefulTasks.stream().collect(Collectors.toMap(task -> task, t -> numStandbyReplicas));
+    private void assignStandbyReplicaTasks(final TreeMap<UUID, ClientState> clientStates,
+                                           final Set<TaskId> allTaskIds,
+                                           final Set<TaskId> statefulTasks,
+                                           final AssignmentConfigs configs) {
+        if (configs.numStandbyReplicas == 0) {
+            return;
+        }
 
-        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
-            (client, task) -> !clientStates.get(client).hasAssignedTask(task),
-            client -> clientStates.get(client).assignedTaskLoad()
-        );
-        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+        final StandbyTaskAssignor standbyTaskAssignor = createStandbyTaskAssignor(configs);
 
-        for (final TaskId task : statefulTasks) {
-            int numRemainingStandbys = tasksToRemainingStandbys.get(task);
-            while (numRemainingStandbys > 0) {
-                final UUID client = standbyTaskClientsByTaskLoad.poll(task);
-                if (client == null) {
-                    break;
-                }
-                clientStates.get(client).assignStandby(task);
-                numRemainingStandbys--;
-                standbyTaskClientsByTaskLoad.offer(client);
-            }
-
-            if (numRemainingStandbys > 0) {
-                log.warn("Unable to assign {} of {} standby tasks for task [{}]. " +
-                             "There is not enough available capacity. You should " +
-                             "increase the number of application instances " +
-                             "to maintain the requested number of standby replicas.",
-                         numRemainingStandbys, numStandbyReplicas, task);
-            }
-        }
+        standbyTaskAssignor.assign(clientStates, allTaskIds, statefulTasks, configs);
 
         balanceTasksOverThreads(
             clientStates,
             ClientState::standbyTasks,
             ClientState::unassignStandby,
-            ClientState::assignStandby
+            ClientState::assignStandby,
+            standbyTaskAssignor::isAllowedTaskMovement
         );
     }
 
+    // Visible for testing
+    static StandbyTaskAssignor createStandbyTaskAssignor(final AssignmentConfigs configs) {
+        if (!configs.rackAwareAssignmentTags.isEmpty()) {
+            return new ClientTagAwareStandbyTaskAssignor();
+        } else {
+            return new DefaultStandbyTaskAssignor();
+        }
+    }
+

Review comment:
       This can be done in a follow-up PR:
   I am not a big fan of `// Visible for testing` because it often means that we missed to extract code to separate classes. Here I would definitely extract this code to a factory class. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,337 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.createLeastLoadedPrioritySetConstrainedByAssignedTask;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = createLeastLoadedPrioritySetConstrainedByAssignedTask(clients);
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients,
+                        pendingStandbyTasksToClientId
+                    );
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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;
+    }
+
+    // Visible for testing
+    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);
+            });
+        }
+    }
+
+    // Visible for testing
+    static void assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                             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 Map<TaskId, UUID> pendingStandbyTasksToClientId) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
+
+        UUID lastUsedClient = activeTaskClient;
+        do {
+            updateClientsOnAlreadyUsedTagEntries(
+                lastUsedClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues,
+                tagEntryToUsedClients
+            );
+
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(uuid, tagEntryToUsedClients)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+            numRemainingStandbys--;
+
+            lastUsedClient = clientOnUnusedTagDimensions;
+        } while (numRemainingStandbys > 0);
+
+        if (numRemainingStandbys > 0) {
+            pendingStandbyTasksToClientId.put(activeTaskId, activeTaskClient);
+            tasksToRemainingStandbys.put(activeTaskId, numRemainingStandbys);
+        } else {
+            tasksToRemainingStandbys.remove(activeTaskId);
+        }
+    }
+
+    private static boolean isClientUsedOnAnyOfTheTagEntries(final UUID client,
+                                                            final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        return tagEntryToUsedClients.values().stream().anyMatch(usedClients -> usedClients.contains(client));
+    }
+
+    private static void updateClientsOnAlreadyUsedTagEntries(final UUID usedClient,
+                                                             final int countOfUsedClients,
+                                                             final Set<String> rackAwareAssignmentTags,
+                                                             final Map<UUID, ClientState> clientStates,
+                                                             final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                             final Map<String, Set<String>> tagKeyToValues,
+                                                             final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+
+            // Consider the following client setup where we need to distribute 2 standby tasks for each stateful task.
+            //
+            // # Kafka Streams Client 1
+            // client.tag.zone: eu-central-1a
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 2
+            // client.tag.zone: eu-central-1b
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 3
+            // client.tag.zone: eu-central-1c
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 4
+            // client.tag.zone: eu-central-1a
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 5
+            // client.tag.zone: eu-central-1b
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 6
+            // client.tag.zone: eu-central-1c
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // Since we have only two unique `cluster` tag values,
+            // we can only achieve "ideal" distribution on the 1st standby task assignment.
+            // Ideal distribution for the 1st standby task can be achieved because we can assign standby task
+            // to the client located on different cluster and zone compared to an active task.
+            // We can't consider the `cluster` tag for the 2nd standby task assignment because the 1st standby
+            // task would already be assigned on different cluster compared to the active one, which means
+            // we have already used all the available cluster tag values. Taking the `cluster` tag into consideration
+            // for the 2nd standby task assignment would affectively mean excluding all the clients.

Review comment:
       ```suggestion
               // for the 2nd standby task assignment would effectively mean excluding all the clients.
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,337 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.createLeastLoadedPrioritySetConstrainedByAssignedTask;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = createLeastLoadedPrioritySetConstrainedByAssignedTask(clients);
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients,
+                        pendingStandbyTasksToClientId
+                    );
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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;
+    }
+
+    // Visible for testing
+    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);
+            });
+        }
+    }
+
+    // Visible for testing
+    static void assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                             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 Map<TaskId, UUID> pendingStandbyTasksToClientId) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
+
+        UUID lastUsedClient = activeTaskClient;
+        do {
+            updateClientsOnAlreadyUsedTagEntries(
+                lastUsedClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues,
+                tagEntryToUsedClients
+            );
+
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(uuid, tagEntryToUsedClients)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+            numRemainingStandbys--;
+
+            lastUsedClient = clientOnUnusedTagDimensions;
+        } while (numRemainingStandbys > 0);
+
+        if (numRemainingStandbys > 0) {
+            pendingStandbyTasksToClientId.put(activeTaskId, activeTaskClient);
+            tasksToRemainingStandbys.put(activeTaskId, numRemainingStandbys);
+        } else {
+            tasksToRemainingStandbys.remove(activeTaskId);
+        }
+    }
+
+    private static boolean isClientUsedOnAnyOfTheTagEntries(final UUID client,
+                                                            final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        return tagEntryToUsedClients.values().stream().anyMatch(usedClients -> usedClients.contains(client));
+    }
+
+    private static void updateClientsOnAlreadyUsedTagEntries(final UUID usedClient,
+                                                             final int countOfUsedClients,
+                                                             final Set<String> rackAwareAssignmentTags,
+                                                             final Map<UUID, ClientState> clientStates,
+                                                             final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                             final Map<String, Set<String>> tagKeyToValues,
+                                                             final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+
+            // Consider the following client setup where we need to distribute 2 standby tasks for each stateful task.
+            //
+            // # Kafka Streams Client 1
+            // client.tag.zone: eu-central-1a
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 2
+            // client.tag.zone: eu-central-1b
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 3
+            // client.tag.zone: eu-central-1c
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 4
+            // client.tag.zone: eu-central-1a
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 5
+            // client.tag.zone: eu-central-1b
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 6
+            // client.tag.zone: eu-central-1c
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // Since we have only two unique `cluster` tag values,
+            // we can only achieve "ideal" distribution on the 1st standby task assignment.
+            // Ideal distribution for the 1st standby task can be achieved because we can assign standby task
+            // to the client located on different cluster and zone compared to an active task.
+            // We can't consider the `cluster` tag for the 2nd standby task assignment because the 1st standby
+            // task would already be assigned on different cluster compared to the active one, which means
+            // we have already used all the available cluster tag values. Taking the `cluster` tag into consideration
+            // for the 2nd standby task assignment would affectively mean excluding all the clients.
+            // Instead, for the 2nd standby task, we can only achieve partial rack awareness based on the `zone` tag.
+            // As we don't consider the `cluster` tag for the 2nd standby task assignment, partial rack awareness
+            // can be satisfied by placing the 2nd standby client on a different `zone` tag compared to active and corresponding standby tasks.
+            // The `zone` on either `cluster` tags are valid candidates for the partial rack awareness, as our goal is to distribute clients on the different `zone` tags.
+
+            // This statement checks if we have used more clients than the all the unique values for the given tag,

Review comment:
       ```suggestion
               // This statement checks if we have used more clients than the number of unique values for the given tag,
   ```




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



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

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


   Hi @showuon, @cadonna 
   
   Thanks for the valuable feedback. I've addressed your comments and pushed the changes. I also resolved conversations feel free to unresolve them if you think I haven't addressed your comments.
   Looking forward hearing your thoughts.
   


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



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

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


   Hi @cadonna small update from my side.
   I cam back from holidays a week ago so will continue working on this PR this week.
   Sorry for the delay.


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



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

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


   @lkokhreidze Sorry for the long silence! Some other tasks got in my way. I plan to review this PR tomorrow.
   


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Solved by storing tuple of tag key and value as map entry instead of just tag value.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Solved by storing tuple of tag key and value as map key instead of just tag value.




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



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

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


   Hi @cadonna 
   Very sorry for disappearing, didn't find time to deal with this PR.
   I've addressed your comments. Please have another look when you got time.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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);
+
+    private final AssignmentConfigs configs;
+
+    ClientTagAwareStandbyTaskAssignor(final AssignmentConfigs configs) {
+        this.configs = configs;
+    }
+
+    @Override
+    public void assignStandbyTasks(final SortedMap<UUID, ClientState> clientStates, final Set<TaskId> statefulTasks) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+        final Map<TaskId, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTasks.forEach(statefulTaskId -> clientStates.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final StandbyTaskDistributor standbyTaskDistributor = new StandbyTaskDistributor(
+            numStandbyReplicas,
+            clientStates,
+            rackAwareAssignmentTags,
+            statefulTasksWithClients
+        );
+
+        statefulTasksWithClients.forEach(standbyTaskDistributor::assignStandbyTasksForActiveTask);
+    }
+
+    @Override
+    public boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        final Map<String, String> sourceClientTags = taskMovementAttempt.sourceClient().clientTags();
+        final Map<String, String> destinationClientTags = taskMovementAttempt.destinationClient().clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static final class StandbyTaskDistributor {
+        private final int numStandbyReplicas;
+        private final SortedMap<UUID, ClientState> clientStates;
+        private final Set<String> rackAwareAssignmentTags;
+        private final Map<String, Set<String>> tagKeyToTagValuesMapping;
+        private final Map<String, Set<UUID>> clientsPerTagValue;
+        private final ConstrainedPrioritySet standbyTaskClientsByTaskLoad;
+
+        private final Set<UUID> usedClients = new HashSet<>();
+        private final Map<TaskId, Integer> tasksToRemainingStandbys;
+
+        StandbyTaskDistributor(final int numStandbyReplicas,
+                               final SortedMap<UUID, ClientState> clientStates,
+                               final Set<String> rackAwareAssignmentTags,
+                               final Map<TaskId, UUID> statefulTasksWithClients) {
+            tagKeyToTagValuesMapping = new HashMap<>();
+            clientsPerTagValue = new HashMap<>();
+            standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+                (client, t) -> !clientStates.get(client).hasAssignedTask(t),
+                client -> clientStates.get(client).assignedTaskLoad()
+            );

Review comment:
       It's already part of the poll constraint of the priority queue. Example:
   
   ```
   final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
       activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
   );
   ```
   I don't think it will be doable with constructor constraint, because we need to update constraint on each poll.




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



[GitHub] [kafka] lkokhreidze edited a comment on pull request #10851: KAFKA-6718 / Rack aware standby task assignor

Posted by GitBox <gi...@apache.org>.
lkokhreidze edited a comment on pull request #10851:
URL: https://github.com/apache/kafka/pull/10851#issuecomment-905514160


   The general thought on the implementation. 
   As of now, we choose concrete `StandbyTaskAssignor` implementation based on passed `AssignmentConfigs` value.
   Instead, an alternative approach would be to have a chained standby task assignment based on multiple implementations. For instance, when have required client tag configuration is present, we can try to assign the standby tasks based on `List.of(new ClientTagAwareStandbyTaskAssignor(), new LeastLoadedClientStandbyTaskAssignor())`. This way, `ClientTagAwareStandbyTaskAssignor` can try to distribute the standby tasks based on tags dimensions. If there are no more client tag dimensions available, and we still have `any(tasksToRemainingStandbys) > 0`, the next implementation in the chain (in this case `DefaultStandbyTaskAssignor`) will be called. With this, `DefaultStandbyTaskAssignor` can default to assigning the remaining standbys to the least loaded clients. The benefit of this approach is that, right now, `ClientTagAwareStandbyTaskAssignor` does both assignments based on available dimensions and fallback to the least loaded if there are no enough tag dimensions. Current implementa
 tion leads to more complex code. While with the approach above, we can clearly separate the implementations without duplication (there's no code duplication, rather "logic" duplication).
   For now, I've chosen to go with the simplest approach - having two independent implementations and selecting an appropriate one based on passed `AssignmentConfigs.` Still, I wanted to share this idea here, just in case. 


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



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

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


   gentle nudge @cadonna @vvcephei @ableegoldman


-- 
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 #10851: KAFKA-6718 / Rack aware standby task assignor

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));
+
+        return true;
+    }
+
+    @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<String, Set<UUID>> tagValueToClients,
+                                                 final Map<String, Set<String>> tagKeyToTagValues) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue);
+                tagValueToClients.computeIfAbsent(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>> tagKeyToTagValues,
+                                                        final Map<String, Set<UUID>> tagValueToClients) {
+        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(
+                usedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagValueToClients,
+                tagKeyToTagValues
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            standbyTaskClientsByTaskLoad.offerAll(clientsOnAlreadyUsedTagDimensions);

Review comment:
       Are you sure, because I cannot confirm the failure of the test on my side?




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.SortedMap;
+import java.util.UUID;
+
+abstract class StandbyTaskAssignor {

Review comment:
       I would prefer to have an interface instead of an abstract class. In the past, it turned out to be cleaner and easier maintainable even if we need to duplicate the `configs` field in the implementations of this interface.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorInitializer.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.internals.assignment.AssignorConfiguration.AssignmentConfigs;
+
+/**
+ * Decides which {@link TaskAssignor} implementation to use for standby task assignment based on {@link AssignmentConfigs}.
+ */
+class StandbyTaskAssignorInitializer {

Review comment:
       I think a factory method as used [here](https://github.com/apache/kafka/blob/f413435585a3ed735ea9d4c551aa4f4f533d6a13/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java#L607) should suffice instead of an entire class.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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 extends StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    ClientTagAwareStandbyTaskAssignor(final AssignmentConfigs configs) {
+        super(configs);
+    }
+
+    @Override
+    public void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                   final TreeMap<UUID, ClientState> clientStates) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+
+        final StandbyTaskDistributor standbyTaskDistributor = new StandbyTaskDistributor(
+            numStandbyReplicas,
+            clientStates,
+            rackAwareAssignmentTags,
+            statefulTasksWithClients
+        );
+
+        statefulTasksWithClients.forEach(standbyTaskDistributor::assignStandbyTasksForActiveTask);
+    }
+
+    @Override
+    public boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        final Map<String, String> sourceClientTags = taskMovementAttempt.sourceClient().clientTags();
+        final Map<String, String> destinationClientTags = taskMovementAttempt.destinationClient().clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static final class StandbyTaskDistributor {

Review comment:
       Method `assignStandbyTasks()` is only invoked once per assignment, as far as I can see. I do not see the need to avoid invalidating caches. Or am I missing somethings?  

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -51,12 +57,12 @@ public boolean assign(final Map<UUID, ClientState> clients,
         final SortedSet<TaskId> statefulTasks = new TreeSet<>(statefulTaskIds);
         final TreeMap<UUID, ClientState> clientStates = new TreeMap<>(clients);
 
-        assignActiveStatefulTasks(clientStates, statefulTasks);
+        final Map<TaskId, UUID> statefulTasksClientMappings = assignActiveStatefulTasks(clientStates, statefulTasks);

Review comment:
       As far as I can see, we would iterate only over the active tasks in both cases. The difference is that in case we have one loop and in the other we have two nested loops. In the nested loop case, the code in the innermost loop is executed the same number of times as in the one loop case. That is, as many times as the number of active tasks.
   In general, I would not change too much code for a performance improvement before we hit a performance issue. You know, as Donald E. Knuth stated "premature optimization is the root of all evil". 🙂 




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



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

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


   Thanks @cadonna for the valuable feedback 🙇 
   I've replied/addressed your comments.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorInitializer.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.internals.assignment.AssignorConfiguration.AssignmentConfigs;
+
+/**
+ * Decides which {@link TaskAssignor} implementation to use for standby task assignment based on {@link AssignmentConfigs}.
+ */
+class StandbyTaskAssignorInitializer {

Review comment:
       Moved factory method in `StandbyTaskAssignor` interface itself.




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



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

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



##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.uuidForInt;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ClientTagAwareStandbyTaskAssignorTest {
+    private static final String ZONE_TAG = "zone";
+    private static final String CLUSTER_TAG = "cluster";
+
+    private static final String ZONE_1 = "zone1";
+    private static final String ZONE_2 = "zone2";
+    private static final String ZONE_3 = "zone3";
+
+    private static final String CLUSTER_1 = "cluster1";
+    private static final String CLUSTER_2 = "cluster2";
+    private static final String CLUSTER_3 = "cluster3";
+
+    private static final UUID UUID_1 = uuidForInt(1);
+    private static final UUID UUID_2 = uuidForInt(2);
+    private static final UUID UUID_3 = uuidForInt(3);
+    private static final UUID UUID_4 = uuidForInt(4);
+    private static final UUID UUID_5 = uuidForInt(5);
+    private static final UUID UUID_6 = uuidForInt(6);
+    private static final UUID UUID_7 = uuidForInt(7);
+    private static final UUID UUID_8 = uuidForInt(8);
+    private static final UUID UUID_9 = uuidForInt(9);
+
+    @Test
+    public void shouldPermitTaskMovementWhenClientTagsMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertTrue(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDeclineTaskMovementWhenClientTagsDoNotMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertFalse(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameZone() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)), TASK_0_2, TASK_1_2)),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameCluster() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_2, TASK_1_2)),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDoThePartialRackAwareness() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_1)), TASK_0_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_3)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_1)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_3)), TASK_1_0))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, CLUSTER_TAG, ZONE_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3),
+                    mkSet(UUID_5, UUID_6)

Review comment:
       `UUID_5` is essentially "ideal" distribution because it has both different cluster and zone compared to an active task.
   However, when we assign 2nd standby, we can only choose client on different `zone`. `cluster` tag is excluded as we don't have enough unique values to exclude the `cluster`. So for the 3rd standby task, both `cluster1` and `cluster2` are valid. That means that clients with `UUID_3` (`cluster1`, `zone3`) and `UUID_6` (`cluster2`, `zone3`) are valid destinations. 
   On the high level, idea is that, if any of the values of the `cluster` tag goes offline, no matter on which `cluster` we distribute the 2nd standby `cluster1` or `cluster2`, we either way will loose two clients at the same time. So from availability perspective it doesn't make difference where the 2nd standby will be assigned. One may argue that it would be better to always distribute to a different tags compared to an active task, but this will complicate algorithm even further, so I guess it's better to keep it simpler in a first iteration.
   
   Hope this makes sense, I will add more info as a comment.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedClientTagDimensions(final UUID usedClient,
+                                                                  final int countOfUsedClients,
+                                                                  final Set<String> rackAwareAssignmentTags,
+                                                                  final Map<UUID, ClientState> clientStates,
+                                                                  final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                                  final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+            final String tagValue = usedClientTagEntry.getValue();
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag.
+            if (allTagValues.size() <= countOfUsedClients) {
+                continue;

Review comment:
       Consider the following example
   
   ```
   # Kafka Streams Client 1
   client.tag.zone: eu-central-1a
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
   
   # Kafka Streams Client 2
   client.tag.zone: eu-central-1b
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 3
   client.tag.zone: eu-central-1c
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
   
   # Kafka Streams Client 4
   client.tag.zone: eu-central-1a
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 5
   client.tag.zone: eu-central-1b
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 6
   client.tag.zone: eu-central-1c
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
   ```
   
   With the above we have following number of unique tag values:
   
   ```
   zone: 3 (eu-central-1a, eu-central-1b, eu-central-1c)
   cluster: 2 (k8s-cluster1, k8s-cluster2)
   ```
   
   Now lets say we have standby replica count as `2` and we want to active task is located on `Client 1`
   
   `usedClients=1` (because of the active task)
   
   ### 1st standby assignment
   
   During 1st standby takes assignment, we will exclude clients on following dimensions:
   
   ```
   zone: eu-central-1a
   cluster: k8s-cluster1
   ```
   
   Used clients will get incremented since we can allocate the client on different `zone` and `cluster`.
   `usedClients=2`
   
   ### 2nd standby assignment
   
   We will have to exclude `zone: eu-central-1a and (eu-central-1b || eu-central-1c)` tag values. We can do that, because
   after we exclude clients on the new tag, we still have clients on the one free tag value we can assign the next standby
   to.
   
   We can't exclude `cluster` because we have already used two clients, and we just have two unique values for
   the `cluster`
   tag, so it's impossible to get the ideal distribution with this configuration and number of instances. So we can only
   achieve partial distribution.
   
   So idea of this check is to ignore tags where we have less unique values than the clients we have already used. If we
   don't have this check, for the 2nd standby task assignment we would have excluded all the clients located
   on `k8s-cluster1` and `k8s-cluster2`, and there wouldn't be any client left to assign the standby task to. We would fall
   back to the least loaded client, BUT there will be no guarantee that least loaded client assignment would honor partial
   rack awareness.
   
   Hope this makes sense.
   




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



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

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


   @lkokhreidze I will try to have a look this week. I would also like to get this PR merged as soon as possible. Since @showuon  has already approved it. If I will not manage to have a look, @showuon can merge it. 


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



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

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


   @lkokhreidze Let me know if you want to address my minor comments in this PR. After that @showuon or I can merge this PR.
   
   Nice work!


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,

Review comment:
       Makes sense. Renamed to `assignStandbyTasksToClientsWithDifferentTags`. Hope this works too.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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,

Review comment:
       Refactored javadocs a bit. Moved some content from class level javadoc to the `assign` method. Hope this works.




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



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

Posted by GitBox <gi...@apache.org>.
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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);

Review comment:
       Would it be possible to decrement the numbers in `tasksToRemainingStandbys` to maintain the remaining standbys to distribute instead of using `pendingStandbyTaskToNumberRemainingStandbys`? 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            if (shouldBalanceLoad(clientStates.values(), standbyTaskClient)) {

Review comment:
       I see what you want to do. However, the capacity is the number of consumers on the Streams client, i.e., the number of stream threads running on the Streams client. With this check, you only allow to assign standby tasks to clients that have less tasks assigned as stream threads running. That is actually rather an unlikely case. Normally, you have more tasks assigned to a Streams client than the number of stream threads running on the client.
   I would keep it simple and ignore the balance for now.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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 extends StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    ClientTagAwareStandbyTaskAssignor(final AssignmentConfigs configs) {
+        super(configs);
+    }
+
+    @Override
+    public void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                   final TreeMap<UUID, ClientState> clientStates) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+
+        final StandbyTaskDistributor standbyTaskDistributor = new StandbyTaskDistributor(
+            numStandbyReplicas,
+            clientStates,
+            rackAwareAssignmentTags,
+            statefulTasksWithClients
+        );
+
+        statefulTasksWithClients.forEach(standbyTaskDistributor::assignStandbyTasksForActiveTask);
+    }
+
+    @Override
+    public boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        final Map<String, String> sourceClientTags = taskMovementAttempt.sourceClient().clientTags();
+        final Map<String, String> destinationClientTags = taskMovementAttempt.destinationClient().clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static final class StandbyTaskDistributor {

Review comment:
       That is correct. Currently, `StandbyTaskAssignor` implementations are created once per `TaskAssignor#assign` method call, and `assignStandbyTasks` is called only once. 
   I just didn't want to assume how and how many times `assignStandbyTasks` is called as I didn't want to leak the implementation details to the caller.
   However, if you feel strongly that it's better to have the implementation in the `ClientTagAwareStandbyTaskAssignor`, I can refactor the code. Since it's internal contract of the assignment, maybe it's okay.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);

Review comment:
       ```suggestion
           final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
               numStandbyReplicas,
               statefulTasksWithClients
           );
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));

Review comment:
       ```suggestion
           statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
               numStandbyReplicas,
               taskId,
               clientId,
               rackAwareAssignmentTags,
               clients,
               tasksToRemainingStandbys
           ));
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {

Review comment:
       Yes, I think that makes sense. In this way you can also directly test the method.
   BTW, you can simply pass `statefulTaskIds`  to this method instead of `statefulTasksWithClients`. The keys in `statefulTasksWithClients` should be the task IDs in `statefulTaskIds` and the values in `statefulTasksWithClients`  are never used. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });
+
+                clientsPerTagValue.compute(tagValue, (key, currentValue) -> {
+                    final Set<UUID> clientIdsForTagValue = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    clientIdsForTagValue.add(clientId);
+                    return clientIdsForTagValue;
+                });

Review comment:
       Isn't this the same as:
   ```
   clientsPerTagValue.computeIfAbsent(tagValue, (ignored) -> new HashSet<>()).add(clientId);
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });
+
+                clientsPerTagValue.compute(tagValue, (key, currentValue) -> {
+                    final Set<UUID> clientIdsForTagValue = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    clientIdsForTagValue.add(clientId);
+                    return clientIdsForTagValue;
+                });
+            });
+        }
+    }
+
+    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>> tagKeyToTagValuesMapping = new HashMap<>();
+        final Map<String, Set<UUID>> clientsPerTagValue = new HashMap<>();
+
+        fillClientsTagStatistics(clientStates, clientsPerTagValue, tagKeyToTagValuesMapping);

Review comment:
       Why is the map from tag key to tag values computed for each active task? The should not change during the assignment and we can just compute it once in `assign()`. Do you agree?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });
+
+                clientsPerTagValue.compute(tagValue, (key, currentValue) -> {
+                    final Set<UUID> clientIdsForTagValue = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    clientIdsForTagValue.add(clientId);
+                    return clientIdsForTagValue;
+                });
+            });
+        }
+    }
+
+    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>> tagKeyToTagValuesMapping = new HashMap<>();
+        final Map<String, Set<UUID>> clientsPerTagValue = new HashMap<>();
+
+        fillClientsTagStatistics(clientStates, clientsPerTagValue, tagKeyToTagValuesMapping);
+
+        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(usedClients,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clientStates,
+                                                                                               clientsPerTagValue,
+                                                                                               tagKeyToTagValuesMapping);

Review comment:
       ```suggestion
               final Set<UUID> clientsOnAlreadyUsedTagDimensions = findClientsOnUsedTagDimensions(
                   usedClients,
                   rackAwareAssignmentTags,
                   clientStates,
                   clientsPerTagValue,
                   tagKeyToTagValuesMapping
               );
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });

Review comment:
       Isn't this the same as:
   ```
   tagKeyToTagValuesMapping.computeIfAbsent(tagKey, (ignored) -> new HashSet<>()).add(tagValue);
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {

Review comment:
       IMO, the code is easier readable if you name the variables consistently like `tagValueToClients` and `tagKeyToTagValues` or `clientsForTagValue` and `tagValuesForTagKey`. I prefer the former because it better visualises the mapping, but that is a matter of taste, I guess.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));
+
+        return true;
+    }
+
+    @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<String, Set<UUID>> tagValueToClients,
+                                                 final Map<String, Set<String>> tagKeyToTagValues) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue);
+                tagValueToClients.computeIfAbsent(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>> tagKeyToTagValues,
+                                                        final Map<String, Set<UUID>> tagValueToClients) {
+        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(
+                usedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagValueToClients,
+                tagKeyToTagValues
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            standbyTaskClientsByTaskLoad.offerAll(clientsOnAlreadyUsedTagDimensions);

Review comment:
       Yeah, sorry. You're right. This is not needed. 




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });
+
+                clientsPerTagValue.compute(tagValue, (key, currentValue) -> {
+                    final Set<UUID> clientIdsForTagValue = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    clientIdsForTagValue.add(clientId);
+                    return clientIdsForTagValue;
+                });
+            });
+        }
+    }
+
+    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>> tagKeyToTagValuesMapping = new HashMap<>();
+        final Map<String, Set<UUID>> clientsPerTagValue = new HashMap<>();
+
+        fillClientsTagStatistics(clientStates, clientsPerTagValue, tagKeyToTagValuesMapping);

Review comment:
       Good catch. I missed it during refactoring, you're correct.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -51,12 +57,12 @@ public boolean assign(final Map<UUID, ClientState> clients,
         final SortedSet<TaskId> statefulTasks = new TreeSet<>(statefulTaskIds);
         final TreeMap<UUID, ClientState> clientStates = new TreeMap<>(clients);
 
-        assignActiveStatefulTasks(clientStates, statefulTasks);
+        final Map<TaskId, UUID> statefulTasksClientMappings = assignActiveStatefulTasks(clientStates, statefulTasks);

Review comment:
       As far as I can see, this map is only used in `ClientTagAwareStandbyTaskAssignor` and it is only used to iterate over pairs (taskId, uuid). That can also be accomplished by iterating over the client states and for each client state  iterate over the assigned active tasks. I do not think that we need to modify the signature of `assignActiveStatefulTasks()`. Or am I missing something? 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.TreeMap;
+import java.util.UUID;
+
+abstract class StandbyTaskAssignor {
+    protected final AssignmentConfigs configs;
+
+    StandbyTaskAssignor(final AssignmentConfigs configs) {
+        this.configs = configs;
+    }
+
+    abstract void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                     final TreeMap<UUID, ClientState> clientStates);

Review comment:
       I think it should be `SortedMap` instead of `TreeMap`. I also saw that we sometimes missed to use `SortedMap` instead of `TreeMap` in some signatures. It needs to be a sorted map because the assignments should be stable otherwise it could happen that we compute different assignments for the same input which could lead to unnecessary state migrations.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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 extends StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    ClientTagAwareStandbyTaskAssignor(final AssignmentConfigs configs) {
+        super(configs);
+    }
+
+    @Override
+    public void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                   final TreeMap<UUID, ClientState> clientStates) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+
+        final StandbyTaskDistributor standbyTaskDistributor = new StandbyTaskDistributor(
+            numStandbyReplicas,
+            clientStates,
+            rackAwareAssignmentTags,
+            statefulTasksWithClients
+        );
+
+        statefulTasksWithClients.forEach(standbyTaskDistributor::assignStandbyTasksForActiveTask);
+    }
+
+    @Override
+    public boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        final Map<String, String> sourceClientTags = taskMovementAttempt.sourceClient().clientTags();
+        final Map<String, String> destinationClientTags = taskMovementAttempt.destinationClient().clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static final class StandbyTaskDistributor {

Review comment:
       Why do we need this internal class? Wouldn't it be simpler to structure the code with methods directly under `ClientTagAwareStandbyTaskAssignor`?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultStandbyTaskAssignor.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import static java.util.stream.Collectors.toMap;
+
+class DefaultStandbyTaskAssignor extends StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(DefaultStandbyTaskAssignor.class);
+
+    public DefaultStandbyTaskAssignor(final AssignorConfiguration.AssignmentConfigs configs) {
+        super(configs);
+    }
+
+    @Override
+    public void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                   final TreeMap<UUID, ClientState> clientStates) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<TaskId> statefulTasks = statefulTasksWithClients.keySet();
+        final Map<TaskId, Integer> tasksToRemainingStandbys = statefulTasks.stream()
+                                                                           .collect(
+                                                                               toMap(
+                                                                                   task -> task,
+                                                                                   t -> numStandbyReplicas
+                                                                               )
+                                                                           );

Review comment:
       ```suggestion
           final Map<TaskId, Integer> tasksToRemainingStandbys =
               statefulTasks.stream().collect(Collectors.toMap(task -> task, t -> numStandbyReplicas));
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -51,12 +57,12 @@ public boolean assign(final Map<UUID, ClientState> clients,
         final SortedSet<TaskId> statefulTasks = new TreeSet<>(statefulTaskIds);
         final TreeMap<UUID, ClientState> clientStates = new TreeMap<>(clients);
 
-        assignActiveStatefulTasks(clientStates, statefulTasks);
+        final Map<TaskId, UUID> statefulTasksClientMappings = assignActiveStatefulTasks(clientStates, statefulTasks);
 
         assignStandbyReplicaTasks(
             clientStates,
-            statefulTasks,
-            configs.numStandbyReplicas
+            statefulTasksClientMappings,
+            configs
         );

Review comment:
       I was wondering whether we can simply standby assignment if `configs.numStandbyReplicas == 0`. Here or as first step in the method body of  `assignStandbyReplicaTasks()`. In this way we can remove `NoopStandbyTaskAssignor`.




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



[GitHub] [kafka] lkokhreidze edited a comment on pull request #10851: KAFKA-6718 / Rack aware standby task assignor

Posted by GitBox <gi...@apache.org>.
lkokhreidze edited a comment on pull request #10851:
URL: https://github.com/apache/kafka/pull/10851#issuecomment-905514160


   The general thought on the implementation. 
   As of now, we choose concrete `StandbyTaskAssignor` implementation based on passed `AssignmentConfigs` value.
   Instead, an alternative approach would be to have a chained standby task assignment based on multiple implementations. For instance, when required client tag configuration is present, we can try to assign the standby tasks based on `List.of(new ClientTagAwareStandbyTaskAssignor(), new LeastLoadedClientStandbyTaskAssignor())`. This way, `ClientTagAwareStandbyTaskAssignor` can try to distribute the standby tasks based on tags dimensions. If there are no more client tag dimensions available, and we still have `any(tasksToRemainingStandbys) > 0`, the next implementation in the chain (in this case `LeastLoadedClientStandbyTaskAssignor`) will be called. With this, `LeastLoadedClientStandbyTaskAssignor` can default to assigning the remaining standbys to the least loaded clients. The benefit of this approach is that, right now, `ClientTagAwareStandbyTaskAssignor` does both assignments based on available dimensions and fallback to the least loaded if there are no enough tag dimensions. Cur
 rent implementation leads to more complex code. While with the approach above, we can clearly separate the implementations without duplication (there's no code duplication, rather "logic" duplication).
   For now, I've chosen to go with the simplest approach - having two independent implementations and selecting an appropriate one based on passed `AssignmentConfigs.` Still, I wanted to share this idea here, just in case. 


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedClientTagDimensions(final UUID usedClient,
+                                                                  final int countOfUsedClients,
+                                                                  final Set<String> rackAwareAssignmentTags,
+                                                                  final Map<UUID, ClientState> clientStates,
+                                                                  final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                                  final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+            final String tagValue = usedClientTagEntry.getValue();
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag.
+            if (allTagValues.size() <= countOfUsedClients) {
+                continue;

Review comment:
       Consider the following example
   
   ```
   # Kafka Streams Client 1
   client.tag.zone: eu-central-1a
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
   
   # Kafka Streams Client 2
   client.tag.zone: eu-central-1b
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 3
   client.tag.zone: eu-central-1c
   client.tag.cluster: k8s-cluster1
   rack.aware.assignment.tags: zone,cluster
   
   # Kafka Streams Client 4
   client.tag.zone: eu-central-1a
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 5
   client.tag.zone: eu-central-1b
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
    
   # Kafka Streams Client 6
   client.tag.zone: eu-central-1c
   client.tag.cluster: k8s-cluster2
   rack.aware.assignment.tags: zone,cluster
   ```
   
   With the above we have following number of unique tag values:
   
   ```
   zone: 3 (eu-central-1a, eu-central-1b, eu-central-1c)
   cluster: 2 (k8s-cluster1, k8s-cluster2)
   ```
   
   Now lets say we have standby replica count as `2` and we want to active task is located on `Client 1`
   
   `usedClients=1` (because of the active task)
   
   ### 1st standby assignment
   
   During 1st standby takes assignment, we will exclude clients on following dimensions:
   
   ```
   zone: eu-central-1a
   cluster: k8s-cluster1
   ```
   
   Used clients will get incremented since we can allocate the client on different `zone` and `cluster`.
   `usedClients=2`
   
   ### 2nd standby assignment
   
   We will have to exclude `zone: eu-central-1a and (eu-central-1b || eu-central-1c)` tag values. We can do that, because after we exclude clients on the new tag, we still have clients on the one free tag value we can assign the next standby to.
   
   We can't exclude `cluster` because we have already used two clients, and we just have two unique values for the `cluster` tag, so it's impossible to get the ideal distribution with this configuration and number of instances. So we can only achieve partial distribution.
   
   So idea of this check is to ignore tags where we have less unique values than the clients we have already used. If we don't have this check, for the 2nd standby task assignment we would have excluded all the clients located
   on `k8s-cluster1` and `k8s-cluster2`, and there wouldn't be any client left to assign the standby task to. We would fall back to the least loaded client, BUT there will be no guarantee that least loaded client assignment would honor partial rack awareness.
   
   Hope this makes sense.
   




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



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

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



##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.uuidForInt;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ClientTagAwareStandbyTaskAssignorTest {
+    private static final String ZONE_TAG = "zone";
+    private static final String CLUSTER_TAG = "cluster";
+
+    private static final String ZONE_1 = "zone1";
+    private static final String ZONE_2 = "zone2";
+    private static final String ZONE_3 = "zone3";
+
+    private static final String CLUSTER_1 = "cluster1";
+    private static final String CLUSTER_2 = "cluster2";
+    private static final String CLUSTER_3 = "cluster3";
+
+    private static final UUID UUID_1 = uuidForInt(1);
+    private static final UUID UUID_2 = uuidForInt(2);
+    private static final UUID UUID_3 = uuidForInt(3);
+    private static final UUID UUID_4 = uuidForInt(4);
+    private static final UUID UUID_5 = uuidForInt(5);
+    private static final UUID UUID_6 = uuidForInt(6);
+    private static final UUID UUID_7 = uuidForInt(7);
+    private static final UUID UUID_8 = uuidForInt(8);
+    private static final UUID UUID_9 = uuidForInt(9);
+
+    @Test
+    public void shouldPermitTaskMovementWhenClientTagsMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertTrue(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDeclineTaskMovementWhenClientTagsDoNotMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertFalse(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameZone() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)), TASK_0_2, TASK_1_2)),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameCluster() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_2, TASK_1_2)),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDoThePartialRackAwareness() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_1)), TASK_0_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_3)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_1)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_3)), TASK_1_0))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, CLUSTER_TAG, ZONE_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3),
+                    mkSet(UUID_5, UUID_6)

Review comment:
       After your explanation of partial rack awareness, I can understand the distribution will be `UUID_5, UUID_6`. But I don't know why it's possible with the result `UUID_5, UUID_3`? 
   I thought after 1st standby task assignment, we'll exclude `CLUSTER_1` and `ZONE_1` tags clients. So the remaining clients will be `UUID_5, UUID_6`. Therefore, the only possible results will be `UUID_5, UUID_6`. Is my understanding correct? Anything I missed?
   Thanks.

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.uuidForInt;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ClientTagAwareStandbyTaskAssignorTest {
+    private static final String ZONE_TAG = "zone";
+    private static final String CLUSTER_TAG = "cluster";
+
+    private static final String ZONE_1 = "zone1";
+    private static final String ZONE_2 = "zone2";
+    private static final String ZONE_3 = "zone3";
+
+    private static final String CLUSTER_1 = "cluster1";
+    private static final String CLUSTER_2 = "cluster2";
+    private static final String CLUSTER_3 = "cluster3";
+
+    private static final UUID UUID_1 = uuidForInt(1);
+    private static final UUID UUID_2 = uuidForInt(2);
+    private static final UUID UUID_3 = uuidForInt(3);
+    private static final UUID UUID_4 = uuidForInt(4);
+    private static final UUID UUID_5 = uuidForInt(5);
+    private static final UUID UUID_6 = uuidForInt(6);
+    private static final UUID UUID_7 = uuidForInt(7);
+    private static final UUID UUID_8 = uuidForInt(8);
+    private static final UUID UUID_9 = uuidForInt(9);
+
+    @Test
+    public void shouldPermitTaskMovementWhenClientTagsMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertTrue(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDeclineTaskMovementWhenClientTagsDoNotMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertFalse(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameZone() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)), TASK_0_2, TASK_1_2)),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameCluster() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_2, TASK_1_2)),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDoThePartialRackAwareness() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_1)), TASK_0_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_3)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_1)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_3)), TASK_1_0))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, CLUSTER_TAG, ZONE_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3),
+                    mkSet(UUID_5, UUID_6)

Review comment:
       And I think we should add some comments here, to have a simple explanation like we I did above to explain why we have these results. Thanks.




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



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

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


   Hi @cadonna ,
   Will you have time to look at this PR again?


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }
+
+        // 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<String, Set<UUID>> tagValueToClients,
+                                                 final Map<String, Set<String>> tagKeyToTagValues) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue);
+                tagValueToClients.computeIfAbsent(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>> tagKeyToTagValues,
+                                                        final Map<String, Set<UUID>> tagValueToClients) {
+        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(
+                usedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagValueToClients,
+                tagKeyToTagValues
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            numRemainingStandbys--;
+        }
+
+        if (numRemainingStandbys > 0) {
+            log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                     "There is not enough available capacity. You should " +
+                     "increase the number of application instances " +
+                     "on different client tag dimensions " +
+                     "to maintain the requested number of standby replicas. " +
+                     "Rack awareness is configured with [{}] tags.",
+                     numRemainingStandbys, numStandbyReplicas, activeTaskId,
+                     clientStates.get(activeTaskClient).clientTags(), rackAwareAssignmentTags);
+        }
+    }
+
+    private static Set<UUID> findClientsOnUsedTagDimensions(final Set<UUID> usedClients,
+                                                            final Set<String> rackAwareAssignmentTags,
+                                                            final Map<UUID, ClientState> clientStates,
+                                                            final Map<String, Set<UUID>> clientsPerTagValue,
+                                                            final Map<String, Set<String>> tagKeyToTagValuesMapping) {

Review comment:
       Could you please use `tagValueToClients` and `tagKeyToValues` here as in the rest of the class?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Currently the code iterates over the active tasks and assigns all standby tasks for each active task. If the standby tasks cannot all be assigned, we might end up with all standby tasks assigned for some active task but none for others. What do you think about to assign one standby task for all active task and then assign the second standby task for all active task, and so on. In this way, it is more likely that at all active tasks have at least one standby task assigned.
   I am aware that the default standby assignor has the same drawback.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Thanks, good catch.




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



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

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


   The general thought on the implementation. 
   As of now, we choose concrete `StandbyTaskAssignor` implementation based on passed `AssignmentConfigs` value.
   Instead, an alternative approach would be to have a chained standby task assignment based on multiple implementations. For instance, when have required client tag configuration is present, we can try to assign the standby tasks based on `List.of(new ClientTagAwareStandbyTaskAssignor(), new DefaultStandbyTaskAssignor())`. This way, `ClientTagAwareStandbyTaskAssignor` can try to distribute the standby tasks based on tags dimensions. If there are no more client tag dimensions available, and we still have `any(tasksToRemainingStandbys) > 0`, the next implementation in the chain (in this case `DefaultStandbyTaskAssignor`) will be called. With this, `DefaultStandbyTaskAssignor` can default to assigning the remaining standbys to the least loaded clients. The benefit of this approach is that, right now, `ClientTagAwareStandbyTaskAssignor` does both assignments based on available dimensions and fallback to the least loaded if there are no enough tag dimensions. Current implementation leads
  to more complex code. While with the approach above, we can clearly separate the implementations without duplication (there's no code duplication, rather "logic" duplication).
   For now, I've chosen to go with the simplest approach - having two independent implementations and selecting an appropriate one based on passed `AssignmentConfigs.` Still, I wanted to share this idea here, just in case. 


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



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

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


   @lkokhreidze , thanks for the PR. I'll take a look next week. Thanks.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
##########
@@ -59,29 +60,31 @@
 
     private int capacity;
 
-    public ClientState() {
-        this(0);
+    public ClientState(final Map<String, String> clientTags) {
+        this(0, clientTags);
     }
 
-    ClientState(final int capacity) {
+    ClientState(final int capacity, final Map<String, String> clientTags) {

Review comment:
       This is an internal class, so I think it won't be changed/used many times. I think change to my previous suggestion is better. Thanks.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );

Review comment:
       Good call. Improved the code in a latest commit.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)

Review comment:
       Thanks! Renamed to `clientOnNotUsedTagDimensions` to be consistent with the rest of the codebase. Since we refer to client UUIDs as just `client` in the codebase. Hope this works.




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



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

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


   Thanks @showuon for the valuable feedback.
   I've addressed your comments, please have a look when you got time.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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(

Review comment:
       Updated tests for the `StandbyTaskAssignmentUtils#pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks` and also added separate test for the `ClientTagAwareStandbyTaskAssignor`.
   
   For the `ClientTagAwareStandbyTaskAssignor` I decided to make few things package private to be able to test this logic. As otherwise, there was no easy way to test if `tasksToRemainingStandbys` was getting updated properly. Hope this is okay. 




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
+
+        fillClientsOnAlreadyUsedTagEntries(
+            activeTaskClient,
+            countOfUsedClients,
+            rackAwareAssignmentTags,
+            clientStates,
+            tagEntryToClients,
+            tagKeyToValues,
+            tagEntryToUsedClients
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(uuid, tagEntryToUsedClients)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+            numRemainingStandbys--;
+
+            if (numRemainingStandbys == 0) {
+                break;
+            }
+
+            fillClientsOnAlreadyUsedTagEntries(
+                clientOnUnusedTagDimensions,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues,
+                tagEntryToUsedClients
+            );
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static boolean isClientUsedOnAnyOfTheTagEntries(final UUID client,
+                                                            final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        return tagEntryToUsedClients.values().stream().anyMatch(usedClients -> usedClients.contains(client));
+    }
+
+    private static void fillClientsOnAlreadyUsedTagEntries(final UUID usedClient,
+                                                           final int countOfUsedClients,
+                                                           final Set<String> rackAwareAssignmentTags,
+                                                           final Map<UUID, ClientState> clientStates,
+                                                           final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                           final Map<String, Set<String>> tagKeyToValues,
+                                                           final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag, because it'll exclude all the clients.
+            // Instead, we need to discard all the clients that were marked as "used" on that tag key values.
+            // Please check ClientTagAwareStandbyTaskAssignorTest#shouldDoThePartialRackAwareness test for more info.

Review comment:
       Pushed the changes. I added detailed explanation with an example. Also tests have the similar example. Hopefully this change makes logic more 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.

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

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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Makes sense @cadonna.
   Should I update default standby task assignor, or prefer to leave it out of scope of this PR?




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



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

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


   Hi @cadonna,
   I've address your comments with the latest commit.
   Please have a look when you got time.
   Thanks. 


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



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

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


   Hi @cadonna 
   Mind having another look? Hoping to finalise this PR as soon as possible :)
   Thanks


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.TreeMap;
+import java.util.UUID;
+
+abstract class StandbyTaskAssignor {
+    protected final AssignmentConfigs configs;
+
+    StandbyTaskAssignor(final AssignmentConfigs configs) {
+        this.configs = configs;
+    }
+
+    abstract void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                     final TreeMap<UUID, ClientState> clientStates);

Review comment:
       Can you please clarify why `TreeMap<UUID, ClientState>` is used here? Would `Map<UUID, ClientState>` suffice?




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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 extends StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    ClientTagAwareStandbyTaskAssignor(final AssignmentConfigs configs) {
+        super(configs);
+    }
+
+    @Override
+    public void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                   final TreeMap<UUID, ClientState> clientStates) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+
+        final StandbyTaskDistributor standbyTaskDistributor = new StandbyTaskDistributor(
+            numStandbyReplicas,
+            clientStates,
+            rackAwareAssignmentTags,
+            statefulTasksWithClients
+        );
+
+        statefulTasksWithClients.forEach(standbyTaskDistributor::assignStandbyTasksForActiveTask);
+    }
+
+    @Override
+    public boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        final Map<String, String> sourceClientTags = taskMovementAttempt.sourceClient().clientTags();
+        final Map<String, String> destinationClientTags = taskMovementAttempt.destinationClient().clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static final class StandbyTaskDistributor {

Review comment:
       Addressed with 9841d25




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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 extends StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    ClientTagAwareStandbyTaskAssignor(final AssignmentConfigs configs) {
+        super(configs);
+    }
+
+    @Override
+    public void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                   final TreeMap<UUID, ClientState> clientStates) {
+        final int numStandbyReplicas = configs.numStandbyReplicas;
+        final Set<String> rackAwareAssignmentTags = new HashSet<>(configs.rackAwareAssignmentTags);
+
+        final StandbyTaskDistributor standbyTaskDistributor = new StandbyTaskDistributor(
+            numStandbyReplicas,
+            clientStates,
+            rackAwareAssignmentTags,
+            statefulTasksWithClients
+        );
+
+        statefulTasksWithClients.forEach(standbyTaskDistributor::assignStandbyTasksForActiveTask);
+    }
+
+    @Override
+    public boolean isValidTaskMovement(final TaskMovementAttempt taskMovementAttempt) {
+        final Map<String, String> sourceClientTags = taskMovementAttempt.sourceClient().clientTags();
+        final Map<String, String> destinationClientTags = taskMovementAttempt.destinationClient().clientTags();
+
+        for (final Entry<String, String> sourceClientTagEntry : sourceClientTags.entrySet()) {
+            if (!sourceClientTagEntry.getValue().equals(destinationClientTags.get(sourceClientTagEntry.getKey()))) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static final class StandbyTaskDistributor {

Review comment:
       Thanks for the feedback Bruno.
   
   I reasoned that, since internal states like `clientsPerTagValue`, `standbyTaskClientsByTaskLoad`, etc., have to be allocated per invocation of `assignStandbyTasks` method, it felt easier and more readable to create one single internal object rather than invalidating local caches in `ClientTagAwareStandbyTaskAssignor`.




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



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

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


   Hi @cadonna,
   I've addressed/replied to your comments. Thanks for the feedback.
   FYI - I'll be offline from next week for 2 weeks.


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



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

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


   Hi @cadonna,
   Small update on my side - wasn't able to find time to work on this PR this week.
   Will try to prioritise this for the next week.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedClientTagDimensions(final UUID usedClient,
+                                                                  final int countOfUsedClients,
+                                                                  final Set<String> rackAwareAssignmentTags,
+                                                                  final Map<UUID, ClientState> clientStates,
+                                                                  final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                                  final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+            final String tagValue = usedClientTagEntry.getValue();
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag.
+            if (allTagValues.size() <= countOfUsedClients) {
+                continue;

Review comment:
       I added `shouldDoThePartialRackAwareness` test to verify this behaviour.




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



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

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


   Hi @showuon Thanks for the feedback.
   I've addressed your comments.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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,

Review comment:
       Refactored javadocs a bit. Moved some content from class level javadoc to the `assign` method.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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(

Review comment:
       Updated tests for the `StandbyTaskAssignmentUtils#pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks` and also added separate test for the `ClientTagAwareStandbyTaskAssignor`.
   
   For the `ClientTagAwareStandbyTaskAssignor` I decided to make few things package private to be able to test this logic. As otherwise, there was no easy way to test if `tasksToRemainingStandbys` was getting updates properly. Hope this is okay. 




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -195,17 +195,18 @@ private static int assignStandbyTasksToClientsWithDifferentTags(final Constraine
 
         final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
 
-        fillClientsOnAlreadyUsedTagEntries(
-            activeTaskClient,
-            countOfUsedClients,
-            rackAwareAssignmentTags,
-            clientStates,
-            tagEntryToClients,
-            tagKeyToValues,
-            tagEntryToUsedClients
-        );
+        UUID lastUsedclient = activeTaskClient;

Review comment:
       nit: lastUsedclient -> lastUsedClient

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
+
+        UUID lastUsedclient = activeTaskClient;
+        do {
+            fillClientsOnAlreadyUsedTagEntries(

Review comment:
       Since we will remove tags, I think we can rename to `updateClieintsOnAlreadyUsedTagEntries`. WDYT?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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(

Review comment:
       Nice catch! And maybe we should add a test to address this.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
+
+        fillClientsOnAlreadyUsedTagEntries(
+            activeTaskClient,
+            countOfUsedClients,
+            rackAwareAssignmentTags,
+            clientStates,
+            tagEntryToClients,
+            tagKeyToValues,
+            tagEntryToUsedClients
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(uuid, tagEntryToUsedClients)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+            numRemainingStandbys--;
+
+            if (numRemainingStandbys == 0) {
+                break;
+            }
+
+            fillClientsOnAlreadyUsedTagEntries(
+                clientOnUnusedTagDimensions,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues,
+                tagEntryToUsedClients
+            );
+        }

Review comment:
       Good suggestion!




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -252,10 +253,53 @@ private static void fillClientsOnAlreadyUsedTagEntries(final UUID usedClient,
 
             final Set<String> allTagValues = tagKeyToValues.get(tagKey);
 
-            // If we have used more clients than all the tag's unique values,
-            // we can't filter out clients located on that tag, because it'll exclude all the clients.
-            // Instead, we need to discard all the clients that were marked as "used" on that tag key values.
-            // Please check ClientTagAwareStandbyTaskAssignorTest#shouldDoThePartialRackAwareness test for more info.
+            // Consider the following client setup where we need to distribute 2 standby tasks for each stateful task.
+            //
+            // # Kafka Streams Client 1
+            // client.tag.zone: eu-central-1a
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 2
+            // client.tag.zone: eu-central-1b
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 3
+            // client.tag.zone: eu-central-1c
+            // client.tag.cluster: k8s-cluster1
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 4
+            // client.tag.zone: eu-central-1a
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 5
+            // client.tag.zone: eu-central-1b
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // # Kafka Streams Client 6
+            // client.tag.zone: eu-central-1c
+            // client.tag.cluster: k8s-cluster2
+            // rack.aware.assignment.tags: zone,cluster
+            //
+            // Since we have only two unique `cluster` tag values,
+            // we can only achieve "ideal" distribution on the 1st standby task assignment.
+            // Ideal distribution for the 1st standby task can be achieved because we can assign standby task
+            // to the client located on different cluster and zone compared to an active task.
+            // We can't consider the `cluster` tag for the 2nd standby task assignment because the 1st standby
+            // task would already be assigned on different cluster compared to the active one, which means
+            // we have already used all the available cluster tag values. Taking the `cluster` tag into consideration
+            // for the  2nd standby task assignment would affectively mean excluding all the clients.

Review comment:
       nit: additional space between `the` and `2nd`




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



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

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


   @cadonna no worries and thank you.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultStandbyTaskAssignor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+class DefaultStandbyTaskAssignor implements StandbyTaskAssignor {

Review comment:
       I know there was no any java doc for the default least load assignor. But do you think we can add some comments to it, just like in `ClientTagAwareStandbyTaskAssignor`? I believe not everyone knows default assignor algorithm is least loaded assignor.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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,

Review comment:
       Could we add some java doc to this assign to briefly mention about the algorithm used in the assignor? Thanks.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
##########
@@ -59,29 +60,31 @@
 
     private int capacity;
 
-    public ClientState() {
-        this(0);
+    public ClientState(final Map<String, String> clientTags) {
+        this(0, clientTags);
     }
 
-    ClientState(final int capacity) {
+    ClientState(final int capacity, final Map<String, String> clientTags) {

Review comment:
       I'm wondering could we keep the original constructor and pass empty map into the new one? So that we don't have to make changes to the old caller. That is:
   
   ```
   public ClientState() {
       this(0, Collections.emptyMap());
   }
   
   ClientState(final int capacity) {
       this(capacity, Collections.emptyMap());
   }
   ```
   WDYT?




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



[GitHub] [kafka] lkokhreidze edited a comment on pull request #10851: KAFKA-6718 / Rack aware standby task assignor

Posted by GitBox <gi...@apache.org>.
lkokhreidze edited a comment on pull request #10851:
URL: https://github.com/apache/kafka/pull/10851#issuecomment-905514160


   The general thought on the implementation. 
   As of now, we choose concrete `StandbyTaskAssignor` implementation based on passed `AssignmentConfigs` value.
   Instead, an alternative approach would be to have a chained standby task assignment based on multiple implementations. For instance, when required client tag configuration is present, we can try to assign the standby tasks based on `List.of(new ClientTagAwareStandbyTaskAssignor(), new LeastLoadedClientStandbyTaskAssignor())`. This way, `ClientTagAwareStandbyTaskAssignor` can try to distribute the standby tasks based on tags dimensions. If there are no more client tag dimensions available, and we still have `any(tasksToRemainingStandbys) > 0`, the next implementation in the chain (in this case `LeastLoadedClientStandbyTaskAssignor`) will be called. With this, `LeastLoadedClientStandbyTaskAssignor` can default to assigning the remaining standbys to the least loaded clients. Right now, `ClientTagAwareStandbyTaskAssignor` does both assignments based on available dimensions and fallback to the least loaded if there are no enough tag dimensions. Current implementation leads to more comp
 lex code. While with the approach above, we can clearly separate the implementations without duplication (there's no code duplication, rather "logic" duplication).
   For now, I've chosen to go with the simplest approach - having two independent implementations and selecting an appropriate one based on passed `AssignmentConfigs.` Still, I wanted to share this idea here, just in case. 


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



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

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


   Hi @cadonna , thanks for the review.
   I agree with what you said and made a note to myself to address your comments in the follow-up PRs. So if it's okay, I think we can merge this one.
   
   Thank you!


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



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

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


   Failed tests are unrelated and also failed in `trunk` build.
   ```
   Build / ARM / org.apache.kafka.streams.integration.NamedTopologyIntegrationTest.shouldRemoveOneNamedTopologyWhileAnotherContinuesProcessing
   Build / JDK 11 and Scala 2.13 / kafka.server.DynamicBrokerReconfigurationTest.testThreadPoolResize()
   Build / JDK 11 and Scala 2.13 / kafka.server.DynamicBrokerReconfigurationTest.testThreadPoolResize()
   Build / JDK 11 and Scala 2.13 / org.apache.kafka.streams.integration.NamedTopologyIntegrationTest.shouldRemoveOneNamedTopologyWhileAnotherContinuesProcessing
   Build / JDK 17 and Scala 2.13 / kafka.server.DynamicBrokerReconfigurationTest.testThreadPoolResize()
   Build / JDK 17 and Scala 2.13 / kafka.server.DynamicBrokerReconfigurationTest.testThreadPoolResize()
   Build / JDK 17 and Scala 2.13 / org.apache.kafka.streams.integration.NamedTopologyIntegrationTest.shouldRemoveOneNamedTopologyWhileAnotherContinuesProcessing
   Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.integration.RebalanceSourceConnectorsIntegrationTest.testDeleteConnector
   Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.integration.RebalanceSourceConnectorsIntegrationTest.testDeleteConnector
   Build / JDK 8 and Scala 2.12 / org.apache.kafka.streams.integration.NamedTopologyIntegrationTest.shouldRemoveOneNamedTopologyWhileAnotherContinuesProcessing
   ```


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



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

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



##########
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:
       Fixed with e3aff39.
   Now we only create `clientsOnAlreadyUsedTagDimensions` once and populate it for the each standby task assignment instead of re-creating it.




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



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

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


   Hi @cadonna thanks for the feedback.
   I'll address your comments shortly.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Thanks, good catch.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Makes sense @cadonna.
   Should I update default standby task assignor, or prefer to leave it out of scope of this PR?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Solved by storing tuple of tag key and value as map entry instead of just tag value.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Solved by storing tuple of tag key and value as map key instead of just tag value.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Also wondering if it's better to do this as a separate task altogether. Since, as you've mentioned, it's the same behaviour as with default standby task assignor.
   But if you feel it's better to do it in current PR, happy to do so.




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



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

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


   @lkokhreidze I am really sorry that I haven't found the time to look at your updated PR. I will put it on my list for next week. 


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));
+
+        return true;
+    }
+
+    @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<String, Set<UUID>> tagValueToClients,
+                                                 final Map<String, Set<String>> tagKeyToTagValues) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue);
+                tagValueToClients.computeIfAbsent(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>> tagKeyToTagValues,
+                                                        final Map<String, Set<UUID>> tagValueToClients) {
+        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(
+                usedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagValueToClients,
+                tagKeyToTagValues
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            standbyTaskClientsByTaskLoad.offerAll(clientsOnAlreadyUsedTagDimensions);

Review comment:
       We need to add those clients back because we need to fallback to least loaded clients if we don't have any more valid clients on other dimensions.
   For instance, without this `shouldDistributeTasksOnLeastLoadedClientsWhenThereAreNoEnoughUniqueTagDimensions` test will fail.
   




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));
+
+        return true;
+    }
+
+    @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<String, Set<UUID>> tagValueToClients,
+                                                 final Map<String, Set<String>> tagKeyToTagValues) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue);
+                tagValueToClients.computeIfAbsent(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>> tagKeyToTagValues,
+                                                        final Map<String, Set<UUID>> tagValueToClients) {
+        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(
+                usedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagValueToClients,
+                tagKeyToTagValues
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            standbyTaskClientsByTaskLoad.offerAll(clientsOnAlreadyUsedTagDimensions);

Review comment:
       We need to add those clients back because we need to fallback to least loaded clients if we don't have any more valid clients on other dimensions.
   For instance, without this `shouldDistributeTasksOnLeastLoadedClientsWhenThereAreNoEnoughUniqueTagDimensions` test will fail.
   




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



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

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


   Hi @cadonna 
   Thank you for valuable feedback. 
   I've addressed your comments. Please have a look once you got time.
   


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
##########
@@ -64,6 +66,7 @@ public ClientState() {
         previousStandbyTasks.taskIds(new TreeSet<>());
         previousActiveTasks.taskIds(new TreeSet<>());
 
+        clientTags = new HashMap<>();

Review comment:
       I guess it would be better to move the assignment to the field declaration to avoid duplication among constructors.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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(

Review comment:
       Yes, a test is absolutely necessary!




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



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

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


   Thanks @cadonna, appreciate it.
   @showuon please also note that next PR, the protocol change, is also ready to be reviewed. https://github.com/apache/kafka/pull/10802
   
   Whenever it's possible, please have a look at that too.
   


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });

Review comment:
       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.

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

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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.TreeMap;
+import java.util.UUID;
+
+abstract class StandbyTaskAssignor {
+    protected final AssignmentConfigs configs;
+
+    StandbyTaskAssignor(final AssignmentConfigs configs) {
+        this.configs = configs;
+    }
+
+    abstract void assignStandbyTasks(final Map<TaskId, UUID> statefulTasksWithClients,
+                                     final TreeMap<UUID, ClientState> clientStates);

Review comment:
       Yes, the `TreeMap` in the signatures has been already there before this PR. Thank you for fixing this!




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));
+
+        return true;

Review comment:
       Pushed the changes.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Let's assume you have two clients. `clientX` has tags `keyA:value1` and `keyB:value2` and `clientY` has tags `keyA:value2` and `keyB:value3`. Notice that `keyA` and `keyB` share `value2`. With your code, we will end up with a `tagValueToClients` map that looks like this:
   ```
   value1 -> {clientX}
   value2 -> {clientX, clientY}
   value3 -> {clientY}
   ```  
   Now, let's assume that an active task is assigned to `clientX`. It would be totally fine if we assign the standby task to `clientY` since each tag key of both clients do not share a value. However, your algorithm does not allow it, because on line 198 it also adds `clientY` to the clients that are not allowed to get the standby. The reason is that `tagValueToClients` only looks for clients that contain value `value2` and not for clients that contain it as a value for `keyA`.
   
   The following test fails because of this:
   ```
       @Test
       public void shouldTestBrunosTheory() {
           final Map<UUID, ClientState> clientStates = mkMap(
               mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0)),
               mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, CLUSTER_1), mkEntry(CLUSTER_TAG, CLUSTER_3))))
           );
   
           final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
           final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(1, ZONE_TAG, CLUSTER_TAG);
   
           new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
   
           assertTrue(
               standbyClientsHonorRackAwareness(
                   TASK_0_0,
                   clientStates,
                   asList(
                       mkSet(UUID_2)
                   )
               )
           );
       }
   ```




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



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

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


   Thanks for the feedback @cadonna , I've pushed the new changes.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {

Review comment:
       Done




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -51,12 +57,12 @@ public boolean assign(final Map<UUID, ClientState> clients,
         final SortedSet<TaskId> statefulTasks = new TreeSet<>(statefulTaskIds);
         final TreeMap<UUID, ClientState> clientStates = new TreeMap<>(clients);
 
-        assignActiveStatefulTasks(clientStates, statefulTasks);
+        final Map<TaskId, UUID> statefulTasksClientMappings = assignActiveStatefulTasks(clientStates, statefulTasks);

Review comment:
       Fair enough, done.




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



[GitHub] [kafka] lkokhreidze edited a comment on pull request #10851: KAFKA-6718 / Rack aware standby task assignor

Posted by GitBox <gi...@apache.org>.
lkokhreidze edited a comment on pull request #10851:
URL: https://github.com/apache/kafka/pull/10851#issuecomment-1017438641


   Hi @cadonna small update from my side.
   I came back from holidays a week ago so will continue working on this PR this week.
   Sorry for the delay.


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



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

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


   The latest commit (be3dcff4dc463fd8d23998537e36f852b99ec083) has a few changes.
   
   1. There's explicit fallback to fallback to `DefaultStandbyTaskAssignor` logic if rack aware standby task assignment fails to assign standby tasks.
   2. Rack aware standby task assignment logic takes into account client capacity when assigning the standby task. If client has reached the capacity, algorithm won't assign standby task to it. When that happens, we will fallback to the least loaded clients. I believe this is better approach, as we will avoid overloading the clients. There will be a warning log, to inform the user.
   
   Looking forward hearing your thoughts, @cadonna 


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



[GitHub] [kafka] lkokhreidze edited a comment on pull request #10851: KAFKA-6718 / Rack aware standby task assignor

Posted by GitBox <gi...@apache.org>.
lkokhreidze edited a comment on pull request #10851:
URL: https://github.com/apache/kafka/pull/10851#issuecomment-1018535614


   The latest commit (be3dcff4dc463fd8d23998537e36f852b99ec083) has a few changes.
   
   1. There's explicit fallback to fallback to `DefaultStandbyTaskAssignor` logic if rack aware standby task assignment fails to assign standby tasks.
   2. Rack aware standby task assignment logic takes into account client capacity when assigning the standby task. If client has reached the capacity, algorithm won't assign standby task to it. When that happens, we will fallback to the least loaded clients. I believe this is better approach, as we will avoid overloading the clients. There will be a warning log, to inform the user.
   
   **EDIT:** Instead of just checking if capacity is reached on the client, we also now check if load can be balanced [here](https://github.com/apache/kafka/pull/10851/commits/a010d9093889f8e985022e2f727254d087f54891#diff-9988f8ab5112be9826a66ecc4b2df28eea61e8ceb15ed19a1b94b747a6fb48c8R213).
   
   Looking forward hearing your thoughts, @cadonna 


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.

Review comment:
       nit: the algorithm will fall back to the least-loaded clients without **taking** rack awareness constraints into consideration.
   

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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<>();

Review comment:
       I checked the use of `tagKeyToValues`. It is only used for total value count of each key. Is that right? If so, could we just store the `Map<String, Integer> tagKeyToValueCount` only?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedClientTagDimensions(final UUID usedClient,
+                                                                  final int countOfUsedClients,
+                                                                  final Set<String> rackAwareAssignmentTags,
+                                                                  final Map<UUID, ClientState> clientStates,
+                                                                  final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                                  final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+            final String tagValue = usedClientTagEntry.getValue();
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag.
+            if (allTagValues.size() <= countOfUsedClients) {
+                continue;

Review comment:
       Sorry, I didn't understand the reason why we can't filter out clients located on that tag when `allTagValues.size() <= countOfUsedClients`. Could you help explain to me? Thanks.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);

Review comment:
       Would it be possible to decrement the numbers in `tasksToRemainingStandbys` to maintain the remaining standbys to distribute instead of using `pendingStandbyTaskToNumberRemainingStandbys`? 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            if (shouldBalanceLoad(clientStates.values(), standbyTaskClient)) {

Review comment:
       I see what you want to do. However, the capacity is the number of consumers on the Streams client, i.e., the number of stream threads running on the Streams client. With this check, you only allow to assign standby tasks to clients that have less tasks assigned as stream threads running. That is actually rather an unlikely case. Normally, you have more tasks assigned to a Streams client than the number of stream threads running on the client.
   I would keep it simple and ignore the balance for now.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
##########
@@ -59,29 +60,31 @@
 
     private int capacity;
 
-    public ClientState() {
-        this(0);
+    public ClientState(final Map<String, String> clientTags) {
+        this(0, clientTags);
     }
 
-    ClientState(final int capacity) {
+    ClientState(final int capacity, final Map<String, String> clientTags) {

Review comment:
       Thanks for the feedback. No objections from my side. The reason why I avoided that, was because to make sure that client tags are always passed, to emphasise that it's mandatory parameter when constructing the `ClientState` object. Please note that we have made `ClientState#clientTags` immutable. So there're no setters for the client tags.
   But if you feel like it's better to default to empty map, happy to change it.
   Will wait for your response on this.




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



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

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



##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.uuidForInt;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ClientTagAwareStandbyTaskAssignorTest {
+    private static final String ZONE_TAG = "zone";
+    private static final String CLUSTER_TAG = "cluster";
+
+    private static final String ZONE_1 = "zone1";
+    private static final String ZONE_2 = "zone2";
+    private static final String ZONE_3 = "zone3";
+
+    private static final String CLUSTER_1 = "cluster1";
+    private static final String CLUSTER_2 = "cluster2";
+    private static final String CLUSTER_3 = "cluster3";
+
+    private static final UUID UUID_1 = uuidForInt(1);
+    private static final UUID UUID_2 = uuidForInt(2);
+    private static final UUID UUID_3 = uuidForInt(3);
+    private static final UUID UUID_4 = uuidForInt(4);
+    private static final UUID UUID_5 = uuidForInt(5);
+    private static final UUID UUID_6 = uuidForInt(6);
+    private static final UUID UUID_7 = uuidForInt(7);
+    private static final UUID UUID_8 = uuidForInt(8);
+    private static final UUID UUID_9 = uuidForInt(9);
+
+    @Test
+    public void shouldPermitTaskMovementWhenClientTagsMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertTrue(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDeclineTaskMovementWhenClientTagsDoNotMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertFalse(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameZone() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)), TASK_0_2, TASK_1_2)),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );

Review comment:
       I can understand what you tried to assert here. But I think we should also assert that the standby tasks count in each client is as what we expected, because under current verification, we only focus on the tasks distributed with rack awareness. However, there is still possibility that standby tasks don't distribute evenly, right?
   
   The following tests should also update. Thanks.

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.uuidForInt;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ClientTagAwareStandbyTaskAssignorTest {
+    private static final String ZONE_TAG = "zone";
+    private static final String CLUSTER_TAG = "cluster";
+
+    private static final String ZONE_1 = "zone1";
+    private static final String ZONE_2 = "zone2";
+    private static final String ZONE_3 = "zone3";
+
+    private static final String CLUSTER_1 = "cluster1";
+    private static final String CLUSTER_2 = "cluster2";
+    private static final String CLUSTER_3 = "cluster3";
+
+    private static final UUID UUID_1 = uuidForInt(1);
+    private static final UUID UUID_2 = uuidForInt(2);
+    private static final UUID UUID_3 = uuidForInt(3);
+    private static final UUID UUID_4 = uuidForInt(4);
+    private static final UUID UUID_5 = uuidForInt(5);
+    private static final UUID UUID_6 = uuidForInt(6);
+    private static final UUID UUID_7 = uuidForInt(7);
+    private static final UUID UUID_8 = uuidForInt(8);
+    private static final UUID UUID_9 = uuidForInt(9);
+
+    @Test
+    public void shouldPermitTaskMovementWhenClientTagsMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertTrue(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDeclineTaskMovementWhenClientTagsDoNotMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertFalse(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameZone() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)), TASK_0_2, TASK_1_2)),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameCluster() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_2, TASK_1_2)),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDoThePartialRackAwareness() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_1)), TASK_0_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_3)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_1)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_3)), TASK_1_0))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, CLUSTER_TAG, ZONE_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+

Review comment:
       We can add comments here like:
   
   // We need 2 standby tasks (+ 1 active task) to distribute, but we only have 2 cluster tag, so we will won't exclude all clients when 2nd standby tasks assignment, and will try to distribute the 2nd standby tasks with taking partial rack tag into consideration.
   
   WDYT?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedClientTagDimensions(final UUID usedClient,
+                                                                  final int countOfUsedClients,
+                                                                  final Set<String> rackAwareAssignmentTags,
+                                                                  final Map<UUID, ClientState> clientStates,
+                                                                  final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                                  final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+            final String tagValue = usedClientTagEntry.getValue();
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag.
+            if (allTagValues.size() <= countOfUsedClients) {
+                continue;

Review comment:
       I'm thinking, we can make it much clear by adding comments, though I know this is hard to explain in simple words. How do you think we add comments like this:
   
               // If we have used more clients than all the tag's unique values,
               // we can't filter out clients located on that tag, because it'll excluded all the clients. 
               // Please check ClientTagAwareStandbyTaskAssignorTest#shouldDoThePartialRackAwareness test for more info.
   
   And we can make more description in `shouldDoThePartialRackAwareness` test.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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<>();

Review comment:
       A ha, you're right! We only need the distinct count values. No need to refactor it then. Thanks for the explanation. 

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.uuidForInt;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ClientTagAwareStandbyTaskAssignorTest {
+    private static final String ZONE_TAG = "zone";
+    private static final String CLUSTER_TAG = "cluster";
+
+    private static final String ZONE_1 = "zone1";
+    private static final String ZONE_2 = "zone2";
+    private static final String ZONE_3 = "zone3";
+
+    private static final String CLUSTER_1 = "cluster1";
+    private static final String CLUSTER_2 = "cluster2";
+    private static final String CLUSTER_3 = "cluster3";
+
+    private static final UUID UUID_1 = uuidForInt(1);
+    private static final UUID UUID_2 = uuidForInt(2);
+    private static final UUID UUID_3 = uuidForInt(3);
+    private static final UUID UUID_4 = uuidForInt(4);
+    private static final UUID UUID_5 = uuidForInt(5);
+    private static final UUID UUID_6 = uuidForInt(6);
+    private static final UUID UUID_7 = uuidForInt(7);
+    private static final UUID UUID_8 = uuidForInt(8);
+    private static final UUID UUID_9 = uuidForInt(9);
+
+    @Test
+    public void shouldPermitTaskMovementWhenClientTagsMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertTrue(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDeclineTaskMovementWhenClientTagsDoNotMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertFalse(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameZone() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)), TASK_0_2, TASK_1_2)),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_9), mkSet(UUID_3, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3), mkSet(UUID_2, UUID_6)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameCluster() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_2, TASK_1_2)),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_9, UUID_5), mkSet(UUID_6, UUID_8)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_1,
+                clientStates,
+                asList(
+                    mkSet(UUID_4, UUID_9), mkSet(UUID_6, UUID_7)
+                )
+            )
+        );
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_2,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_7), mkSet(UUID_4, UUID_8)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDoThePartialRackAwareness() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_1)), TASK_0_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_1), mkEntry(ZONE_TAG, ZONE_3)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_1)))),
+            mkEntry(UUID_5, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(1, mkMap(mkEntry(CLUSTER_TAG, CLUSTER_2), mkEntry(ZONE_TAG, ZONE_3)), TASK_1_0))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, CLUSTER_TAG, ZONE_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_0_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_5, UUID_3),
+                    mkSet(UUID_5, UUID_6)
+                )
+            )
+        );
+        assertTrue(
+            standbyClientsHonorRackAwareness(
+                TASK_1_0,
+                clientStates,
+                asList(
+                    mkSet(UUID_2, UUID_4),
+                    mkSet(UUID_2, UUID_1)
+                )
+            )
+        );
+    }
+
+    @Test
+    public void shouldDistributeClientsOnDifferentZoneTagsEvenWhenClientsReachedCapacity() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0)),
+            mkEntry(UUID_4, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_1_0)),
+
+            mkEntry(UUID_2, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_1)),
+            mkEntry(UUID_5, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_1_1)),
+
+            mkEntry(UUID_3, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_2)),
+            mkEntry(UUID_6, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_1_2))

Review comment:
       nit: could we be consistent with other tests that make the UUID_seq in order? It makes me a little confused when reading this test. That is, 
   ```
              mkEntry(UUID_1, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0)),
               mkEntry(UUID_2, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_1_0)),
   
               mkEntry(UUID_3, createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_1)),
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedClientTagDimensions(final UUID usedClient,
+                                                                  final int countOfUsedClients,
+                                                                  final Set<String> rackAwareAssignmentTags,
+                                                                  final Map<UUID, ClientState> clientStates,
+                                                                  final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                                  final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+            final String tagValue = usedClientTagEntry.getValue();
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag.
+            if (allTagValues.size() <= countOfUsedClients) {
+                continue;

Review comment:
       In 1st standby assignment:
   > Used clients will get incremented since we can allocate the client on different zone and cluster.
   usedClients=2
   
   I think this used client should be 5 or 6, right? But I got your idea.
   Thanks for the explanation. Makes sense to me. 
   




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,296 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );

Review comment:
       Will get it done asap.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedClientTagDimensions(final UUID usedClient,
+                                                                  final int countOfUsedClients,
+                                                                  final Set<String> rackAwareAssignmentTags,
+                                                                  final Map<UUID, ClientState> clientStates,
+                                                                  final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                                  final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+            final String tagValue = usedClientTagEntry.getValue();
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag.
+            if (allTagValues.size() <= countOfUsedClients) {
+                continue;

Review comment:
       Added more explanation in `shouldDoThePartialRackAwareness` test.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,296 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;

Review comment:
       nit: we can directly break the while when `numRemainingStandbys == 0`, so that we don't need to run the redundant `findClientsOnUsedClientTagDimensions` in the last run. Ex:
   
   ```java
   countOfUsedClients++;
   numRemainingStandbys--;
   if (numRemainingStandbys == 0) {
     break;
   }
   clientsOnAlreadyUsedTagDimensions.addAll(
                   findClientsOnUsedClientTagDimensions(
                       clientOnUnusedTagDimensions,
                       countOfUsedClients,
                       rackAwareAssignmentTags,
                       clientStates,
                       tagEntryToClients,
                       tagKeyToValues
                   )
               );
   
   ```
   

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignorTest.java
##########
@@ -0,0 +1,536 @@
+/*
+ * 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.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_0;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_1;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_2;
+import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.uuidForInt;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ClientTagAwareStandbyTaskAssignorTest {
+    private static final String ZONE_TAG = "zone";
+    private static final String CLUSTER_TAG = "cluster";
+
+    private static final String ZONE_1 = "zone1";
+    private static final String ZONE_2 = "zone2";
+    private static final String ZONE_3 = "zone3";
+
+    private static final String CLUSTER_1 = "cluster1";
+    private static final String CLUSTER_2 = "cluster2";
+    private static final String CLUSTER_3 = "cluster3";
+
+    private static final UUID UUID_1 = uuidForInt(1);
+    private static final UUID UUID_2 = uuidForInt(2);
+    private static final UUID UUID_3 = uuidForInt(3);
+    private static final UUID UUID_4 = uuidForInt(4);
+    private static final UUID UUID_5 = uuidForInt(5);
+    private static final UUID UUID_6 = uuidForInt(6);
+    private static final UUID UUID_7 = uuidForInt(7);
+    private static final UUID UUID_8 = uuidForInt(8);
+    private static final UUID UUID_9 = uuidForInt(9);
+
+    @Test
+    public void shouldPermitTaskMovementWhenClientTagsMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertTrue(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDeclineTaskMovementWhenClientTagsDoNotMatch() {
+        final ClientTagAwareStandbyTaskAssignor standbyTaskAssignor = new ClientTagAwareStandbyTaskAssignor();
+        final ClientState source = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+        final ClientState destination = createClientStateWithCapacity(1, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)));
+
+        assertFalse(standbyTaskAssignor.isAllowedTaskMovement(source, destination));
+    }
+
+    @Test
+    public void shouldDistributeStandbyTasksWhenActiveTasksAreLocatedOnSameZone() {
+        final Map<UUID, ClientState> clientStates = mkMap(
+            mkEntry(UUID_1, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_1)), TASK_0_0, TASK_1_0)),
+            mkEntry(UUID_2, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+            mkEntry(UUID_3, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_1)))),
+
+            mkEntry(UUID_4, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_2)), TASK_0_1, TASK_1_1)),
+            mkEntry(UUID_5, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+            mkEntry(UUID_6, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_2)))),
+
+            mkEntry(UUID_7, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_1), mkEntry(CLUSTER_TAG, CLUSTER_3)), TASK_0_2, TASK_1_2)),
+            mkEntry(UUID_8, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_2), mkEntry(CLUSTER_TAG, CLUSTER_3)))),
+            mkEntry(UUID_9, createClientStateWithCapacity(2, mkMap(mkEntry(ZONE_TAG, ZONE_3), mkEntry(CLUSTER_TAG, CLUSTER_3))))
+        );
+
+        final Set<TaskId> allActiveTasks = findAllActiveTasks(clientStates);
+        final AssignmentConfigs assignmentConfigs = newAssignmentConfigs(2, ZONE_TAG, CLUSTER_TAG);
+
+        new ClientTagAwareStandbyTaskAssignor().assign(clientStates, allActiveTasks, allActiveTasks, assignmentConfigs);
+
+        assertTrue(clientStates.values().stream().allMatch(ClientState::reachedCapacity));
+
+        Stream.of(UUID_1, UUID_4, UUID_7).forEach(client -> assertStandbyTaskCountForClientEqualsTo(clientStates, client, 0));
+        Stream.of(UUID_2, UUID_3, UUID_5, UUID_6, UUID_8, UUID_9).forEach(client -> assertStandbyTaskCountForClientEqualsTo(clientStates, client, 2));
+        assertTotalNumberOfStandbyTasksEqualsTo(clientStates, 12);

Review comment:
       Nice tests!

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,296 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );

Review comment:
       Thanks for the explanation for partial rack awareness assignment. I think that algorithm makes sense. However, I don't think the implementation matches what you described. You said in the `shouldDoThePartialRackAwareness` test, in 2nd standby assignment for task_0_0, we will only consider `zone`, but in current implementation, we will also consider `cluster`. That is, when entering the `while (numRemainingStandbys > 0) {` loop, the `clientsOnAlreadyUsedTagDimensions` already excluded the `cluster_1` and `zone_1`. And in the 1st standby assignment, `UUID_5` will be chosen, we'll exclude `zone_2` only, and not exclude `cluster_2`. So , the only client left is `UUID_6`. That's the current design, isn't it? I don't see where we only consider `zone` in 2nd assignment. Could you help elaborate more? Thank you.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -51,12 +57,12 @@ public boolean assign(final Map<UUID, ClientState> clients,
         final SortedSet<TaskId> statefulTasks = new TreeSet<>(statefulTaskIds);
         final TreeMap<UUID, ClientState> clientStates = new TreeMap<>(clients);
 
-        assignActiveStatefulTasks(clientStates, statefulTasks);
+        final Map<TaskId, UUID> statefulTasksClientMappings = assignActiveStatefulTasks(clientStates, statefulTasks);
 
         assignStandbyReplicaTasks(
             clientStates,
-            statefulTasks,
-            configs.numStandbyReplicas
+            statefulTasksClientMappings,
+            configs
         );

Review comment:
       Sure! done.
   Personal preference. Having all the strategies of standby task assignment implementations in a single class makes unit testing a bit easier. But I do agree that removing one extra class is indeed good idea.




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



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

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


   Hi @cadonna is it possible to continue pushing this PR forward? I'm back from my holidays.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Sorry, can you elaborate more on this?
   Currently, when deciding the distribution, algorithm takes into account both, tag key, as well as tag value. So it will treat `key1: value2` and `key2: value2` as different dimensions. Do you think it's something that has to be addressed?




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



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

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


   Welcome back @lkokhreidze! Sorry for the silence around this PR. I got side-tracked by other tasks. I will try to review this PR this week.


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



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

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


   Hi @cadonna 
   Very sorry for disappearing, didn't find time to deal with this PR.
   I've addressed your comments. Please have another look when you got time.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Also wondering if it's better to do this as a separate task altogether. Since, as you've mentioned, it's the same behaviour as with default standby task assignor.
   But if you feel it's better to do it in current PR, happy to do so.




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



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

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


   @lkokhreidze I will try to review this Friday. Sorry for the delay but I was sick for the last two weeks.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            if (shouldBalanceLoad(clientStates.values(), standbyTaskClient)) {
+                final Map<String, String> standbyClientTags = standbyTaskClient.clientTags();
+                log.warn("Can't assign {} of {} standby task for task [{}]. " +
+                         "There is not enough capacity on client(s) with {} tag dimensions. " +
+                         "To have a proper distribution of standby tasks across different tag dimensions, " +
+                         "increase the number of application instances with [{}] tag dimensions.",
+                         numRemainingStandbys, numStandbyReplicas, activeTaskId, standbyClientTags, standbyClientTags);
+                break;
+            }
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedTagDimensions(final Set<UUID> usedClients,
+                                                            final Set<String> rackAwareAssignmentTags,
+                                                            final Map<UUID, ClientState> clientStates,
+                                                            final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                            final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        for (final UUID usedClientId : usedClients) {
+            final Map<String, String> usedClientTags = clientStates.get(usedClientId).clientTags();
+
+            for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+                final String tagKey = usedClientTagEntry.getKey();
+
+                if (!rackAwareAssignmentTags.contains(tagKey)) {
+                    continue;

Review comment:
       I think having a warn log is a good call. We can add validation rules (if necessary) when doing last part of this KIP - updating streams configuration.




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



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

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


   Hi @cadonna ,
   Sorry for the ping. Any chance we could review PR this week?
   Thanks


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
+
+        fillClientsOnAlreadyUsedTagEntries(
+            activeTaskClient,
+            countOfUsedClients,
+            rackAwareAssignmentTags,
+            clientStates,
+            tagEntryToClients,
+            tagKeyToValues,
+            tagEntryToUsedClients
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(uuid, tagEntryToUsedClients)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+            numRemainingStandbys--;
+
+            if (numRemainingStandbys == 0) {
+                break;
+            }
+
+            fillClientsOnAlreadyUsedTagEntries(
+                clientOnUnusedTagDimensions,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues,
+                tagEntryToUsedClients
+            );
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static boolean isClientUsedOnAnyOfTheTagEntries(final UUID client,
+                                                            final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        return tagEntryToUsedClients.values().stream().anyMatch(usedClients -> usedClients.contains(client));
+    }
+
+    private static void fillClientsOnAlreadyUsedTagEntries(final UUID usedClient,
+                                                           final int countOfUsedClients,
+                                                           final Set<String> rackAwareAssignmentTags,
+                                                           final Map<UUID, ClientState> clientStates,
+                                                           final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                           final Map<String, Set<String>> tagKeyToValues,
+                                                           final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag, because it'll exclude all the clients.
+            // Instead, we need to discard all the clients that were marked as "used" on that tag key values.
+            // Please check ClientTagAwareStandbyTaskAssignorTest#shouldDoThePartialRackAwareness test for more info.

Review comment:
       Thanks Bruno, I'll add the example to the comment.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,296 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );

Review comment:
       I think we should re-work the `assignStandbyTasksToClientsWithDifferentTags` method to match what you described. That makes more sense. Thanks.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,296 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );

Review comment:
       @showuon Ah, you're absolutely right! I'm very sorry for the confusion. It's been a while and got lost myself. I will update comments to reflect this. Do you think it makes sense to leave the implementation as is, or we should re-work it based on what I described before? Either is fine with me.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,296 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>(
+            findClientsOnUsedClientTagDimensions(
+                activeTaskClient,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues
+            )
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedClientTagDimensions(
+                    clientOnUnusedTagDimensions,
+                    countOfUsedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );

Review comment:
       Updated implementation in https://github.com/apache/kafka/pull/10851/commits/5d1543eab455e5bc361800ee0976d4c7f88fdfe6




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



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

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


   🥳 


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.Collections;
+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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        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(
+                        standbyTaskClientsByTaskLoad,
+                        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 ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                        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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            if (standbyTaskClient.reachedCapacity()) {
+                log.warn("Capacity was reached when assigning standby task [{}] to client with tags [{}]. " +

Review comment:
       Hi @cadonna
   Would appreciate your feedback on this.
   As of now, algorithm ignores a case when client has reached capacity and it will try to assign the standby task to it as long as it satisfies the rack awareness. There's a even test for it `shouldDistributeClientsOnDifferentZoneTagsEvenWhenClientsReachedCapacity`. For me it makes sense that rack awareness, if configured, takes precedence in this case. Added log to inform the user, just want to make sure if you think this is a valid approach.
   It is not a lot of work to take capacity into account, so we can redo algorithm if you think that makes more sense.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.Collections;
+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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        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(
+                        standbyTaskClientsByTaskLoad,
+                        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 ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                        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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            if (standbyTaskClient.reachedCapacity()) {
+                log.warn("Capacity was reached when assigning standby task [{}] to client with tags [{}]. " +

Review comment:
       No longer relevant, I reworked things a bit, check out this comment https://github.com/apache/kafka/pull/10851#issuecomment-1018535614




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



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

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



##########
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:
       Fixed with e3aff39c7687a358cc8672accd5bbf6a27193a04.
   Algorithm will try to achieve partial rack awareness as there are different `cluster` tag dimensions. 
   




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));
+
+        return true;

Review comment:
       Although we never use the returned value from a standby task assignor, I would return `false` since a standby task assignment will never require a follow-up probing rebalance.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));

Review comment:
       Map `statefulTasksWithClients` is only used to iterate over its entries. I think it would be better to use the following nested loops and remove `statefulTasksWithClients`:
   
   ```suggestion
           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,
                           tagKeyToTagValues,
                           tagValueToClients
                       );
                   }
               }
           }
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(
+            numStandbyReplicas,
+            taskId,
+            clientId,
+            rackAwareAssignmentTags,
+            clients,
+            tasksToRemainingStandbys,
+            tagKeyToTagValues,
+            tagValueToClients
+        ));
+
+        return true;
+    }
+
+    @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<String, Set<UUID>> tagValueToClients,
+                                                 final Map<String, Set<String>> tagKeyToTagValues) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue);
+                tagValueToClients.computeIfAbsent(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>> tagKeyToTagValues,
+                                                        final Map<String, Set<UUID>> tagValueToClients) {
+        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(
+                usedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagValueToClients,
+                tagKeyToTagValues
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            standbyTaskClientsByTaskLoad.offerAll(clientsOnAlreadyUsedTagDimensions);

Review comment:
       I do not understand why you re-add `clientsOnAlreadyUsedTagDimensions`. Those clients were not modified and not polled for sure due to line 140. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       I think this map does not work for distinct tag keys that have overlapping tag values. For example, `key1` contains one of `{value1, value2}` and `key2` contains one of `{value2, value3}`.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java
##########
@@ -51,12 +57,12 @@ public boolean assign(final Map<UUID, ClientState> clients,
         final SortedSet<TaskId> statefulTasks = new TreeSet<>(statefulTaskIds);
         final TreeMap<UUID, ClientState> clientStates = new TreeMap<>(clients);
 
-        assignActiveStatefulTasks(clientStates, statefulTasks);
+        final Map<TaskId, UUID> statefulTasksClientMappings = assignActiveStatefulTasks(clientStates, statefulTasks);

Review comment:
       I tried to avoid unnecessary iterations. With that we would have to do separate iteration in the `ClientTagAwareStandbyTaskAssignor`, which felt redundant, since `assignActiveStatefulTasks` can return necessary mapping since it has to iterate over client states either way.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
##########
@@ -64,6 +66,7 @@ public ClientState() {
         previousStandbyTasks.taskIds(new TreeSet<>());
         previousActiveTasks.taskIds(new TreeSet<>());
 
+        clientTags = new HashMap<>();

Review comment:
       Good point about client tags being constant. Added it as constructor parameter.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignor.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.SortedMap;
+import java.util.UUID;
+
+abstract class StandbyTaskAssignor {

Review comment:
       Done




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {

Review comment:
       This method is needed in `ClientTagAwareStandbyTaskAssignor` and `DefaultStandbyTaskAssignor`. Was thinking to create `StandbyTaskAssignmentUtils` and extract this logic in there. Wdyt @cadonna ?




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * 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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(numStandbyReplicas,
+                                                                                              statefulTasksWithClients);
+
+        statefulTasksWithClients.forEach((taskId, clientId) -> assignStandbyTasksForActiveTask(numStandbyReplicas,
+                                                                                               taskId,
+                                                                                               clientId,
+                                                                                               rackAwareAssignmentTags,
+                                                                                               clients,
+                                                                                               tasksToRemainingStandbys));
+
+        return true;
+    }
+
+    @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 Map<TaskId, Integer> computeTasksToRemainingStandbys(final int numStandbyReplicas, final Map<TaskId, UUID> statefulTasksWithClients) {
+        return statefulTasksWithClients.keySet().stream().collect(toMap(Function.identity(), t -> numStandbyReplicas));
+    }
+
+    private static void fillClientsTagStatistics(final Map<UUID, ClientState> clientStates,
+                                                 final Map<String, Set<UUID>> clientsPerTagValue,
+                                                 final Map<String, Set<String>> tagKeyToTagValuesMapping) {
+        for (final Entry<UUID, ClientState> clientStateEntry : clientStates.entrySet()) {
+            final UUID clientId = clientStateEntry.getKey();
+            final ClientState clientState = clientStateEntry.getValue();
+
+            clientState.clientTags().forEach((tagKey, tagValue) -> {
+                tagKeyToTagValuesMapping.compute(tagKey, (key, currentValue) -> {
+                    final Set<String> tagValuesForKey = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    tagValuesForKey.add(tagValue);
+                    return tagValuesForKey;
+                });
+
+                clientsPerTagValue.compute(tagValue, (key, currentValue) -> {
+                    final Set<UUID> clientIdsForTagValue = Optional.ofNullable(currentValue).orElse(new HashSet<>());
+                    clientIdsForTagValue.add(clientId);
+                    return clientIdsForTagValue;
+                });
+            });
+        }
+    }
+
+    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>> tagKeyToTagValuesMapping = new HashMap<>();
+        final Map<String, Set<UUID>> clientsPerTagValue = new HashMap<>();
+
+        fillClientsTagStatistics(clientStates, clientsPerTagValue, tagKeyToTagValuesMapping);

Review comment:
       Why is the map from tag key to tag values computed for each active task? They should not change during the assignment and we can just compute it once in `assign()`. Do you agree?




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



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

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


   @lkokhreidze FYI: I will be offline the next two weeks. I am sorry that I haven't had time to review this PR this week.


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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Thanks, good catch.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Makes sense @cadonna.
   Should I update default standby task assignor, or prefer to leave it out of scope of this PR?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Solved by storing tuple of tag key and value as map entry instead of just tag value.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Solved by storing tuple of tag key and value as map key instead of just tag value.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Also wondering if it's better to do this as a separate task altogether. Since, as you've mentioned, it's the same behaviour as with default standby task assignor.
   But if you feel it's better to do it in current PR, happy to do so.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Thanks, good catch.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Makes sense @cadonna.
   Should I update default standby task assignor, or prefer to leave it out of scope of this PR?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Solved by storing tuple of tag key and value as map entry instead of just tag value.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.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, UUID> statefulTasksWithClients = new HashMap<>();
+
+        statefulTaskIds.forEach(statefulTaskId -> clients.forEach((uuid, clientState) -> {
+            if (clientState.activeTasks().contains(statefulTaskId)) {
+                statefulTasksWithClients.put(statefulTaskId, uuid);
+            }
+        }));
+
+        final Map<TaskId, Integer> tasksToRemainingStandbys = computeTasksToRemainingStandbys(
+            numStandbyReplicas,
+            allTaskIds
+        );
+
+        final Map<String, Set<String>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();

Review comment:
       Solved by storing tuple of tag key and value as map key instead of just tag value.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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>> tagKeyToTagValues = new HashMap<>();
+        final Map<String, Set<UUID>> tagValueToClients = new HashMap<>();
+
+        fillClientsTagStatistics(clients, tagValueToClients, tagKeyToTagValues);
+
+        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,
+                        tagKeyToTagValues,
+                        tagValueToClients
+                    );
+                }
+            }
+        }

Review comment:
       Also wondering if it's better to do this as a separate task altogether. Since, as you've mentioned, it's the same behaviour as with default standby task assignor.
   But if you feel it's better to do it in current PR, happy to do so.




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



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

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


   Thanks for the feedback @cadonna 
   I will review and address your comments this week.


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



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

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


   Hi @cadonna !
   Thanks for the feedback. I will address your comments this week.


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



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

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



##########
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:
       I reworked things a bit, check out this comment https://github.com/apache/kafka/pull/10851#issuecomment-1018535614




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.TaskAssignmentUtils.computeTasksToRemainingStandbys;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+import static org.apache.kafka.streams.processor.internals.assignment.TaskAssignmentUtils.shouldBalanceLoad;
+
+/**
+ * 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's not enough capacity on the clients
+ * with 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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys = new HashMap<>();
+        final Map<TaskId, UUID> pendingStandbyTaskToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksForActiveTask(
+                        standbyTaskClientsByTaskLoad,
+                        numStandbyReplicas,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTaskToNumberRemainingStandbys.put(statefulTaskId, numberOfRemainingStandbys);
+                        pendingStandbyTaskToClientId.put(statefulTaskId, clientId);
+                    }
+                }
+            }
+        }
+
+        if (!pendingStandbyTaskToNumberRemainingStandbys.isEmpty()) {
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          pendingStandbyTaskToNumberRemainingStandbys,
+                                                          pendingStandbyTaskToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksForActiveTask(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                       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 Set<UUID> usedClients = new HashSet<>();
+
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        usedClients.add(activeTaskClient);
+
+        final Set<UUID> clientsOnAlreadyUsedTagDimensions = new HashSet<>();
+
+        while (numRemainingStandbys > 0) {
+            clientsOnAlreadyUsedTagDimensions.addAll(
+                findClientsOnUsedTagDimensions(
+                    usedClients,
+                    rackAwareAssignmentTags,
+                    clientStates,
+                    tagEntryToClients,
+                    tagKeyToValues
+                )
+            );
+
+            final UUID polledClient = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !clientsOnAlreadyUsedTagDimensions.contains(uuid)
+            );
+
+            if (polledClient == null) {
+                break;
+            }
+
+            final ClientState standbyTaskClient = clientStates.get(polledClient);
+
+            if (shouldBalanceLoad(clientStates.values(), standbyTaskClient)) {
+                final Map<String, String> standbyClientTags = standbyTaskClient.clientTags();
+                log.warn("Can't assign {} of {} standby task for task [{}]. " +
+                         "There is not enough capacity on client(s) with {} tag dimensions. " +
+                         "To have a proper distribution of standby tasks across different tag dimensions, " +
+                         "increase the number of application instances with [{}] tag dimensions.",
+                         numRemainingStandbys, numStandbyReplicas, activeTaskId, standbyClientTags, standbyClientTags);
+                break;
+            }
+
+            standbyTaskClient.assignStandby(activeTaskId);
+
+            usedClients.add(polledClient);
+
+            numRemainingStandbys--;
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static Set<UUID> findClientsOnUsedTagDimensions(final Set<UUID> usedClients,
+                                                            final Set<String> rackAwareAssignmentTags,
+                                                            final Map<UUID, ClientState> clientStates,
+                                                            final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                            final Map<String, Set<String>> tagKeyToValues) {
+        final Set<UUID> filteredClients = new HashSet<>();
+
+        for (final UUID usedClientId : usedClients) {
+            final Map<String, String> usedClientTags = clientStates.get(usedClientId).clientTags();
+
+            for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+                final String tagKey = usedClientTagEntry.getKey();
+
+                if (!rackAwareAssignmentTags.contains(tagKey)) {
+                    continue;

Review comment:
       OK




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.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.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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal distribution is impossible, the algorithm will fall back to the least-loaded clients without considering rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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<>();

Review comment:
       Can do, but also `Set` makes it easier to handle duplicate values as we are looking for distinct count values here. Not sure if refactoring is worth it though.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
##########
@@ -59,29 +60,31 @@
 
     private int capacity;
 
-    public ClientState() {
-        this(0);
+    public ClientState(final Map<String, String> clientTags) {
+        this(0, clientTags);
     }
 
-    ClientState(final int capacity) {
+    ClientState(final int capacity, final Map<String, String> clientTags) {

Review comment:
       Thanks for the feedback. No objections from my side. The reason why I avoided that was to make sure that client tags are always passed. To emphasise that it's mandatory parameter when constructing the `ClientState` object. Please note that we have made `ClientState#clientTags` immutable; so there're no setters for the client tags.
   But if you feel like it's better to default to empty map, happy to change it.
   Will wait for your response on this.




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



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

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
+
+        fillClientsOnAlreadyUsedTagEntries(
+            activeTaskClient,
+            countOfUsedClients,
+            rackAwareAssignmentTags,
+            clientStates,
+            tagEntryToClients,
+            tagKeyToValues,
+            tagEntryToUsedClients
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(uuid, tagEntryToUsedClients)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+            numRemainingStandbys--;
+
+            if (numRemainingStandbys == 0) {
+                break;
+            }
+
+            fillClientsOnAlreadyUsedTagEntries(
+                clientOnUnusedTagDimensions,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues,
+                tagEntryToUsedClients
+            );
+        }
+
+        return numRemainingStandbys;
+    }
+
+    private static boolean isClientUsedOnAnyOfTheTagEntries(final UUID client,
+                                                            final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        return tagEntryToUsedClients.values().stream().anyMatch(usedClients -> usedClients.contains(client));
+    }
+
+    private static void fillClientsOnAlreadyUsedTagEntries(final UUID usedClient,
+                                                           final int countOfUsedClients,
+                                                           final Set<String> rackAwareAssignmentTags,
+                                                           final Map<UUID, ClientState> clientStates,
+                                                           final Map<TagEntry, Set<UUID>> tagEntryToClients,
+                                                           final Map<String, Set<String>> tagKeyToValues,
+                                                           final Map<TagEntry, Set<UUID>> tagEntryToUsedClients) {
+        final Map<String, String> usedClientTags = clientStates.get(usedClient).clientTags();
+
+        for (final Entry<String, String> usedClientTagEntry : usedClientTags.entrySet()) {
+            final String tagKey = usedClientTagEntry.getKey();
+
+            if (!rackAwareAssignmentTags.contains(tagKey)) {
+                log.warn("Client tag with key [{}] will be ignored when computing rack aware standby " +
+                         "task assignment because it is not part of the configured rack awareness [{}].",
+                         tagKey, rackAwareAssignmentTags);
+                continue;
+            }
+
+            final Set<String> allTagValues = tagKeyToValues.get(tagKey);
+
+            // If we have used more clients than all the tag's unique values,
+            // we can't filter out clients located on that tag, because it'll exclude all the clients.
+            // Instead, we need to discard all the clients that were marked as "used" on that tag key values.
+            // Please check ClientTagAwareStandbyTaskAssignorTest#shouldDoThePartialRackAwareness test for more info.

Review comment:
       That is quite challenging to understand. After reading it a couple of times I understood that if we've used a number of clients that is equal to or greater than the number of unique values of the tag, we cannot guarantee that each standby is on a different tag value than the active and other standbys. So the rack-awareness becomes partial. Is that correct?
   Could you reformulate it, so that it states that the rack-awareness guarantee does not hold anymore.
   And why "more clients than all tag's unique values"? When the number of used clients is equal to the unique tag values, we are already in the partial rack-awareness situation, right?
   Maybe you should give here an example as in the mentioned test. I find referencing the test is a bit cumbersome, because if the test gets renamed this comment becomes useless.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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);
+
+        final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = new ConstrainedPrioritySet(
+            (client, t) -> !clients.get(client).hasAssignedTask(t),
+            client -> clients.get(client).assignedTaskLoad()
+        );
+
+        final Map<TaskId, UUID> pendingStandbyTasksToClientId = new HashMap<>();
+
+        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)) {
+                    final int numberOfRemainingStandbys = assignStandbyTasksToClientsWithDifferentTags(
+                        standbyTaskClientsByTaskLoad,
+                        statefulTaskId,
+                        clientId,
+                        rackAwareAssignmentTags,
+                        clients,
+                        tasksToRemainingStandbys,
+                        tagKeyToValues,
+                        tagEntryToClients
+                    );
+
+                    if (numberOfRemainingStandbys > 0) {
+                        pendingStandbyTasksToClientId.put(statefulTaskId, clientId);
+                    } else {
+                        tasksToRemainingStandbys.remove(statefulTaskId);
+                    }
+                }
+            }
+        }
+
+        if (!tasksToRemainingStandbys.isEmpty()) {
+            log.debug("Rack aware standby task assignment was not able to assign all standby tasks. " +
+                      "tasksToRemainingStandbys=[{}], pendingStandbyTasksToClientId=[{}]. " +
+                      "Will distribute the remaining standby tasks to least loaded clients.",
+                      tasksToRemainingStandbys, pendingStandbyTasksToClientId);
+
+            assignPendingStandbyTasksToLeastLoadedClients(clients,
+                                                          numStandbyReplicas,
+                                                          rackAwareAssignmentTags,
+                                                          standbyTaskClientsByTaskLoad,
+                                                          tasksToRemainingStandbys,
+                                                          pendingStandbyTasksToClientId);
+        }
+
+        // returning false, because standby task assignment will never require a follow-up probing rebalance.
+        return false;
+    }
+
+    private static void assignPendingStandbyTasksToLeastLoadedClients(final Map<UUID, ClientState> clients,
+                                                                      final int numStandbyReplicas,
+                                                                      final Set<String> rackAwareAssignmentTags,
+                                                                      final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                      final Map<TaskId, Integer> pendingStandbyTaskToNumberRemainingStandbys,
+                                                                      final Map<TaskId, UUID> pendingStandbyTaskToClientId) {
+        // We need to re offer all the clients to find the least loaded ones
+        standbyTaskClientsByTaskLoad.offerAll(clients.keySet());
+
+        for (final Entry<TaskId, Integer> pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) {
+            final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey();
+            final UUID clientId = pendingStandbyTaskToClientId.get(activeTaskId);
+
+            final int numberOfRemainingStandbys = pollClientAndMaybeAssignRemainingStandbyTasks(
+                clients,
+                pendingStandbyTaskToNumberRemainingStandbys,
+                standbyTaskClientsByTaskLoad,
+                activeTaskId
+            );
+
+            if (numberOfRemainingStandbys > 0) {
+                log.warn("Unable to assign {} of {} standby tasks for task [{}] with client tags [{}]. " +
+                         "There is not enough available capacity. You should " +
+                         "increase the number of application instances " +
+                         "on different client tag dimensions " +
+                         "to maintain the requested number of standby replicas. " +
+                         "Rack awareness is configured with [{}] tags.",
+                         numberOfRemainingStandbys, numStandbyReplicas, activeTaskId,
+                         clients.get(clientId).clientTags(), rackAwareAssignmentTags);
+            }
+        }
+    }
+
+    @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 int assignStandbyTasksToClientsWithDifferentTags(final ConstrainedPrioritySet standbyTaskClientsByTaskLoad,
+                                                                    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) {
+        standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet());
+
+        // We set countOfUsedClients as 1 because client where active task is located has to be considered as used.
+        int countOfUsedClients = 1;
+        int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId);
+
+        final Map<TagEntry, Set<UUID>> tagEntryToUsedClients = new HashMap<>();
+
+        fillClientsOnAlreadyUsedTagEntries(
+            activeTaskClient,
+            countOfUsedClients,
+            rackAwareAssignmentTags,
+            clientStates,
+            tagEntryToClients,
+            tagKeyToValues,
+            tagEntryToUsedClients
+        );
+
+        while (numRemainingStandbys > 0) {
+            final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
+                activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(uuid, tagEntryToUsedClients)
+            );
+
+            if (clientOnUnusedTagDimensions == null) {
+                break;
+            }
+
+            clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
+
+            countOfUsedClients++;
+            numRemainingStandbys--;
+
+            if (numRemainingStandbys == 0) {
+                break;
+            }
+
+            fillClientsOnAlreadyUsedTagEntries(
+                clientOnUnusedTagDimensions,
+                countOfUsedClients,
+                rackAwareAssignmentTags,
+                clientStates,
+                tagEntryToClients,
+                tagKeyToValues,
+                tagEntryToUsedClients
+            );
+        }

Review comment:
       Wouldn't this be equivalent and maybe a bit more concise?
   
   ```suggestion
           UUID lastUsedclient = activeTaskClient;
           do {
               fillClientsOnAlreadyUsedTagEntries(
                   lastUsedclient,
                   countOfUsedClients,
                   rackAwareAssignmentTags,
                   clientStates,
                   tagEntryToClients,
                   tagKeyToValues,
                   tagEntryToUsedClients
               );
   
               final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll(
                   activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(uuid, tagEntryToUsedClients)
               );
   
               if (clientOnUnusedTagDimensions == null) {
                   break;
               }
   
               clientStates.get(clientOnUnusedTagDimensions).assignStandby(activeTaskId);
   
               countOfUsedClients++;
               numRemainingStandbys--;
   
               lastUsedclient = clientOnUnusedTagDimensions;
           } while (numRemainingStandbys > 0);
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -0,0 +1,303 @@
+/*
+ * 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;
+import static org.apache.kafka.streams.processor.internals.assignment.StandbyTaskAssignmentUtils.pollClientAndMaybeAssignRemainingStandbyTasks;
+
+/**
+ * Distributes standby tasks over different tag dimensions. Standby task distribution is on a best-effort basis.
+ * If rack aware standby task assignment is not possible, implementation fall backs to distributing standby tasks on least-loaded clients.
+ *
+ * @see DefaultStandbyTaskAssignor
+ */
+class ClientTagAwareStandbyTaskAssignor implements StandbyTaskAssignor {
+    private static final Logger log = LoggerFactory.getLogger(ClientTagAwareStandbyTaskAssignor.class);
+
+    /**
+     * The algorithm distributes standby tasks for the {@param statefulTaskIds} over different tag dimensions.
+     * For each stateful task, the number of standby tasks will be assigned based on configured {@link AssignmentConfigs#numStandbyReplicas}.
+     * Rack aware standby tasks distribution only takes into account tags specified via {@link AssignmentConfigs#rackAwareAssignmentTags}.
+     * Ideally, all standby tasks for any given stateful task will be located on different tag dimensions to have the best possible distribution.
+     * However, if the ideal (or partially ideal) distribution is impossible, the algorithm will fall back to the least-loaded clients without taking rack awareness constraints into consideration.
+     * The least-loaded clients are determined based on the total number of tasks (active and standby tasks) assigned to the client.
+     */
+    @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(

Review comment:
       I could not find where you decrement the number of remaining standbys. If you get a value from this map and put it into an `int` variable, you do not have a reference to the `Integer` value in the map anymore. This might become a problem in `StandbyTaskAssignmentUtils#pollClientAndMaybeAssignRemainingStandbyTasks()`.




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



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

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


   Hi @cadonna @showuon 
   I've addressed your comments. Please have a look.
   Thank you!


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



[GitHub] [kafka] showuon merged pull request #10851: KAFKA-6718 / Rack aware standby task assignor

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


   


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