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



##########
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/RackAwarenessIntegrationTest.java
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.KeyValue;
+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.Transformer;
+import org.apache.kafka.streams.processor.ProcessorContext;
+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.TestCondition;
+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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.function.Predicate;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+
+@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";
+
+    @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;
+
+    @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 
shouldDistributeStandbyReplicasWhenAllClientsAreLocatedOnASameClusterTag() 
throws Exception {
+        final Topology topology = createStatefulTopology();
+        final int numberOfStandbyReplicas = 1;
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1A, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1A, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1A, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1B, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1B, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1B, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+
+        waitUntilAllKafkaStreamsClientsAreRunning();
+        
waitUntilIdealTaskDistributionIsReachedForTags(singletonList(TAG_ZONE));
+    }
+
+    @Test
+    public void shouldDistributeStandbyReplicasOverMultipleClientTags() throws 
Exception {
+        final Topology topology = createStatefulTopology();
+        final int numberOfStandbyReplicas = 2;
+
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1A, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1B, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1C, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1A, 
TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1B, 
TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1C, 
TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1A, 
TAG_VALUE_K8_CLUSTER_3), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1B, 
TAG_VALUE_K8_CLUSTER_3), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1C, 
TAG_VALUE_K8_CLUSTER_3), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+
+        waitUntilAllKafkaStreamsClientsAreRunning();
+        waitUntilIdealTaskDistributionIsReachedForTags(asList(TAG_ZONE, 
TAG_CLUSTER));
+    }
+
+    @Test
+    public void 
shouldDistributeStandbyReplicasWhenIdealDistributionCanNotBeAchieved() throws 
Exception {
+        final Topology topology = createStatefulTopology();
+        final int numberOfStandbyReplicas = 2;
+
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1A, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1B, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1C, 
TAG_VALUE_K8_CLUSTER_1), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1A, 
TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1B, 
TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+        createAndStart(topology, buildClientTags(TAG_VALUE_EU_CENTRAL_1C, 
TAG_VALUE_K8_CLUSTER_2), asList(TAG_ZONE, TAG_CLUSTER), 
numberOfStandbyReplicas);
+
+        waitUntilAllKafkaStreamsClientsAreRunning();
+        
waitUntilIdealTaskDistributionIsReachedForTags(singletonList(TAG_ZONE));
+        
waitUntilPartialTaskDistributionIsReachedForTags(singletonList(TAG_CLUSTER));

Review comment:
       Yes good point. Will update the 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-unsubscr...@kafka.apache.org

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


Reply via email to