Skip to content

Commit 197a5d5

Browse files
sjmittalguozhangwang
authored andcommitted
KAFKA-4848: Fix retryWithBackoff deadlock issue
Fixes related to handling of MAX_POLL_INTERVAL_MS_CONFIG during deadlock and CommitFailedException on partition revoked. Author: Sachin Mittal <[email protected]> Reviewers: Matthias J. Sax, Damian Guy, Guozhang Wang Closes apache#2642 from sjmittal/trunk
1 parent 5a2fcdd commit 197a5d5

File tree

7 files changed

+22
-8
lines changed

7 files changed

+22
-8
lines changed

Diff for: clients/.gitignore

+1
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
/bin/

Diff for: connect/api/.gitignore

+1
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
/bin/

Diff for: connect/json/.gitignore

+1
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
/bin/

Diff for: core/.gitignore

+3
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,3 @@
1+
.cache-main
2+
.cache-tests
3+
/bin/

Diff for: streams/.gitignore

+1
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
/bin/

Diff for: streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java

+1
Original file line numberDiff line numberDiff line change
@@ -401,6 +401,7 @@ public class StreamsConfig extends AbstractConfig {
401401
tempConsumerDefaultOverrides.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "1000");
402402
tempConsumerDefaultOverrides.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
403403
tempConsumerDefaultOverrides.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
404+
tempConsumerDefaultOverrides.put(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "300000");
404405

405406
CONSUMER_DEFAULT_OVERRIDES = Collections.unmodifiableMap(tempConsumerDefaultOverrides);
406407
}

Diff for: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java

+14-8
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,8 @@
2626
import org.apache.kafka.clients.producer.Producer;
2727
import org.apache.kafka.common.KafkaException;
2828
import org.apache.kafka.common.TopicPartition;
29+
import org.apache.kafka.common.config.ConfigDef;
30+
import org.apache.kafka.common.config.ConfigDef.Type;
2931
import org.apache.kafka.common.metrics.Metrics;
3032
import org.apache.kafka.common.metrics.Sensor;
3133
import org.apache.kafka.common.metrics.stats.Avg;
@@ -197,6 +199,7 @@ private synchronized void setStateWhenNotInPendingShutdown(final State newState)
197199
private final Map<TaskId, StreamTask> suspendedTasks;
198200
private final Map<TaskId, StandbyTask> suspendedStandbyTasks;
199201
private final Time time;
202+
private final int rebalanceTimeoutMs;
200203
private final long pollTimeMs;
201204
private final long cleanTimeMs;
202205
private final long commitTimeMs;
@@ -290,6 +293,8 @@ public StreamThread(TopologyBuilder builder,
290293
this.standbyRecords = new HashMap<>();
291294

292295
this.stateDirectory = new StateDirectory(applicationId, config.getString(StreamsConfig.STATE_DIR_CONFIG), time);
296+
final Object maxPollInterval = consumerConfigs.get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG);
297+
this.rebalanceTimeoutMs = (Integer) ConfigDef.parseType(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, maxPollInterval, Type.INT);
293298
this.pollTimeMs = config.getLong(StreamsConfig.POLL_MS_CONFIG);
294299
this.commitTimeMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG);
295300
this.cleanTimeMs = config.getLong(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG);
@@ -855,7 +860,7 @@ private void closeNonAssignedSuspendedStandbyTasks() {
855860
}
856861
}
857862

858-
private void addStreamTasks(Collection<TopicPartition> assignment) {
863+
private void addStreamTasks(Collection<TopicPartition> assignment, final long start) {
859864
if (partitionAssignor == null)
860865
throw new IllegalStateException(logPrefix + " Partition assignor has not been initialized while adding stream tasks: this should not happen.");
861866

@@ -893,7 +898,7 @@ private void addStreamTasks(Collection<TopicPartition> assignment) {
893898

894899
// create all newly assigned tasks (guard against race condition with other thread via backoff and retry)
895900
// -> other thread will call removeSuspendedTasks(); eventually
896-
taskCreator.retryWithBackoff(newTasks);
901+
taskCreator.retryWithBackoff(newTasks, start);
897902
}
898903

899904
StandbyTask createStandbyTask(TaskId id, Collection<TopicPartition> partitions) {
@@ -910,7 +915,7 @@ StandbyTask createStandbyTask(TaskId id, Collection<TopicPartition> partitions)
910915
}
911916
}
912917

913-
private void addStandbyTasks() {
918+
private void addStandbyTasks(final long start) {
914919
if (partitionAssignor == null)
915920
throw new IllegalStateException(logPrefix + " Partition assignor has not been initialized while adding standby tasks: this should not happen.");
916921

@@ -937,7 +942,7 @@ private void addStandbyTasks() {
937942

938943
// create all newly assigned standby tasks (guard against race condition with other thread via backoff and retry)
939944
// -> other thread will call removeSuspendedStandbyTasks(); eventually
940-
new StandbyTaskCreator(checkpointedOffsets).retryWithBackoff(newStandbyTasks);
945+
new StandbyTaskCreator(checkpointedOffsets).retryWithBackoff(newStandbyTasks, start);
941946

942947
restoreConsumer.assign(new ArrayList<>(checkpointedOffsets.keySet()));
943948

@@ -1126,7 +1131,7 @@ public void removeAllSensors() {
11261131
}
11271132

11281133
abstract class AbstractTaskCreator {
1129-
void retryWithBackoff(final Map<TaskId, Set<TopicPartition>> tasksToBeCreated) {
1134+
void retryWithBackoff(final Map<TaskId, Set<TopicPartition>> tasksToBeCreated, final long start) {
11301135
long backoffTimeMs = 50L;
11311136
while (true) {
11321137
final Iterator<Map.Entry<TaskId, Set<TopicPartition>>> it = tasksToBeCreated.entrySet().iterator();
@@ -1138,13 +1143,14 @@ void retryWithBackoff(final Map<TaskId, Set<TopicPartition>> tasksToBeCreated) {
11381143
try {
11391144
createTask(taskId, partitions);
11401145
it.remove();
1146+
backoffTimeMs = 50L;
11411147
} catch (final LockException e) {
11421148
// ignore and retry
11431149
log.warn("Could not create task {}. Will retry.", taskId, e);
11441150
}
11451151
}
11461152

1147-
if (tasksToBeCreated.isEmpty()) {
1153+
if (tasksToBeCreated.isEmpty() || time.milliseconds() - start > rebalanceTimeoutMs) {
11481154
break;
11491155
}
11501156

@@ -1207,9 +1213,9 @@ public void onPartitionsAssigned(Collection<TopicPartition> assignment) {
12071213
// will become active or vice versa
12081214
closeNonAssignedSuspendedStandbyTasks();
12091215
closeNonAssignedSuspendedTasks();
1210-
addStreamTasks(assignment);
1216+
addStreamTasks(assignment, start);
12111217
storeChangelogReader.restore();
1212-
addStandbyTasks();
1218+
addStandbyTasks(start);
12131219
streamsMetadataState.onChange(partitionAssignor.getPartitionsByHostState(), partitionAssignor.clusterMetadata());
12141220
lastCleanMs = time.milliseconds(); // start the cleaning cycle
12151221
setStateWhenNotInPendingShutdown(State.RUNNING);

0 commit comments

Comments
 (0)