ableegoldman commented on code in PR #16033: URL: https://github.com/apache/kafka/pull/16033#discussion_r1618287082
########## streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java: ########## @@ -16,78 +16,408 @@ */ package org.apache.kafka.streams.processor.assignment; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.SortedSet; +import java.util.UUID; +import java.util.stream.Collectors; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.assignment.KafkaStreamsAssignment.AssignedTask; +import org.apache.kafka.streams.processor.internals.assignment.Graph; +import org.apache.kafka.streams.processor.internals.assignment.MinTrafficGraphConstructor; +import org.apache.kafka.streams.processor.internals.assignment.RackAwareGraphConstructor; +import org.apache.kafka.streams.processor.internals.assignment.RackAwareGraphConstructorFactory; +import org.apache.kafka.streams.StreamsConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A set of utilities to help implement task assignment via the {@link TaskAssignor} */ public final class TaskAssignmentUtils { + private static final Logger LOG = LoggerFactory.getLogger(TaskAssignmentUtils.class); + + private TaskAssignmentUtils() {} + /** - * Assign standby tasks to KafkaStreams clients according to the default logic. - * <p> - * If rack-aware client tags are configured, the rack-aware standby task assignor will be used + * Return a "no-op" assignment that just copies the previous assignment of tasks to KafkaStreams clients * - * @param applicationState the metadata and other info describing the current application state - * @param kafkaStreamsAssignments the current assignment of tasks to KafkaStreams clients + * @param applicationState the metadata and other info describing the current application state * - * @return a new map containing the mappings from KafkaStreamsAssignments updated with the default - * standby assignment + * @return a new map containing an assignment that replicates exactly the previous assignment reported + * in the applicationState */ - public static Map<ProcessId, KafkaStreamsAssignment> defaultStandbyTaskAssignment( - final ApplicationState applicationState, - final Map<ProcessId, KafkaStreamsAssignment> kafkaStreamsAssignments - ) { - throw new UnsupportedOperationException("Not Implemented."); + public static Map<ProcessId, KafkaStreamsAssignment> identityAssignment(final ApplicationState applicationState) { + final Map<ProcessId, KafkaStreamsAssignment> assignments = new HashMap<>(); + applicationState.kafkaStreamsStates(false).forEach((processId, state) -> { + final Set<AssignedTask> tasks = new HashSet<>(); + state.previousActiveTasks().forEach(taskId -> { + tasks.add(new AssignedTask(taskId, + AssignedTask.Type.ACTIVE)); + }); + state.previousStandbyTasks().forEach(taskId -> { + tasks.add(new AssignedTask(taskId, + AssignedTask.Type.STANDBY)); + }); + + final KafkaStreamsAssignment newAssignment = KafkaStreamsAssignment.of(processId, tasks); + assignments.put(processId, newAssignment); + }); + return assignments; } /** - * Optimize the active task assignment for rack-awareness + * Optimize active task assignment for rack awareness. This optimization is based on the + * {@link StreamsConfig#RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG trafficCost} + * and {@link StreamsConfig#RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG nonOverlapCost} + * configs which balance cross rack traffic minimization and task movement. + * Setting {@code trafficCost} to a larger number reduces the overall cross rack traffic of the resulting + * assignment, but can increase the number of tasks shuffled around between clients. + * Setting {@code nonOverlapCost} to a larger number increases the affinity of tasks to their intended client + * and reduces the amount by which the rack-aware optimization can shuffle tasks around, at the cost of higher + * cross-rack traffic. + * In an extreme case, if we set {@code nonOverlapCost} to 0 and @{code trafficCost} to a positive value, + * the resulting assignment will have an absolute minimum of cross rack traffic. If we set {@code trafficCost} to 0, + * and {@code nonOverlapCost} to a positive value, the resulting assignment will be identical to the input assignment. + * <p> + * This method optimizes cross-rack traffic for active tasks only. For standby task optimization, + * use {@link #optimizeRackAwareStandbyTasks}. * * @param applicationState the metadata and other info describing the current application state * @param kafkaStreamsAssignments the current assignment of tasks to KafkaStreams clients - * @param tasks the set of tasks to reassign if possible. Must already be assigned - * to a KafkaStreams client + * @param tasks the set of tasks to reassign if possible. Must already be assigned to a KafkaStreams client * - * @return a new map containing the mappings from KafkaStreamsAssignments updated with the default - * rack-aware assignment for active tasks + * @return a new map containing the mappings from KafkaStreamsAssignments updated with the default rack-aware assignment for active tasks */ public static Map<ProcessId, KafkaStreamsAssignment> optimizeRackAwareActiveTasks( final ApplicationState applicationState, final Map<ProcessId, KafkaStreamsAssignment> kafkaStreamsAssignments, final SortedSet<TaskId> tasks ) { - throw new UnsupportedOperationException("Not Implemented."); + if (tasks.isEmpty()) { + return kafkaStreamsAssignments; + } + + if (!hasValidRackInformation(applicationState)) { + LOG.warn("Cannot optimize active tasks with invalid rack information."); + return kafkaStreamsAssignments; + } + + final int crossRackTrafficCost = applicationState.assignmentConfigs().rackAwareTrafficCost(); + final int nonOverlapCost = applicationState.assignmentConfigs().rackAwareNonOverlapCost(); + final long currentCost = computeTaskCost( + applicationState.allTasks().stream().filter(taskInfo -> tasks.contains(taskInfo.id())).collect( + Collectors.toSet()), + applicationState.kafkaStreamsStates(false), + crossRackTrafficCost, + nonOverlapCost, + false, + false + ); + LOG.info("Assignment before active task optimization has cost {}", currentCost); + + final List<UUID> clientIds = kafkaStreamsAssignments.keySet().stream().map(ProcessId::id).collect( + Collectors.toList()); + final Map<ProcessId, KafkaStreamsState> kafkaStreamsStates = applicationState.kafkaStreamsStates(false); + final Map<UUID, Optional<String>> clientRacks = kafkaStreamsStates.values().stream().collect( + Collectors.toMap(state -> state.processId().id(), KafkaStreamsState::rackId)); + final Map<UUID, Set<TaskId>> previousTaskIdsByProcess = kafkaStreamsStates.values().stream().collect(Collectors.toMap( + state -> state.processId().id(), + KafkaStreamsState::previousActiveTasks + )); + final Map<TaskId, Set<TaskTopicPartition>> topicPartitionsByTaskId = applicationState.allTasks().stream() + .filter(taskInfo -> tasks.contains(taskInfo.id())) + .collect(Collectors.toMap(TaskInfo::id, TaskInfo::topicPartitions)); + + final List<TaskId> taskIds = new ArrayList<>(tasks); Review Comment: I ended up having to redo a bunch of the data structures as part of the KafkaStreamsAssignment POC ([this](https://github.com/apourchet/kafka/pull/1)) so I happened to address this comment as well. You can see in the POC PR -- there's not actually very much we need to compute if we just do it all up front -- 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