vvcephei commented on a change in pull request #8818:
URL: https://github.com/apache/kafka/pull/8818#discussion_r438421222



##########
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+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.Materialized;
+import org.apache.kafka.streams.processor.StateRestoreListener;
+import 
org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentListener;
+import 
org.apache.kafka.streams.processor.internals.assignment.HighAvailabilityTaskAssignor;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.util.Collection;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+
+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.mkObjectProperties;
+import static org.apache.kafka.common.utils.Utils.mkProperties;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+
+@Category(IntegrationTest.class)
+public class HighAvailabilityTaskAssignorIntegrationTest {
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    @Test
+    public void shouldScaleOutWithWarmupTasksAndInMemoryStores() throws 
InterruptedException {
+        // NB: this test takes at least a minute to run, because it needs a 
probing rebalance, and the minimum
+        // value is one minute
+        shouldScaleOutWithWarmupTasks(storeName -> 
Materialized.as(Stores.inMemoryKeyValueStore(storeName)));
+    }
+
+    @Test
+    public void shouldScaleOutWithWarmupTasksAndPersistentStores() throws 
InterruptedException {
+        // NB: this test takes at least a minute to run, because it needs a 
probing rebalance, and the minimum
+        // value is one minute
+        shouldScaleOutWithWarmupTasks(storeName -> 
Materialized.as(Stores.persistentKeyValueStore(storeName)));
+    }
+
+    private void shouldScaleOutWithWarmupTasks(final Function<String, 
Materialized<Object, Object, KeyValueStore<Bytes, byte[]>>> 
materializedFunction) throws InterruptedException {
+        final String testId = safeUniqueTestName(getClass(), testName);
+        final String appId = "appId_" + System.currentTimeMillis() + "_" + 
testId;
+        final String inputTopic = "input" + testId;
+        final String storeName = "store" + testId;
+        final String storeChangelog = appId + "-store" + testId + "-changelog";
+        final Set<TopicPartition> changelogTopicPartitions = mkSet(
+            new TopicPartition(storeChangelog, 0),
+            new TopicPartition(storeChangelog, 1)
+        );
+
+        IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, 2, inputTopic, 
storeChangelog);
+
+        final ReentrantLock assignmentLock = new ReentrantLock();
+        final AtomicInteger assignmentsCompleted = new AtomicInteger(0);
+        final AtomicBoolean assignmentStable = new AtomicBoolean(false);
+        final AssignmentListener assignmentListener =
+            stable -> {
+                assignmentLock.lock();
+                try {
+                    assignmentsCompleted.incrementAndGet();
+                    assignmentStable.set(stable);
+                } finally {
+                    assignmentLock.unlock();
+                }
+            };
+
+        final StreamsBuilder builder = new StreamsBuilder();
+        builder.table(inputTopic, materializedFunction.apply(storeName));
+        final Topology topology = builder.build();
+
+        final Properties producerProperties = mkProperties(
+            mkMap(
+                mkEntry(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+                mkEntry(ProducerConfig.ACKS_CONFIG, "all"),
+                mkEntry(ProducerConfig.RETRIES_CONFIG, "0"),
+                mkEntry(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName()),
+                mkEntry(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName())
+            )
+        );
+
+        final StringBuilder kiloBuilder = new StringBuilder(1000);
+        for (int i = 0; i < 1000; i++) {
+            kiloBuilder.append('0');
+        }
+        final String kilo = kiloBuilder.toString();
+
+        try (final Producer<String, String> producer = new 
KafkaProducer<>(producerProperties)) {
+            for (int i = 0; i < 1000; i++) {
+                producer.send(new ProducerRecord<>(inputTopic, 
String.valueOf(i), kilo));
+            }
+        }
+
+        final Properties consumerProperties = mkProperties(
+            mkMap(
+                mkEntry(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+                mkEntry(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class.getName()),
+                mkEntry(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class.getName())
+            )
+        );
+
+
+        try (final KafkaStreams kafkaStreams0 = new KafkaStreams(topology, 
streamsProperties(appId, assignmentListener));
+             final KafkaStreams kafkaStreams1 = new KafkaStreams(topology, 
streamsProperties(appId, assignmentListener));
+             final Consumer<String, String> consumer = new 
KafkaConsumer<>(consumerProperties)) {
+            kafkaStreams0.start();
+
+            // wait until all the input records are in the changelog
+            TestUtils.waitForCondition(
+                () -> getChangelogOffsetSum(changelogTopicPartitions, 
consumer) == 1000,
+                120_000L,
+                () -> "Input records haven't all been written to the 
changelog: " + getChangelogOffsetSum(changelogTopicPartitions, consumer)
+            );
+
+            final AtomicLong instance1TotalRestored = new AtomicLong(-1);
+            final AtomicLong instance1NumRestored = new AtomicLong(-1);
+            final CountDownLatch restoreCompleteLatch = new CountDownLatch(1);
+            kafkaStreams1.setGlobalStateRestoreListener(new 
StateRestoreListener() {
+                @Override
+                public void onRestoreStart(final TopicPartition topicPartition,
+                                           final String storeName,
+                                           final long startingOffset,
+                                           final long endingOffset) {
+                }
+
+                @Override
+                public void onBatchRestored(final TopicPartition 
topicPartition,
+                                            final String storeName,
+                                            final long batchEndOffset,
+                                            final long numRestored) {
+                    instance1NumRestored.accumulateAndGet(
+                        numRestored,
+                        (prev, restored) -> prev == -1 ? restored : prev + 
restored
+                    );
+                }
+
+                @Override
+                public void onRestoreEnd(final TopicPartition topicPartition,
+                                         final String storeName,
+                                         final long totalRestored) {
+                    instance1TotalRestored.accumulateAndGet(
+                        totalRestored,
+                        (prev, restored) -> prev == -1 ? restored : prev + 
restored
+                    );
+                    restoreCompleteLatch.countDown();
+                }
+            });
+            final int assignmentsBeforeScaleOut = assignmentsCompleted.get();
+            final AtomicInteger assignmentsAfterScaleOut = new 
AtomicInteger(0);
+            kafkaStreams1.start();
+            TestUtils.waitForCondition(
+                () -> {
+                    assignmentLock.lock();
+                    try {
+                        
assignmentsAfterScaleOut.set(assignmentsCompleted.get());
+                        if (assignmentsBeforeScaleOut < 
assignmentsAfterScaleOut.get()) {
+                            // the first assignment after adding a node should 
be unstable while we warm up the state.
+                            assertThat(assignmentStable.get(), is(false));
+                            return true;
+                        } else {
+                            return false;
+                        }
+                    } finally {
+                        assignmentLock.unlock();
+                    }
+                },
+                120_000L,
+                "Never saw a first assignment after scale out: " + 
assignmentsCompleted.get()
+            );
+
+            TestUtils.waitForCondition(
+                assignmentStable::get,
+                120_000L,
+                "Assignment hasn't become stable: " + 
assignmentsCompleted.get() +
+                    " Note, if this does fail, check and see if the new 
instance just failed to catch up within" +
+                    " the probing rebalance interval. A full minute should be 
long enough to read ~500 records" +
+                    " in any test environment, but you never know..."
+            );
+
+            restoreCompleteLatch.await();
+            // We should finalize the restoration without having restored any 
records (because they're already in
+            // the store. Otherwise, we failed to properly re-use the state 
from the standby.
+            assertThat(instance1TotalRestored.get(), is(0L));
+            // Belt-and-suspenders check that we never even attempt to restore 
any records.
+            assertThat(instance1NumRestored.get(), is(-1L));
+        }
+    }
+
+    private static Properties streamsProperties(final String appId,
+                                                final AssignmentListener 
configuredAssignmentListener) {
+        return mkObjectProperties(
+            mkMap(
+                mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+                mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
+                mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath()),
+                mkEntry(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, "0"),
+                mkEntry(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, "0"), // 
make the warmup catch up completely
+                mkEntry(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, "2"),
+                mkEntry(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, 
"60000"),

Review comment:
       We could, but it would require a KIP update.
   
   
https://cwiki.apache.org/confluence/display/KAFKA/KIP-441%3A+Smooth+Scaling+Out+for+Kafka+Streams#KIP441:SmoothScalingOutforKafkaStreams-Parameters
   
   > Must be at least 1 minute




----------------------------------------------------------------
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.

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


Reply via email to