|
|
|
@ -14,6 +14,7 @@ package org.apache.kafka.clients.consumer.internals;
@@ -14,6 +14,7 @@ package org.apache.kafka.clients.consumer.internals;
|
|
|
|
|
|
|
|
|
|
import org.apache.kafka.clients.ClientResponse; |
|
|
|
|
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; |
|
|
|
|
import org.apache.kafka.clients.consumer.ConsumerWakeupException; |
|
|
|
|
import org.apache.kafka.clients.consumer.OffsetAndMetadata; |
|
|
|
|
import org.apache.kafka.clients.consumer.OffsetCommitCallback; |
|
|
|
|
import org.apache.kafka.common.KafkaException; |
|
|
|
@ -235,7 +236,15 @@ public final class Coordinator implements Closeable {
@@ -235,7 +236,15 @@ public final class Coordinator implements Closeable {
|
|
|
|
|
@Override |
|
|
|
|
public void close() { |
|
|
|
|
// commit offsets prior to closing if auto-commit enabled
|
|
|
|
|
maybeAutoCommitOffsetsSync(); |
|
|
|
|
while (true) { |
|
|
|
|
try { |
|
|
|
|
maybeAutoCommitOffsetsSync(); |
|
|
|
|
return; |
|
|
|
|
} catch (ConsumerWakeupException e) { |
|
|
|
|
// ignore wakeups while closing to ensure we have a chance to commit
|
|
|
|
|
continue; |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private class HeartbeatTask implements DelayedTask { |
|
|
|
@ -430,6 +439,9 @@ public final class Coordinator implements Closeable {
@@ -430,6 +439,9 @@ public final class Coordinator implements Closeable {
|
|
|
|
|
if (autoCommitEnabled) { |
|
|
|
|
try { |
|
|
|
|
commitOffsetsSync(subscriptions.allConsumed()); |
|
|
|
|
} catch (ConsumerWakeupException e) { |
|
|
|
|
// rethrow wakeups since they are triggered by the user
|
|
|
|
|
throw e; |
|
|
|
|
} catch (Exception e) { |
|
|
|
|
// consistent with async auto-commit failures, we do not propagate the exception
|
|
|
|
|
log.error("Auto offset commit failed.", e); |
|
|
|
|