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-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org