kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From guozh...@apache.org
Subject [kafka] branch 2.4 updated: KAFKA-9335: Fix StreamPartitionAssignor regression in repartition topics counts (#7904)
Date Tue, 07 Jan 2020 23:01:12 GMT
This is an automated email from the ASF dual-hosted git repository.

guozhang pushed a commit to branch 2.4
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.4 by this push:
     new 8907a44  KAFKA-9335: Fix StreamPartitionAssignor regression in repartition topics
counts (#7904)
8907a44 is described below

commit 8907a4448cdc6a4e82814711ceced9472e7a87e3
Author: Boyang Chen <boyang@confluent.io>
AuthorDate: Tue Jan 7 14:59:27 2020 -0800

    KAFKA-9335: Fix StreamPartitionAssignor regression in repartition topics counts (#7904)
    
    This PR fixes the regression introduced in 2.4 from 2 refactoring PRs:
    #7249
    #7419
    
    The bug was introduced by having a logical path leading numPartitionsCandidate to be 0,
which is assigned to numPartitions and later being checked by setNumPartitions. In the subsequent
check we will throw illegal argument if the numPartitions is 0.
    
    This bug is both impacting new 2.4 application and upgrades to 2.4 in certain types of
topology. The example in original JIRA was imported as a new integration test to guard against
such regression. We also verify that without the bug fix application will still fail by running
this integration test.
    
    Reviewers: Guozhang Wang <wangguoz@gmail.com>
---
 .../internals/StreamsPartitionAssignor.java        |  11 +-
 ...MultiLevelRepartitionConnectedTopologyTest.java | 151 +++++++++++++++++++++
 2 files changed, 158 insertions(+), 4 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
index 38a150b..9c3cc6d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
@@ -439,7 +439,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor,
Conf
                                 // if this topic is one of the sink topics of this topology,
                                 // use the maximum of all its source topic partitions as
the number of partitions
                                 for (final String sourceTopicName : otherTopicsInfo.sourceTopics)
{
-                                    int numPartitionsCandidate = 0;
+                                    Integer numPartitionsCandidate = null;
                                     // It is possible the sourceTopic is another internal
topic, i.e,
                                     // map().join().join(map())
                                     if (repartitionTopicMetadata.containsKey(sourceTopicName))
{
@@ -459,13 +459,16 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor,
Conf
                                         numPartitionsCandidate = count;
                                     }
 
-                                    if (numPartitions == null || numPartitionsCandidate >
numPartitions) {
-                                        numPartitions = numPartitionsCandidate;
+                                    if (numPartitionsCandidate != null) {
+                                        if (numPartitions == null || numPartitionsCandidate
> numPartitions) {
+                                            numPartitions = numPartitionsCandidate;
+                                        }
                                     }
                                 }
                             }
                         }
-                        // if we still have not find the right number of partitions,
+
+                        // if we still have not found the right number of partitions,
                         // another iteration is needed
                         if (numPartitions == null) {
                             numPartitionsNeeded = true;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/BranchedMultiLevelRepartitionConnectedTopologyTest.java
b/streams/src/test/java/org/apache/kafka/streams/integration/BranchedMultiLevelRepartitionConnectedTopologyTest.java
new file mode 100644
index 0000000..cfff070
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/BranchedMultiLevelRepartitionConnectedTopologyTest.java
@@ -0,0 +1,151 @@
+/*
+ * 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 kafka.utils.MockTime;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+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.Topology;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Test out a topology with 3 level of sub-topology as:
+ *           0
+ *         /   \
+ *        1    3
+ *         \  /
+ *          2
+ * where each pair of the sub topology is connected by repartition topic.
+ * The purpose of this test is to verify the robustness of the stream partition assignor
algorithm,
+ * especially whether it could build the repartition topic counts (step zero) with a complex
topology.
+ * The traversal path 0 -> 1 -> 2 -> 3 hits the case where sub-topology 2 will be
initialized while its
+ * parent 3 hasn't been initialized yet.
+ */
+@Category({IntegrationTest.class})
+public class BranchedMultiLevelRepartitionConnectedTopologyTest {
+
+    private static final Logger log = LoggerFactory.getLogger(BranchedMultiLevelRepartitionConnectedTopologyTest.class);
+
+    private static String inputStream;
+
+    private KafkaStreams kafkaStreams;
+
+    private Properties streamsConfiguration;
+
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);
+    private final MockTime mockTime = CLUSTER.time;
+
+    @Before
+    public void setUp() throws Exception {
+        final Properties props = new Properties();
+        props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.NO_OPTIMIZATION);
+        streamsConfiguration = StreamsTestUtils.getStreamsConfig(
+            "branched-repartition-topic-test",
+            CLUSTER.bootstrapServers(),
+            Serdes.ByteArray().getClass().getName(),
+            Serdes.ByteArray().getClass().getName(),
+            props);
+
+        inputStream = "input-stream";
+        CLUSTER.createTopic(inputStream, 3, 1);
+
+        IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testTopologyBuild() throws InterruptedException, ExecutionException {
+
+        final StreamsBuilder builder = new StreamsBuilder();
+
+        final KStream<byte[], byte[]> input = builder.stream(inputStream);
+
+        final KStream<byte[], byte[]>[] branches = input
+            .flatMapValues(value -> Collections.singletonList(new byte[0]))
+            .branch((k, v) -> true, (k, v) -> false);
+
+        final KTable<byte[], byte[]> b1 = branches[0]
+            .map(KeyValue::new)
+            .groupByKey()
+            .reduce((k, v) -> v, Materialized.as("odd_store"))
+            .toStream()
+            .peek((k, v) -> { })
+            .map(KeyValue::new)
+            .groupByKey()
+            .reduce((k, v) -> v, Materialized.as("odd_store_2"));
+
+        final KTable<byte[], byte[]> b2 = branches[1]
+            .map(KeyValue::new)
+            .groupByKey()
+            .reduce((k, v) -> v, Materialized.as("even_store"))
+            .toStream()
+            .peek((k, v) -> { })
+            .map(KeyValue::new)
+            .groupByKey()
+            .reduce((k, v) -> v, Materialized.as("even_store_2"));
+
+        b1.join(b2, (v1, v2) -> v1, Materialized.as("joined_store"))
+            .toStream();
+
+        final Topology topology = builder.build(streamsConfiguration);
+        log.info("Built topology: {}", topology.describe());
+
+        final Properties producerConfig = TestUtils.producerConfig(
+            CLUSTER.bootstrapServers(), ByteArraySerializer.class, ByteArraySerializer.class);
+
+        final List<KeyValue<byte[], byte[]>> initialKeyValues = Collections.singletonList(
+            KeyValue.pair(new byte[1], new byte[1]));
+
+        IntegrationTestUtils.produceKeyValuesSynchronously(
+            inputStream, initialKeyValues, producerConfig, mockTime);
+
+        kafkaStreams = new KafkaStreams(topology, streamsConfiguration);
+
+        kafkaStreams.cleanUp();
+        kafkaStreams.start();
+
+        TestUtils.waitForCondition(() -> kafkaStreams.state() == KafkaStreams.State.RUNNING,
+                                   "Failed to observe stream transits to RUNNING");
+
+        kafkaStreams.close();
+    }
+}


Mime
View raw message