You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/07/24 19:13:08 UTC

[GitHub] [kafka] abbccdda commented on a change in pull request #9047: KAFKA-9274: Remove `retries` for global task

abbccdda commented on a change in pull request #9047:
URL: https://github.com/apache/kafka/pull/9047#discussion_r460165601



##########
File path: docs/streams/developer-guide/config-streams.html
##########
@@ -326,13 +321,18 @@ <h4><a class="toc-backref" href="#id5">bootstrap.servers</a><a class="headerlink
           <tr class="row-even"><td>state.cleanup.delay.ms</td>
             <td>Low</td>
             <td colspan="2">The amount of time in milliseconds to wait before deleting state when a partition has migrated.</td>
-            <td>600000 milliseconds</td>
+            <td>600000 milliseconds (10 minutes)</td>
           </tr>
           <tr class="row-odd"><td>state.dir</td>
             <td>High</td>
             <td colspan="2">Directory location for state stores.</td>
             <td><code class="docutils literal"><span class="pre">/tmp/kafka-streams</span></code></td>
           </tr>
+          <tr class="row-odd"><td>task.timeout.ms</td>
+            <td>Medium</td>

Review comment:
       nit: for a doc clean-up, it is helpful to include a screenshot of updated paragraph. The changes starting at L331 should be suffice.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
##########
@@ -318,6 +341,72 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback,
         }
     }
 
