guozhangwang commented on a change in pull request #11787:
URL: https://github.com/apache/kafka/pull/11787#discussion_r814202232



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutionMetadata.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.slf4j.Logger;
+
+import static 
org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
+
+/**
+ * Multi-threaded class that tracks the status of active tasks being 
processed. A single instance of this class is
+ * shared between all StreamThreads.
+ */
+public class TaskExecutionMetadata {
+    private final boolean hasNamedTopologies;
+    // map of topologies experiencing errors/currently under backoff
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+
+    public TaskExecutionMetadata(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+    }
+
+    public boolean canProcessTask(final Task task, final long now) {
+        final String topologyName = task.id().topologyName();
+        if (!hasNamedTopologies) {
+            // TODO implement error handling/backoff for non-named topologies 
(needs KIP)
+            return true;
+        } else {
+            final NamedTopologyMetadata metadata = 
topologyNameToErrorMetadata.get(topologyName);
+            return metadata == null || (metadata.canProcess() && 
metadata.canProcessTask(task, now));
+        }
+    }
+
+    public void registerTaskError(final Task task, final Throwable t, final 
long now) {
+        if (hasNamedTopologies) {
+            final String topologyName = task.id().topologyName();
+            topologyNameToErrorMetadata.computeIfAbsent(topologyName, n -> new 
NamedTopologyMetadata(topologyName))
+                .registerTaskError(task, t, now);
+        }
+    }
+
+    class NamedTopologyMetadata {
+        private final Logger log;
+        private final Map<TaskId, Long> tasksToErrorTime = new 
ConcurrentHashMap<>();
+
+        public NamedTopologyMetadata(final String topologyName) {
+            final LogContext logContext = new 
LogContext(String.format("topology-name [%s] ", topologyName));
+            this.log = logContext.logger(NamedTopologyMetadata.class);
+        }
+
+        public boolean canProcess() {
+            // TODO: during long task backoffs, pause the full topology to 
avoid it getting out of sync
+            return true;
+        }
+
+        public boolean canProcessTask(final Task task, final long now) {
+            // TODO: implement exponential backoff, for now we just wait 15s
+            final Long errorTime = tasksToErrorTime.get(task.id());
+            if (errorTime == null) {
+                return true;
+            } else if (now - errorTime > 15000L) {

Review comment:
       Curious why the magic number of 15s?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutionMetadata.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.LogContext;
+import org.apache.kafka.streams.processor.TaskId;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.slf4j.Logger;
+
+import static 
org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
+
+/**
+ * Multi-threaded class that tracks the status of active tasks being 
processed. A single instance of this class is
+ * shared between all StreamThreads.
+ */
+public class TaskExecutionMetadata {
+    private final boolean hasNamedTopologies;
+    // map of topologies experiencing errors/currently under backoff
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+
+    public TaskExecutionMetadata(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+    }
+
+    public boolean canProcessTask(final Task task, final long now) {
+        final String topologyName = task.id().topologyName();
+        if (!hasNamedTopologies) {
+            // TODO implement error handling/backoff for non-named topologies 
(needs KIP)
+            return true;
+        } else {
+            final NamedTopologyMetadata metadata = 
topologyNameToErrorMetadata.get(topologyName);
+            return metadata == null || (metadata.canProcess() && 
metadata.canProcessTask(task, now));
+        }
+    }
+
+    public void registerTaskError(final Task task, final Throwable t, final 
long now) {
+        if (hasNamedTopologies) {
+            final String topologyName = task.id().topologyName();
+            topologyNameToErrorMetadata.computeIfAbsent(topologyName, n -> new 
NamedTopologyMetadata(topologyName))
+                .registerTaskError(task, t, now);
+        }
+    }
+
+    class NamedTopologyMetadata {

Review comment:
       nit: we can declare this as `private class` right?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutor.java
##########
@@ -110,11 +120,11 @@ private long processTask(final Task task, final int 
maxNumRecords, final Time ti
                 "Will trigger a new rebalance and close all tasks as zombies 
together.", task.id());
             throw e;
         } catch (final StreamsException e) {
-            log.error("Failed to process stream task {} due to the following 
error:", task.id(), e);
+            log.error(String.format("Failed to process stream task %s due to 
the following error:", task.id()), e);

Review comment:
       Sorry to see log4j still have not figured out the way for both string 
param and exception in presentation..

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/ErrorHandlingIntegrationTest.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.ConsumerConfig;
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.apache.kafka.common.serialization.IntegerSerializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import 
org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper;
+import 
org.apache.kafka.streams.processor.internals.namedtopology.NamedTopologyBuilder;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.junit.AfterClass;
+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 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.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+@Category(IntegrationTest.class)
+public class ErrorHandlingIntegrationTest {
+
+    private static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+    @BeforeClass
+    public static void startCluster() throws IOException {
+        CLUSTER.start();
+    }
+
+    @AfterClass
+    public static void closeCluster() {
+        CLUSTER.stop();
+    }
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private final String testId = safeUniqueTestName(getClass(), testName);
+    private final String appId = "appId_" + testId;
+    private final Properties properties = props();
+
+    // Task 0
+    private final String inputTopic = "input" + testId;
+    private final String outputTopic = "output" + testId;
+    // Task 1
+    private final String errorInputTopic = "error-input" + testId;
+    private final String errorOutputTopic = "error-output" + testId;
+
+    @Before
+    public void setup() {
+        IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, errorInputTopic, 
errorOutputTopic, inputTopic, outputTopic);
+    }
+
+    private Properties props() {
+        return mkObjectProperties(
+            mkMap(
+                mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+                mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
+                mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory(appId).getPath()),
+                mkEntry(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0),
+                mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 15000L),
+                mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.IntegerSerde.class),
+                mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.StringSerde.class),
+                mkEntry(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 10000),
+                mkEntry(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 10000)
+            )
+        );
+    }
+
+    @Test
+    public void shouldBackOffTaskAndEmitDataWithinSameTopology() throws 
Exception {

Review comment:
       A meta question: do we really need an integration test to bring up the 
full stack for this test coverage? I'd feel a unit test with mock time just on 
the task executor would be sufficient?




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