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-unsubscr...@kafka.apache.org

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


Reply via email to