+    private void retryUntilSuccessOrThrowOnTaskTimeout(final Runnable runnable,
+                                                       final String errorMessage) {
+        long deadlineMs = NO_DEADLINE;
+
+        do {
+            try {
+                runnable.run();
+                return;
+            } catch (final TimeoutException retryableException) {

Review comment:
       `retriableException` to be consistent with the defined exception type in AK.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
##########
@@ -275,31 +259,70 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback,
                               final RecordConverter recordConverter) {
         for (final TopicPartition topicPartition : topicPartitions) {
             globalConsumer.assign(Collections.singletonList(topicPartition));
+            long offset;
             final Long checkpoint = checkpointFileCache.get(topicPartition);
             if (checkpoint != null) {
                 globalConsumer.seek(topicPartition, checkpoint);
+                offset = checkpoint;
             } else {
                 globalConsumer.seekToBeginning(Collections.singletonList(topicPartition));
+                final AtomicLong position = new AtomicLong();

Review comment:
       For the above call, was curious why we couldn't seek for all the topic partitions that are missing positions here, instead of doing one by one look-up?

##########
File path: streams/src/main/java/org/apache/kafka/streams/internals/QuietStreamsConfig.java
##########
@@ -1,33 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.kafka.streams.StreamsConfig;
-
-import java.util.Map;
-
-/**
- * A {@link StreamsConfig} that does not log its configuration on construction.
- *
- * This producer cleaner output for unit tests using the {@code test-utils},
- * since logging the config is not really valuable in this context.
- */
-public class QuietStreamsConfig extends StreamsConfig {

Review comment:
       What's the purpose for this move?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
##########
@@ -275,31 +259,70 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback,
                               final RecordConverter recordConverter) {
         for (final TopicPartition topicPartition : topicPartitions) {
             globalConsumer.assign(Collections.singletonList(topicPartition));
+            long offset;
             final Long checkpoint = checkpointFileCache.get(topicPartition);
             if (checkpoint != null) {
                 globalConsumer.seek(topicPartition, checkpoint);
+                offset = checkpoint;
             } else {
                 globalConsumer.seekToBeginning(Collections.singletonList(topicPartition));
+                final AtomicLong position = new AtomicLong();
+                retryUntilSuccessOrThrowOnTaskTimeout(
+                    () -> position.set(globalConsumer.position(topicPartition)),
+                    String.format(
+                        "Failed to get position for partition %s. The broker may be transiently unavailable at the moment.",
+                        topicPartition
+                    )
+                );
+                offset = position.get();
             }
 
-            long offset = globalConsumer.position(topicPartition);
             final Long highWatermark = highWatermarks.get(topicPartition);
             final RecordBatchingStateRestoreCallback stateRestoreAdapter =
                 StateRestoreCallbackAdapter.adapt(stateRestoreCallback);
 
             stateRestoreListener.onRestoreStart(topicPartition, storeName, offset, highWatermark);
             long restoreCount = 0L;
 
+            long deadlineMs = NO_DEADLINE;
             while (offset < highWatermark) {
                 try {
                     final ConsumerRecords<byte[], byte[]> records = globalConsumer.poll(pollTime);
+                    if (records.isEmpty()) {
+                        if (taskTimeoutMs == 0L) {
+                            deadlineMs = maybeUpdateDeadlineOrThrow(
+                                deadlineMs,
+                                requestTimeoutMs,
+                                new StreamsException(String.format(
+                                    "Global task did not make progress to restore state. Retrying is disabled. You can enable it by setting `%s` to a value larger than zero.",
+                                    StreamsConfig.TASK_TIMEOUT_MS_CONFIG
+                                ))
+                            );
+                        } else {
+                            deadlineMs = maybeUpdateDeadlineOrThrow(deadlineMs);
+                        }
+
+                        continue;
+                    }
+                    deadlineMs = NO_DEADLINE;
+
                     final List<ConsumerRecord<byte[], byte[]>> restoreRecords = new ArrayList<>();
                     for (final ConsumerRecord<byte[], byte[]> record : records.records(topicPartition)) {
                         if (record.key() != null) {
                             restoreRecords.add(recordConverter.convert(record));
                         }
                     }
-                    offset = globalConsumer.position(topicPartition);
+                    try {
+                        offset = globalConsumer.position(topicPartition);
+                    } catch (final TimeoutException error) {
+                        // the `globalConsumer.position()` call should never block, because we know that we did
+                        // a successful `position()` call above for the requested partition and thus the consumer
+                        // should have a valid local position that it can return immediately
+
+                        // hence, a `TimeoutException` indicates a bug and thus we rethrow it as fatal `IllegalStateException`
+                        throw new IllegalStateException(error);

Review comment:
       We should add some explanation in the illegal state exception for why such timeout is fatal

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
##########
@@ -318,6 +341,72 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback,
         }
     }
 
+    private void retryUntilSuccessOrThrowOnTaskTimeout(final Runnable runnable,
+                                                       final String errorMessage) {
+        long deadlineMs = NO_DEADLINE;
+
+        do {
+            try {
+                runnable.run();
+                return;
+            } catch (final TimeoutException retryableException) {
+                if (taskTimeoutMs == 0L) {
+                    throw new StreamsException(
+                        String.format(
+                            "Retrying is disabled. You can enable it by setting `%s` to a value larger than zero.",
+                            StreamsConfig.TASK_TIMEOUT_MS_CONFIG
+                        ),
+                        retryableException
+                    );
+                }
+
+                deadlineMs = maybeUpdateDeadlineOrThrow(deadlineMs);
+
+                log.debug(errorMessage, retryableException);

Review comment:
       Should we log warning here instead? At least this is a timeout.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
##########
@@ -299,7 +318,17 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback,
                             restoreRecords.add(recordConverter.convert(record));
                         }
                     }
-                    offset = globalConsumer.position(topicPartition);
+                    try {
+                        offset = globalConsumer.position(topicPartition);
+                    } catch (final TimeoutException error) {
+                        // the `globalConsumer.position()` call should never block, because we know that we did
+                        // a successful `position()` call above for the requested partition and thus the consumer
+                        // should have a valid local position that it can return immediately

Review comment:
       If we do have a valid position from previous `position` call, do we still need to update the position here again?

##########
File path: streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java
##########
@@ -78,6 +78,7 @@ public void init(final ProcessorContext context, final StateStore root) {
             new File(context.stateDir() + File.separator + name).mkdir();
         }
         this.initialized = true;
+        context.register(root, (k, v) -> { });

Review comment:
       Why do we need this?




----------------------------------------------------------------
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:
users@infra.apache.org