You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by mj...@apache.org on 2019/08/08 00:05:52 UTC

[kafka] branch 2.0 updated: KAFKA-8602: Backport bugfix for standby task creation (#7146)

This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.0 by this push:
     new 79eaddd  KAFKA-8602: Backport bugfix for standby task creation (#7146)
79eaddd is described below

commit 79eaddd25f8d030f18618b367f608b80f5862ea5
Author: cadonna <br...@confluent.io>
AuthorDate: Thu Aug 8 02:05:17 2019 +0200

    KAFKA-8602: Backport bugfix for standby task creation (#7146)
    
    Reviewers: Bill Bejeck <bi...@confluent.io>, Matthias J. Sax <ma...@confluent.io>
---
 .../streams/processor/internals/StreamThread.java  |   2 +-
 .../StandbyTaskCreationIntegrationTest.java        | 159 +++++++++++++++++++++
 .../processor/internals/StreamThreadTest.java      |  65 ++++++++-
 3 files changed, 224 insertions(+), 2 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
index 5e5b81d..038881f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
@@ -487,7 +487,7 @@ public class StreamThread extends Thread {
 
             final ProcessorTopology topology = builder.build(taskId.topicGroupId);
 
-            if (!topology.stateStores().isEmpty()) {
+            if (!topology.stateStores().isEmpty() && !topology.storeToChangelogTopic().isEmpty()) {
                 return new StandbyTask(
                     taskId,
                     partitions,
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskCreationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskCreationIntegrationTest.java
new file mode 100644
index 0000000..6ef20bc
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskCreationIntegrationTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.KafkaStreams.State;
+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.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Transformer;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.ThreadMetadata;
+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.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Properties;
+import java.util.function.Predicate;
+
+@Category({IntegrationTest.class})
+public class StandbyTaskCreationIntegrationTest {
+
+    private static final int NUM_BROKERS = 1;
+
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS);
+
+    private static final String INPUT_TOPIC = "input-topic";
+
+    private KafkaStreams client1;
+    private KafkaStreams client2;
+    private volatile boolean client1IsOk = false;
+    private volatile boolean client2IsOk = false;
+
+    @BeforeClass
+    public static void createTopics() throws InterruptedException {
+        CLUSTER.createTopic(INPUT_TOPIC, 2, 1);
+    }
+
+    @After
+    public void after() {
+        client1.close();
+        client2.close();
+    }
+
+    private Properties streamsConfiguration() {
+        final String applicationId = "testApp";
+        final Properties streamsConfiguration = new Properties();
+        streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
+        streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
+        streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath());
+        streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
+        streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
+        streamsConfiguration.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
+        return streamsConfiguration;
+    }
+
+    @Test
+    public void shouldNotCreateAnyStandByTasksForStateStoreWithLoggingDisabled() throws Exception {
+        final StreamsBuilder builder = new StreamsBuilder();
+        final String stateStoreName = "myTransformState";
+        final StoreBuilder<KeyValueStore<Integer, Integer>> keyValueStoreBuilder =
+            Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore(stateStoreName),
+                                        Serdes.Integer(),
+                                        Serdes.Integer()).withLoggingDisabled();
+        builder.addStateStore(keyValueStoreBuilder);
+        builder.stream(INPUT_TOPIC, Consumed.with(Serdes.Integer(), Serdes.Integer()))
+            .transform(() -> new Transformer<Integer, Integer, KeyValue<Integer, Integer>>() {
+                @SuppressWarnings("unchecked")
+                @Override
+                public void init(final ProcessorContext context) {}
+
+                @Override
+                public KeyValue<Integer, Integer> transform(final Integer key, final Integer value) {
+                    return null;
+                }
+
+                @Override
+                public void close() {}
+            }, stateStoreName);
+
+        final Topology topology = builder.build();
+        createClients(topology, streamsConfiguration(), topology, streamsConfiguration());
+
+        setStateListenersForVerification(thread -> thread.standbyTasks().isEmpty() && !thread.activeTasks().isEmpty());
+
+        startClients();
+
+        waitUntilBothClientAreOK(
+            "At least one client did not reach state RUNNING with active tasks but no stand-by tasks"
+        );
+    }
+
+    private void createClients(final Topology topology1,
+                               final Properties streamsConfiguration1,
+                               final Topology topology2,
+                               final Properties streamsConfiguration2) {
+
+        client1 = new KafkaStreams(topology1, streamsConfiguration1);
+        client2 = new KafkaStreams(topology2, streamsConfiguration2);
+    }
+
+    private void setStateListenersForVerification(final Predicate<ThreadMetadata> taskCondition) {
+        client1.setStateListener((newState, oldState) -> {
+            if (newState == State.RUNNING &&
+                client1.localThreadsMetadata().stream().allMatch(taskCondition)) {
+
+                client1IsOk = true;
+            }
+        });
+        client2.setStateListener((newState, oldState) -> {
+            if (newState == State.RUNNING &&
+                client2.localThreadsMetadata().stream().allMatch(taskCondition)) {
+
+                client2IsOk = true;
+            }
+        });
+    }
+
+    private void startClients() {
+        client1.start();
+        client2.start();
+    }
+
+    private void waitUntilBothClientAreOK(final String message) throws Exception {
+        TestUtils.waitForCondition(
+            () -> client1IsOk && client2IsOk,
+            30 * 1000,
+            message + ": "
+                + "Client 1 is " + (!client1IsOk ? "NOT " : "") + "OK, "
+                + "client 2 is " + (!client2IsOk ? "NOT " : "") + "OK."
+        );
+    }
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
index cf6074d..51825bc 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
@@ -21,6 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.InvalidOffsetException;
 import org.apache.kafka.clients.consumer.MockConsumer;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
 import org.apache.kafka.clients.producer.MockProducer;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.common.Cluster;
@@ -56,11 +57,15 @@ import org.apache.kafka.streams.processor.Punctuator;
 import org.apache.kafka.streams.processor.TaskId;
 import org.apache.kafka.streams.processor.TaskMetadata;
 import org.apache.kafka.streams.processor.ThreadMetadata;
+import org.apache.kafka.streams.processor.internals.StreamThread.StreamsMetricsThreadImpl;
 import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
 import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.StoreBuilder;
 import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
 import org.apache.kafka.test.MockClientSupplier;
+import org.apache.kafka.test.MockProcessor;
 import org.apache.kafka.test.MockStateRestoreListener;
+import org.apache.kafka.test.MockStoreBuilder;
 import org.apache.kafka.test.MockTimestampExtractor;
 import org.apache.kafka.test.TestCondition;
 import org.apache.kafka.test.TestUtils;
@@ -68,6 +73,7 @@ import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
 
 import java.io.File;
 import java.io.IOException;
@@ -88,12 +94,14 @@ import static org.apache.kafka.common.utils.Utils.mkMap;
 import static org.apache.kafka.common.utils.Utils.mkProperties;
 import static org.apache.kafka.streams.processor.internals.AbstractStateManager.CHECKPOINT_FILE_NAME;
 import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -908,6 +916,61 @@ public class StreamThreadTest {
     }
 
     @Test
+    public void shouldCreateStandbyTask() {
+        setupInternalTopologyWithoutState();
+        internalTopologyBuilder.addStateStore(new MockStoreBuilder("myStore", true), "processor1");
+
+        final StandbyTask standbyTask = createStandbyTask();
+
+        assertThat(standbyTask, not(nullValue()));
+    }
+
+    @Test
+    public void shouldNotCreateStandbyTaskWithoutStateStores() {
+        setupInternalTopologyWithoutState();
+
+        final StandbyTask standbyTask = createStandbyTask();
+
+        assertThat(standbyTask, nullValue());
+    }
+
+    @Test
+    public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled() {
+        setupInternalTopologyWithoutState();
+        final StoreBuilder storeBuilder = new MockStoreBuilder("myStore", true);
+        storeBuilder.withLoggingDisabled();
+        internalTopologyBuilder.addStateStore(storeBuilder, "processor1");
+
+        final StandbyTask standbyTask = createStandbyTask();
+
+        assertThat(standbyTask, nullValue());
+    }
+
+    private void setupInternalTopologyWithoutState() {
+        final MockProcessor mockProcessor = new MockProcessor();
+        internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
+        internalTopologyBuilder.addProcessor("processor1", () -> mockProcessor, "source1");
+    }
+
+    private StandbyTask createStandbyTask() {
+        final LogContext logContext = new LogContext("test");
+        final Logger log = logContext.logger(StreamThreadTest.class);
+        final StreamsMetricsThreadImpl streamsMetrics = new StreamsMetricsThreadImpl(metrics, clientId);
+        final StreamThread.StandbyTaskCreator standbyTaskCreator = new StreamThread.StandbyTaskCreator(
+            internalTopologyBuilder,
+            config,
+            streamsMetrics,
+            stateDirectory,
+            new MockChangelogReader(),
+            mockTime,
+            log);
+        return standbyTaskCreator.createTask(
+            new MockConsumer<>(OffsetResetStrategy.EARLIEST),
+            new TaskId(1, 2),
+            Collections.emptySet());
+    }
+
+    @Test
     public void shouldPunctuateActiveTask() {
         final List<Long> punctuatedStreamTime = new ArrayList<>();
         final List<Long> punctuatedWallClockTime = new ArrayList<>();