showuon commented on a change in pull request #11837:
URL: https://github.com/apache/kafka/pull/11837#discussion_r827603227



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientTagAwareStandbyTaskAssignor.java
##########
@@ -60,7 +60,7 @@ public boolean assign(final Map<UUID, ClientState> clients,
 
         final Map<TaskId, Integer> tasksToRemainingStandbys = 
computeTasksToRemainingStandbys(
             numStandbyReplicas,
-            allTaskIds
+            statefulTaskIds

Review comment:
       nice catch, and thanks for adding a test for it.

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignmentUtilsTest.java
##########
@@ -55,38 +65,38 @@ public void setup() {
 
     @Test
     public void shouldReturnNumberOfStandbyTasksThatWereNotAssigned() {
-        final Map<TaskId, Integer> tasksToRemainingStandbys = 
computeTasksToRemainingStandbys(3, ACTIVE_TASKS);
-
-        assertTrue(tasksToRemainingStandbys.keySet()
-                                           .stream()
-                                           .map(taskId -> 
pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(
-                                               clients,
-                                               tasksToRemainingStandbys,
-                                               clientsByTaskLoad,
-                                               taskId
-                                           ))
-                                           .allMatch(numRemainingStandbys -> 
numRemainingStandbys == 1));
+        final Logger logMock = mock(Logger.class);
+        final int numStandbyReplicas = 3;
+        final Map<TaskId, Integer> tasksToRemainingStandbys = 
computeTasksToRemainingStandbys(numStandbyReplicas, ACTIVE_TASKS);
+
+        tasksToRemainingStandbys.keySet().forEach(taskId -> 
pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(numStandbyReplicas,
+                                                                               
                                    clients,
+                                                                               
                                    tasksToRemainingStandbys,
+                                                                               
                                    clientsByTaskLoad,
+                                                                               
                                    taskId,
+                                                                               
                                    logMock));
 
         assertTrue(ACTIVE_TASKS.stream().allMatch(activeTask -> 
tasksToRemainingStandbys.get(activeTask) == 1));
         assertTrue(areStandbyTasksPresentForAllActiveTasks(2));
+        verify(logMock, times(ACTIVE_TASKS.size())).warn(anyString(), 
anyInt(), anyInt(), any());
     }
 
     @Test
     public void shouldReturnZeroWhenAllStandbyTasksWereSuccessfullyAssigned() {
-        final Map<TaskId, Integer> tasksToRemainingStandbys = 
computeTasksToRemainingStandbys(1, ACTIVE_TASKS);
-
-        assertTrue(tasksToRemainingStandbys.keySet()
-                                           .stream()
-                                           .map(taskId -> 
pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(
-                                               clients,
-                                               tasksToRemainingStandbys,
-                                               clientsByTaskLoad,
-                                               taskId
-                                           ))
-                                           .allMatch(numRemainingStandbys -> 
numRemainingStandbys == 0));
+        final Logger logMock = mock(Logger.class);
+        final int numStandbyReplicas = 1;
+        final Map<TaskId, Integer> tasksToRemainingStandbys = 
computeTasksToRemainingStandbys(numStandbyReplicas, ACTIVE_TASKS);
+
+        tasksToRemainingStandbys.keySet().forEach(taskId -> 
pollClientAndMaybeAssignAndUpdateRemainingStandbyTasks(numStandbyReplicas,
+                                                                               
                                    clients,
+                                                                               
                                    tasksToRemainingStandbys,
+                                                                               
                                    clientsByTaskLoad,
+                                                                               
                                    taskId,
+                                                                               
                                    logMock));
 
         assertTrue(ACTIVE_TASKS.stream().allMatch(activeTask -> 
tasksToRemainingStandbys.get(activeTask) == 0));
         assertTrue(areStandbyTasksPresentForAllActiveTasks(1));
+        verifyNoInteractions(logMock);

Review comment:
       Thanks for the tests

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/RackAwarenessIntegrationTest.java
##########
@@ -0,0 +1,438 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.ThreadMetadata;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Repartitioned;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.StoreBuilder;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.function.Predicate;
+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.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({IntegrationTest.class})
+public class RackAwarenessIntegrationTest {
+    private static final int NUM_BROKERS = 1;
+
+    private static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(NUM_BROKERS);
+
+    private static final String TAG_VALUE_K8_CLUSTER_1 = "k8s-cluster-1";
+    private static final String TAG_VALUE_K8_CLUSTER_2 = "k8s-cluster-2";
+    private static final String TAG_VALUE_K8_CLUSTER_3 = "k8s-cluster-3";
+    private static final String TAG_VALUE_EU_CENTRAL_1A = "eu-central-1a";
+    private static final String TAG_VALUE_EU_CENTRAL_1B = "eu-central-1b";
+    private static final String TAG_VALUE_EU_CENTRAL_1C = "eu-central-1c";
+
+    private static final int DEFAULT_NUMBER_OF_STATEFUL_SUB_TOPOLOGIES = 1;
+    private static final int DEFAULT_FANOUT_NUMBER_OF_PARTITIONS = 2;
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private static final String INPUT_TOPIC = "input-topic";
+
+    private static final String TAG_ZONE = "zone";
+    private static final String TAG_CLUSTER = "cluster";
+
+    private List<KafkaStreamsWithConfiguration> kafkaStreamsInstances;
+    private Properties baseConfiguration;
+    private Topology topology;
+
+    @BeforeClass
+    public static void createTopics() throws Exception {
+        CLUSTER.start();
+        CLUSTER.createTopic(INPUT_TOPIC, 6, 1);
+    }
+
+    @Before
+    public void setup() {
+        kafkaStreamsInstances = new ArrayList<>();
+        baseConfiguration = new Properties();
+        final String safeTestName = safeUniqueTestName(getClass(), testName);
+        final String applicationId = "app-" + safeTestName;
+        baseConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, 
applicationId);
+        baseConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        baseConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath());
+        baseConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.Integer().getClass());
+        baseConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.Integer().getClass());
+    }
+
+    @After
+    public void cleanup() throws IOException {
+        for (final KafkaStreamsWithConfiguration kafkaStreamsWithConfiguration 
: kafkaStreamsInstances) {
+            
kafkaStreamsWithConfiguration.kafkaStreams.close(Duration.ofMillis(IntegrationTestUtils.DEFAULT_TIMEOUT));
+            
IntegrationTestUtils.purgeLocalStreamsState(kafkaStreamsWithConfiguration.configuration);
+        }
+        kafkaStreamsInstances.clear();
+    }
+
+    @Test
+    public void shouldDoRebalancingWithMaximumNumberOfClientTags() throws 
Exception {
+        initTopology(100, 2);
+        final Duration timeout = Duration.ofMinutes(3);

Review comment:
       3 mins for a test is too long IMO. Could we reduce to 2 mins or less?

##########
File path: streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
##########
@@ -144,10 +148,15 @@
     private static final long DEFAULT_COMMIT_INTERVAL_MS = 30000L;
     private static final long EOS_DEFAULT_COMMIT_INTERVAL_MS = 100L;
     private static final int DEFAULT_TRANSACTION_TIMEOUT = 10000;
+    private static final short DEFAULT_MAX_CLIENT_TAG_KEY_VALUE_LENGTH = 40;
 
     public static final int DUMMY_THREAD_INDEX = 1;
     public static final long MAX_TASK_IDLE_MS_DISABLED = -1;
 
+    public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE = 10;
+    public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH = 50;
+    public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH = 100;

Review comment:
        @cadonna , 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-unsubscr...@kafka.apache.org

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


Reply via email to