You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by vv...@apache.org on 2020/09/03 19:17:18 UTC

[kafka] branch trunk updated: KAFKA-10355: Throw error when source topic was deleted (#9191)

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

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


The following commit(s) were added to refs/heads/trunk by this push:
     new 1e31354  KAFKA-10355: Throw error when source topic was deleted (#9191)
1e31354 is described below

commit 1e31354557f49ab4a887aac0b71916da53b22d50
Author: Bruno Cadonna <br...@confluent.io>
AuthorDate: Thu Sep 3 21:16:14 2020 +0200

    KAFKA-10355: Throw error when source topic was deleted (#9191)
    
    Before this commit, Kafka Streams would gracefully shut down the whole application when a source topic is deleted. The graceful shutdown does not give the user the possibility to react on the deletion of the source topic in the uncaught exception handler.
    
    This commit changes this behavior and throws an error when a source topic is deleted.
    
    Reviewers: A. Sophie Blee-Goldman <so...@confluent.io>, Guozhang Wang <gu...@apache.org>, John Roesler <vv...@apache.org>
---
 .../errors/MissingSourceTopicException.java        |  26 +++++
 .../internals/StreamsRebalanceListener.java        |   8 +-
 ...HandlingSourceTopicDeletionIntegrationTest.java | 114 +++++++++++++++++++++
 .../internals/StreamsRebalanceListenerTest.java    |  74 +++++++++++++
 4 files changed, 218 insertions(+), 4 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/MissingSourceTopicException.java b/streams/src/main/java/org/apache/kafka/streams/errors/MissingSourceTopicException.java
new file mode 100644
index 0000000..c23ea4f
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/MissingSourceTopicException.java
@@ -0,0 +1,26 @@
+/*
+ * 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.errors;
+
+public class MissingSourceTopicException extends StreamsException {
+
+    private final static long serialVersionUID = 1L;
+
+    public MissingSourceTopicException(final String message) {
+        super(message);
+    }
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
index b594aa6..f794f08 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
@@ -20,6 +20,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.errors.MissingSourceTopicException;
 import org.apache.kafka.streams.processor.internals.StreamThread.State;
 import org.apache.kafka.streams.processor.internals.assignment.AssignorError;
 import org.slf4j.Logger;
@@ -51,12 +52,11 @@ public class StreamsRebalanceListener implements ConsumerRebalanceListener {
         // NB: all task management is already handled by:
         // org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.onAssignment
         if (assignmentErrorCode.get() == AssignorError.INCOMPLETE_SOURCE_TOPIC_METADATA.code()) {
-            log.error("Received error code {} - shutdown", assignmentErrorCode.get());
-            streamThread.shutdown();
-        } else {
-            streamThread.setState(State.PARTITIONS_ASSIGNED);
+            log.error("Received error code {}", assignmentErrorCode.get());
+            throw new MissingSourceTopicException("One or more source topics were missing during rebalance");
         }
 
+        streamThread.setState(State.PARTITIONS_ASSIGNED);
         taskManager.handleRebalanceComplete();
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/HandlingSourceTopicDeletionIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/HandlingSourceTopicDeletionIntegrationTest.java
new file mode 100644
index 0000000..7d3d243
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/HandlingSourceTopicDeletionIntegrationTest.java
@@ -0,0 +1,114 @@
+/*
+ * 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.processor.internals;
+
+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.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.Produced;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+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.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+@Category({IntegrationTest.class})
+public class HandlingSourceTopicDeletionIntegrationTest {
+
+    private static final int NUM_BROKERS = 1;
+    private static final int NUM_THREADS = 2;
+    private static final long TIMEOUT = 60000;
+    private static final String INPUT_TOPIC = "inputTopic";
+    private static final String OUTPUT_TOPIC = "outputTopic";
+
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    @Before
+    public void before() throws InterruptedException {
+        CLUSTER.createTopics(INPUT_TOPIC, OUTPUT_TOPIC);
+    }
+
+    @After
+    public void after() throws InterruptedException {
+        CLUSTER.deleteTopics(INPUT_TOPIC, OUTPUT_TOPIC);
+    }
+
+    @Test
+    public void shouldThrowErrorAfterSourceTopicDeleted() throws InterruptedException {
+        final StreamsBuilder builder = new StreamsBuilder();
+        builder.stream(INPUT_TOPIC, Consumed.with(Serdes.Integer(), Serdes.String()))
+            .to(OUTPUT_TOPIC, Produced.with(Serdes.Integer(), Serdes.String()));
+
+        final String safeTestName = safeUniqueTestName(getClass(), testName);
+        final String appId = "app-" + safeTestName;
+
+        final Properties streamsConfiguration = new Properties();
+        streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
+        streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
+        streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
+        streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
+        streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS);
+        streamsConfiguration.put(StreamsConfig.METADATA_MAX_AGE_CONFIG, 2000);
+
+        final Topology topology = builder.build();
+        final KafkaStreams kafkaStreams1 = new KafkaStreams(topology, streamsConfiguration);
+        final AtomicBoolean calledUncaughtExceptionHandler1 = new AtomicBoolean(false);
+        kafkaStreams1.setUncaughtExceptionHandler((thread, exception) -> calledUncaughtExceptionHandler1.set(true));
+        kafkaStreams1.start();
+        final KafkaStreams kafkaStreams2 = new KafkaStreams(topology, streamsConfiguration);
+        final AtomicBoolean calledUncaughtExceptionHandler2 = new AtomicBoolean(false);
+        kafkaStreams2.setUncaughtExceptionHandler((thread, exception) -> calledUncaughtExceptionHandler2.set(true));
+        kafkaStreams2.start();
+
+        TestUtils.waitForCondition(
+            () -> kafkaStreams1.state() == State.RUNNING && kafkaStreams2.state() == State.RUNNING,
+            TIMEOUT,
+            () -> "Kafka Streams clients did not reach state RUNNING"
+        );
+
+        CLUSTER.deleteTopicAndWait(INPUT_TOPIC);
+
+        TestUtils.waitForCondition(
+            () -> kafkaStreams1.state() == State.ERROR && kafkaStreams2.state() == State.ERROR,
+            TIMEOUT,
+            () -> "Kafka Streams clients did not reach state ERROR"
+        );
+
+        assertThat(calledUncaughtExceptionHandler1.get(), is(true));
+        assertThat(calledUncaughtExceptionHandler2.get(), is(true));
+    }
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListenerTest.java
new file mode 100644
index 0000000..cc81fe3
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListenerTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.processor.internals;
+
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.streams.errors.MissingSourceTopicException;
+import org.apache.kafka.streams.processor.internals.StreamThread.State;
+import org.apache.kafka.streams.processor.internals.assignment.AssignorError;
+import org.junit.Test;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.mock;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertThrows;
+
+public class StreamsRebalanceListenerTest {
+
+    private final TaskManager taskManager = mock(TaskManager.class);
+    private final StreamThread streamThread = mock(StreamThread.class);
+    private final AtomicInteger assignmentErrorCode = new AtomicInteger();
+    private final StreamsRebalanceListener streamsRebalanceListener = new StreamsRebalanceListener(
+        new MockTime(),
+        taskManager,
+        streamThread,
+        LoggerFactory.getLogger(StreamsRebalanceListenerTest.class),
+        assignmentErrorCode
+    );
+
+    @Test
+    public void shouldThrowMissingSourceTopicException() {
+        replay(taskManager, streamThread);
+        assignmentErrorCode.set(AssignorError.INCOMPLETE_SOURCE_TOPIC_METADATA.code());
+
+        final MissingSourceTopicException exception = assertThrows(
+            MissingSourceTopicException.class,
+            () -> streamsRebalanceListener.onPartitionsAssigned(Collections.emptyList())
+        );
+        assertThat(exception.getMessage(), is("One or more source topics were missing during rebalance"));
+        verify(taskManager, streamThread);
+    }
+
+    @Test
+    public void shouldHandleOnPartitionAssigned() {
+        taskManager.handleRebalanceComplete();
+        expect(streamThread.setState(State.PARTITIONS_ASSIGNED)).andStubReturn(null);
+        replay(taskManager, streamThread);
+        assignmentErrorCode.set(AssignorError.NONE.code());
+
+        streamsRebalanceListener.onPartitionsAssigned(Collections.emptyList());
+
+        verify(taskManager, streamThread);
+    }
+}
\ No newline at end of file