mirror of https://github.com/apache/kafka.git
KAFKA-17480: New consumer commit all consumed should retrieve offsets in background thread (#17150)
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Kirk True <ktrue@confluent.io>, TengYao Chi <kitingiao@gmail.com>
This commit is contained in:
parent
0181073d49
commit
b213c64f97
|
@ -249,7 +249,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
||||||
private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
|
private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
|
||||||
// Last triggered async commit future. Used to wait until all previous async commits are completed.
|
// Last triggered async commit future. Used to wait until all previous async commits are completed.
|
||||||
// We only need to keep track of the last one, since they are guaranteed to complete in order.
|
// We only need to keep track of the last one, since they are guaranteed to complete in order.
|
||||||
private CompletableFuture<Void> lastPendingAsyncCommit = null;
|
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> lastPendingAsyncCommit = null;
|
||||||
|
|
||||||
// currentThread holds the threadId of the current thread accessing the AsyncKafkaConsumer
|
// currentThread holds the threadId of the current thread accessing the AsyncKafkaConsumer
|
||||||
// and is used to prevent multithreaded access
|
// and is used to prevent multithreaded access
|
||||||
|
@ -752,43 +752,43 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void commitAsync(OffsetCommitCallback callback) {
|
public void commitAsync(OffsetCommitCallback callback) {
|
||||||
commitAsync(subscriptions.allConsumed(), callback);
|
commitAsync(Optional.empty(), callback);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void commitAsync(Map<TopicPartition, OffsetAndMetadata> offsets, OffsetCommitCallback callback) {
|
public void commitAsync(Map<TopicPartition, OffsetAndMetadata> offsets, OffsetCommitCallback callback) {
|
||||||
|
commitAsync(Optional.of(offsets), callback);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void commitAsync(Optional<Map<TopicPartition, OffsetAndMetadata>> offsets, OffsetCommitCallback callback) {
|
||||||
acquireAndEnsureOpen();
|
acquireAndEnsureOpen();
|
||||||
try {
|
try {
|
||||||
AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets);
|
AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets);
|
||||||
lastPendingAsyncCommit = commit(asyncCommitEvent).whenComplete((r, t) -> {
|
lastPendingAsyncCommit = commit(asyncCommitEvent).whenComplete((committedOffsets, throwable) -> {
|
||||||
|
|
||||||
if (t == null) {
|
if (throwable == null) {
|
||||||
offsetCommitCallbackInvoker.enqueueInterceptorInvocation(offsets);
|
offsetCommitCallbackInvoker.enqueueInterceptorInvocation(committedOffsets);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (callback == null) {
|
if (callback == null) {
|
||||||
if (t != null) {
|
if (throwable != null) {
|
||||||
log.error("Offset commit with offsets {} failed", offsets, t);
|
log.error("Offset commit with offsets {} failed", committedOffsets, throwable);
|
||||||
}
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
offsetCommitCallbackInvoker.enqueueUserCallbackInvocation(callback, offsets, (Exception) t);
|
offsetCommitCallbackInvoker.enqueueUserCallbackInvocation(callback, committedOffsets, (Exception) throwable);
|
||||||
});
|
});
|
||||||
} finally {
|
} finally {
|
||||||
release();
|
release();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private CompletableFuture<Void> commit(final CommitEvent commitEvent) {
|
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commit(final CommitEvent commitEvent) {
|
||||||
maybeThrowInvalidGroupIdException();
|
maybeThrowInvalidGroupIdException();
|
||||||
offsetCommitCallbackInvoker.executeCallbacks();
|
offsetCommitCallbackInvoker.executeCallbacks();
|
||||||
|
|
||||||
Map<TopicPartition, OffsetAndMetadata> offsets = commitEvent.offsets();
|
if (commitEvent.offsets().isPresent() && commitEvent.offsets().get().isEmpty()) {
|
||||||
log.debug("Committing offsets: {}", offsets);
|
|
||||||
offsets.forEach(this::updateLastSeenEpochIfNewer);
|
|
||||||
|
|
||||||
if (offsets.isEmpty()) {
|
|
||||||
return CompletableFuture.completedFuture(null);
|
return CompletableFuture.completedFuture(null);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -828,7 +828,6 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
||||||
} else {
|
} else {
|
||||||
log.info("Seeking to offset {} for partition {}", offset, partition);
|
log.info("Seeking to offset {} for partition {}", offset, partition);
|
||||||
}
|
}
|
||||||
updateLastSeenEpochIfNewer(partition, offsetAndMetadata);
|
|
||||||
|
|
||||||
Timer timer = time.timer(defaultApiTimeoutMs);
|
Timer timer = time.timer(defaultApiTimeoutMs);
|
||||||
SeekUnvalidatedEvent seekUnvalidatedEventEvent = new SeekUnvalidatedEvent(
|
SeekUnvalidatedEvent seekUnvalidatedEventEvent = new SeekUnvalidatedEvent(
|
||||||
|
@ -914,9 +913,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
||||||
calculateDeadlineMs(time, timeout));
|
calculateDeadlineMs(time, timeout));
|
||||||
wakeupTrigger.setActiveTask(event.future());
|
wakeupTrigger.setActiveTask(event.future());
|
||||||
try {
|
try {
|
||||||
final Map<TopicPartition, OffsetAndMetadata> committedOffsets = applicationEventHandler.addAndGet(event);
|
return applicationEventHandler.addAndGet(event);
|
||||||
committedOffsets.forEach(this::updateLastSeenEpochIfNewer);
|
|
||||||
return committedOffsets;
|
|
||||||
} catch (TimeoutException e) {
|
} catch (TimeoutException e) {
|
||||||
throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the last " +
|
throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the last " +
|
||||||
"committed offset for partitions " + partitions + " could be determined. Try tuning " +
|
"committed offset for partitions " + partitions + " could be determined. Try tuning " +
|
||||||
|
@ -1294,13 +1291,12 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
||||||
|
|
||||||
// Visible for testing
|
// Visible for testing
|
||||||
void commitSyncAllConsumed(final Timer timer) {
|
void commitSyncAllConsumed(final Timer timer) {
|
||||||
Map<TopicPartition, OffsetAndMetadata> allConsumed = subscriptions.allConsumed();
|
log.debug("Sending synchronous auto-commit on closing");
|
||||||
log.debug("Sending synchronous auto-commit of offsets {} on closing", allConsumed);
|
|
||||||
try {
|
try {
|
||||||
commitSync(allConsumed, Duration.ofMillis(timer.remainingMs()));
|
commitSync(Duration.ofMillis(timer.remainingMs()));
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// consistent with async auto-commit failures, we do not propagate the exception
|
// consistent with async auto-commit failures, we do not propagate the exception
|
||||||
log.warn("Synchronous auto-commit of offsets {} failed: {}", allConsumed, e.getMessage());
|
log.warn("Synchronous auto-commit failed", e);
|
||||||
}
|
}
|
||||||
timer.update();
|
timer.update();
|
||||||
}
|
}
|
||||||
|
@ -1318,28 +1314,32 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void commitSync(final Duration timeout) {
|
public void commitSync(final Duration timeout) {
|
||||||
commitSync(subscriptions.allConsumed(), timeout);
|
commitSync(Optional.empty(), timeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets) {
|
public void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets) {
|
||||||
commitSync(offsets, Duration.ofMillis(defaultApiTimeoutMs));
|
commitSync(Optional.of(offsets), Duration.ofMillis(defaultApiTimeoutMs));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets, Duration timeout) {
|
public void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets, Duration timeout) {
|
||||||
|
commitSync(Optional.of(offsets), timeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void commitSync(Optional<Map<TopicPartition, OffsetAndMetadata>> offsets, Duration timeout) {
|
||||||
acquireAndEnsureOpen();
|
acquireAndEnsureOpen();
|
||||||
long commitStart = time.nanoseconds();
|
long commitStart = time.nanoseconds();
|
||||||
try {
|
try {
|
||||||
SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, calculateDeadlineMs(time, timeout));
|
SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, calculateDeadlineMs(time, timeout));
|
||||||
CompletableFuture<Void> commitFuture = commit(syncCommitEvent);
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitFuture = commit(syncCommitEvent);
|
||||||
|
|
||||||
Timer requestTimer = time.timer(timeout.toMillis());
|
Timer requestTimer = time.timer(timeout.toMillis());
|
||||||
awaitPendingAsyncCommitsAndExecuteCommitCallbacks(requestTimer, true);
|
awaitPendingAsyncCommitsAndExecuteCommitCallbacks(requestTimer, true);
|
||||||
|
|
||||||
wakeupTrigger.setActiveTask(commitFuture);
|
wakeupTrigger.setActiveTask(commitFuture);
|
||||||
ConsumerUtils.getResult(commitFuture, requestTimer);
|
Map<TopicPartition, OffsetAndMetadata> committedOffsets = ConsumerUtils.getResult(commitFuture, requestTimer);
|
||||||
interceptors.onCommit(offsets);
|
interceptors.onCommit(committedOffsets);
|
||||||
} finally {
|
} finally {
|
||||||
wakeupTrigger.clearTask();
|
wakeupTrigger.clearTask();
|
||||||
kafkaConsumerMetrics.recordCommitSync(time.nanoseconds() - commitStart);
|
kafkaConsumerMetrics.recordCommitSync(time.nanoseconds() - commitStart);
|
||||||
|
@ -1588,11 +1588,6 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
||||||
return groupMetadata.get().isPresent();
|
return groupMetadata.get().isPresent();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAndMetadata offsetAndMetadata) {
|
|
||||||
if (offsetAndMetadata != null)
|
|
||||||
offsetAndMetadata.leaderEpoch().ifPresent(epoch -> metadata.updateLastSeenEpochIfNewer(topicPartition, epoch));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method signals the background thread to {@link CreateFetchRequestsEvent create fetch requests}.
|
* This method signals the background thread to {@link CreateFetchRequestsEvent create fetch requests}.
|
||||||
*
|
*
|
||||||
|
|
|
@ -72,6 +72,7 @@ import static org.apache.kafka.common.protocol.Errors.COORDINATOR_LOAD_IN_PROGRE
|
||||||
public class CommitRequestManager implements RequestManager, MemberStateListener {
|
public class CommitRequestManager implements RequestManager, MemberStateListener {
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final SubscriptionState subscriptions;
|
private final SubscriptionState subscriptions;
|
||||||
|
private final ConsumerMetadata metadata;
|
||||||
private final LogContext logContext;
|
private final LogContext logContext;
|
||||||
private final Logger log;
|
private final Logger log;
|
||||||
private final Optional<AutoCommitState> autoCommitState;
|
private final Optional<AutoCommitState> autoCommitState;
|
||||||
|
@ -102,15 +103,16 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
private final MemberInfo memberInfo;
|
private final MemberInfo memberInfo;
|
||||||
|
|
||||||
public CommitRequestManager(
|
public CommitRequestManager(
|
||||||
final Time time,
|
final Time time,
|
||||||
final LogContext logContext,
|
final LogContext logContext,
|
||||||
final SubscriptionState subscriptions,
|
final SubscriptionState subscriptions,
|
||||||
final ConsumerConfig config,
|
final ConsumerConfig config,
|
||||||
final CoordinatorRequestManager coordinatorRequestManager,
|
final CoordinatorRequestManager coordinatorRequestManager,
|
||||||
final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker,
|
final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker,
|
||||||
final String groupId,
|
final String groupId,
|
||||||
final Optional<String> groupInstanceId,
|
final Optional<String> groupInstanceId,
|
||||||
final Metrics metrics) {
|
final Metrics metrics,
|
||||||
|
final ConsumerMetadata metadata) {
|
||||||
this(time,
|
this(time,
|
||||||
logContext,
|
logContext,
|
||||||
subscriptions,
|
subscriptions,
|
||||||
|
@ -122,7 +124,8 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG),
|
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG),
|
||||||
config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG),
|
config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG),
|
||||||
OptionalDouble.empty(),
|
OptionalDouble.empty(),
|
||||||
metrics);
|
metrics,
|
||||||
|
metadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Visible for testing
|
// Visible for testing
|
||||||
|
@ -138,7 +141,8 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
final long retryBackoffMs,
|
final long retryBackoffMs,
|
||||||
final long retryBackoffMaxMs,
|
final long retryBackoffMaxMs,
|
||||||
final OptionalDouble jitter,
|
final OptionalDouble jitter,
|
||||||
final Metrics metrics) {
|
final Metrics metrics,
|
||||||
|
final ConsumerMetadata metadata) {
|
||||||
Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets");
|
Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets");
|
||||||
this.time = time;
|
this.time = time;
|
||||||
this.logContext = logContext;
|
this.logContext = logContext;
|
||||||
|
@ -155,6 +159,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
this.groupId = groupId;
|
this.groupId = groupId;
|
||||||
this.groupInstanceId = groupInstanceId;
|
this.groupInstanceId = groupInstanceId;
|
||||||
this.subscriptions = subscriptions;
|
this.subscriptions = subscriptions;
|
||||||
|
this.metadata = metadata;
|
||||||
this.retryBackoffMs = retryBackoffMs;
|
this.retryBackoffMs = retryBackoffMs;
|
||||||
this.retryBackoffMaxMs = retryBackoffMaxMs;
|
this.retryBackoffMaxMs = retryBackoffMaxMs;
|
||||||
this.jitter = jitter;
|
this.jitter = jitter;
|
||||||
|
@ -381,20 +386,22 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
* exceptionally depending on the response. If the request fails with a retriable error, the
|
* exceptionally depending on the response. If the request fails with a retriable error, the
|
||||||
* future will be completed with a {@link RetriableCommitFailedException}.
|
* future will be completed with a {@link RetriableCommitFailedException}.
|
||||||
*/
|
*/
|
||||||
public CompletableFuture<Void> commitAsync(final Map<TopicPartition, OffsetAndMetadata> offsets) {
|
public CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitAsync(final Optional<Map<TopicPartition, OffsetAndMetadata>> offsets) {
|
||||||
if (offsets.isEmpty()) {
|
Map<TopicPartition, OffsetAndMetadata> commitOffsets = offsets.orElseGet(subscriptions::allConsumed);
|
||||||
|
if (commitOffsets.isEmpty()) {
|
||||||
log.debug("Skipping commit of empty offsets");
|
log.debug("Skipping commit of empty offsets");
|
||||||
return CompletableFuture.completedFuture(null);
|
return CompletableFuture.completedFuture(Map.of());
|
||||||
}
|
}
|
||||||
OffsetCommitRequestState commitRequest = createOffsetCommitRequest(offsets, Long.MAX_VALUE);
|
maybeUpdateLastSeenEpochIfNewer(commitOffsets);
|
||||||
|
OffsetCommitRequestState commitRequest = createOffsetCommitRequest(commitOffsets, Long.MAX_VALUE);
|
||||||
pendingRequests.addOffsetCommitRequest(commitRequest);
|
pendingRequests.addOffsetCommitRequest(commitRequest);
|
||||||
|
|
||||||
CompletableFuture<Void> asyncCommitResult = new CompletableFuture<>();
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> asyncCommitResult = new CompletableFuture<>();
|
||||||
commitRequest.future.whenComplete((committedOffsets, error) -> {
|
commitRequest.future.whenComplete((committedOffsets, error) -> {
|
||||||
if (error != null) {
|
if (error != null) {
|
||||||
asyncCommitResult.completeExceptionally(commitAsyncExceptionForError(error));
|
asyncCommitResult.completeExceptionally(commitAsyncExceptionForError(error));
|
||||||
} else {
|
} else {
|
||||||
asyncCommitResult.complete(null);
|
asyncCommitResult.complete(commitOffsets);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
return asyncCommitResult;
|
return asyncCommitResult;
|
||||||
|
@ -403,15 +410,20 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
/**
|
/**
|
||||||
* Commit offsets, retrying on expected retriable errors while the retry timeout hasn't expired.
|
* Commit offsets, retrying on expected retriable errors while the retry timeout hasn't expired.
|
||||||
*
|
*
|
||||||
* @param offsets Offsets to commit
|
* @param offsets Offsets to commit
|
||||||
* @param deadlineMs Time until which the request will be retried if it fails with
|
* @param deadlineMs Time until which the request will be retried if it fails with
|
||||||
* an expected retriable error.
|
* an expected retriable error.
|
||||||
* @return Future that will complete when a successful response
|
* @return Future that will complete when a successful response
|
||||||
*/
|
*/
|
||||||
public CompletableFuture<Void> commitSync(final Map<TopicPartition, OffsetAndMetadata> offsets,
|
public CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitSync(final Optional<Map<TopicPartition, OffsetAndMetadata>> offsets,
|
||||||
final long deadlineMs) {
|
final long deadlineMs) {
|
||||||
CompletableFuture<Void> result = new CompletableFuture<>();
|
Map<TopicPartition, OffsetAndMetadata> commitOffsets = offsets.orElseGet(subscriptions::allConsumed);
|
||||||
OffsetCommitRequestState requestState = createOffsetCommitRequest(offsets, deadlineMs);
|
if (commitOffsets.isEmpty()) {
|
||||||
|
return CompletableFuture.completedFuture(Map.of());
|
||||||
|
}
|
||||||
|
maybeUpdateLastSeenEpochIfNewer(commitOffsets);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> result = new CompletableFuture<>();
|
||||||
|
OffsetCommitRequestState requestState = createOffsetCommitRequest(commitOffsets, deadlineMs);
|
||||||
commitSyncWithRetries(requestState, result);
|
commitSyncWithRetries(requestState, result);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
@ -439,14 +451,14 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
}
|
}
|
||||||
|
|
||||||
private void commitSyncWithRetries(OffsetCommitRequestState requestAttempt,
|
private void commitSyncWithRetries(OffsetCommitRequestState requestAttempt,
|
||||||
CompletableFuture<Void> result) {
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> result) {
|
||||||
pendingRequests.addOffsetCommitRequest(requestAttempt);
|
pendingRequests.addOffsetCommitRequest(requestAttempt);
|
||||||
|
|
||||||
// Retry the same commit request while it fails with RetriableException and the retry
|
// Retry the same commit request while it fails with RetriableException and the retry
|
||||||
// timeout hasn't expired.
|
// timeout hasn't expired.
|
||||||
requestAttempt.future.whenComplete((res, error) -> {
|
requestAttempt.future.whenComplete((res, error) -> {
|
||||||
if (error == null) {
|
if (error == null) {
|
||||||
result.complete(null);
|
result.complete(requestAttempt.offsets);
|
||||||
} else {
|
} else {
|
||||||
if (error instanceof RetriableException) {
|
if (error instanceof RetriableException) {
|
||||||
if (requestAttempt.isExpired()) {
|
if (requestAttempt.isExpired()) {
|
||||||
|
@ -531,6 +543,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
"outbound buffer:" + fetchRequest);
|
"outbound buffer:" + fetchRequest);
|
||||||
}
|
}
|
||||||
if (error == null) {
|
if (error == null) {
|
||||||
|
maybeUpdateLastSeenEpochIfNewer(res);
|
||||||
result.complete(res);
|
result.complete(res);
|
||||||
} else {
|
} else {
|
||||||
if (error instanceof RetriableException || isStaleEpochErrorAndValidEpochAvailable(error)) {
|
if (error instanceof RetriableException || isStaleEpochErrorAndValidEpochAvailable(error)) {
|
||||||
|
@ -615,6 +628,13 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
||||||
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, requests);
|
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, requests);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void maybeUpdateLastSeenEpochIfNewer(final Map<TopicPartition, OffsetAndMetadata> offsets) {
|
||||||
|
offsets.forEach((topicPartition, offsetAndMetadata) -> {
|
||||||
|
if (offsetAndMetadata != null)
|
||||||
|
offsetAndMetadata.leaderEpoch().ifPresent(epoch -> metadata.updateLastSeenEpochIfNewer(topicPartition, epoch));
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
class OffsetCommitRequestState extends RetriableRequestState {
|
class OffsetCommitRequestState extends RetriableRequestState {
|
||||||
private Map<TopicPartition, OffsetAndMetadata> offsets;
|
private Map<TopicPartition, OffsetAndMetadata> offsets;
|
||||||
private final String groupId;
|
private final String groupId;
|
||||||
|
|
|
@ -205,7 +205,8 @@ public class RequestManagers implements Closeable {
|
||||||
offsetCommitCallbackInvoker,
|
offsetCommitCallbackInvoker,
|
||||||
groupRebalanceConfig.groupId,
|
groupRebalanceConfig.groupId,
|
||||||
groupRebalanceConfig.groupInstanceId,
|
groupRebalanceConfig.groupInstanceId,
|
||||||
metrics);
|
metrics,
|
||||||
|
metadata);
|
||||||
membershipManager = new ConsumerMembershipManager(
|
membershipManager = new ConsumerMembershipManager(
|
||||||
groupRebalanceConfig.groupId,
|
groupRebalanceConfig.groupId,
|
||||||
groupRebalanceConfig.groupInstanceId,
|
groupRebalanceConfig.groupInstanceId,
|
||||||
|
|
|
@ -198,29 +198,41 @@ public class ApplicationEventProcessor implements EventProcessor<ApplicationEven
|
||||||
}
|
}
|
||||||
|
|
||||||
private void process(final AsyncCommitEvent event) {
|
private void process(final AsyncCommitEvent event) {
|
||||||
if (!requestManagers.commitRequestManager.isPresent()) {
|
if (requestManagers.commitRequestManager.isEmpty()) {
|
||||||
|
event.future().completeExceptionally(new KafkaException("Unable to async commit " +
|
||||||
|
"offset because the CommitRequestManager is not available. Check if group.id was set correctly"));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
try {
|
||||||
CompletableFuture<Void> future = manager.commitAsync(event.offsets());
|
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
||||||
future.whenComplete(complete(event.future()));
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = manager.commitAsync(event.offsets());
|
||||||
|
future.whenComplete(complete(event.future()));
|
||||||
|
} catch (Exception e) {
|
||||||
|
event.future().completeExceptionally(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void process(final SyncCommitEvent event) {
|
private void process(final SyncCommitEvent event) {
|
||||||
if (!requestManagers.commitRequestManager.isPresent()) {
|
if (requestManagers.commitRequestManager.isEmpty()) {
|
||||||
|
event.future().completeExceptionally(new KafkaException("Unable to sync commit " +
|
||||||
|
"offset because the CommitRequestManager is not available. Check if group.id was set correctly"));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
try {
|
||||||
CompletableFuture<Void> future = manager.commitSync(event.offsets(), event.deadlineMs());
|
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
||||||
future.whenComplete(complete(event.future()));
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = manager.commitSync(event.offsets(), event.deadlineMs());
|
||||||
|
future.whenComplete(complete(event.future()));
|
||||||
|
} catch (Exception e) {
|
||||||
|
event.future().completeExceptionally(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void process(final FetchCommittedOffsetsEvent event) {
|
private void process(final FetchCommittedOffsetsEvent event) {
|
||||||
if (!requestManagers.commitRequestManager.isPresent()) {
|
if (requestManagers.commitRequestManager.isEmpty()) {
|
||||||
event.future().completeExceptionally(new KafkaException("Unable to fetch committed " +
|
event.future().completeExceptionally(new KafkaException("Unable to fetch committed " +
|
||||||
"offset because the CommittedRequestManager is not available. Check if group.id was set correctly"));
|
"offset because the CommitRequestManager is not available. Check if group.id was set correctly"));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
||||||
|
@ -523,6 +535,7 @@ public class ApplicationEventProcessor implements EventProcessor<ApplicationEven
|
||||||
|
|
||||||
private void process(final SeekUnvalidatedEvent event) {
|
private void process(final SeekUnvalidatedEvent event) {
|
||||||
try {
|
try {
|
||||||
|
event.offsetEpoch().ifPresent(epoch -> metadata.updateLastSeenEpochIfNewer(event.partition(), epoch));
|
||||||
SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition(
|
SubscriptionState.FetchPosition newPosition = new SubscriptionState.FetchPosition(
|
||||||
event.offset(),
|
event.offset(),
|
||||||
event.offsetEpoch(),
|
event.offsetEpoch(),
|
||||||
|
|
|
@ -20,13 +20,15 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Event to commit offsets without waiting for a response, so the request won't be retried.
|
* Event to commit offsets without waiting for a response, so the request won't be retried.
|
||||||
|
* If no offsets are provided, this event will commit all consumed offsets.
|
||||||
*/
|
*/
|
||||||
public class AsyncCommitEvent extends CommitEvent {
|
public class AsyncCommitEvent extends CommitEvent {
|
||||||
|
|
||||||
public AsyncCommitEvent(final Map<TopicPartition, OffsetAndMetadata> offsets) {
|
public AsyncCommitEvent(final Optional<Map<TopicPartition, OffsetAndMetadata>> offsets) {
|
||||||
super(Type.COMMIT_ASYNC, offsets, Long.MAX_VALUE);
|
super(Type.COMMIT_ASYNC, offsets, Long.MAX_VALUE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,15 +21,16 @@ import org.apache.kafka.common.TopicPartition;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
public abstract class CommitEvent extends CompletableApplicationEvent<Void> {
|
public abstract class CommitEvent extends CompletableApplicationEvent<Map<TopicPartition, OffsetAndMetadata>> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Offsets to commit per partition.
|
* Offsets to commit per partition.
|
||||||
*/
|
*/
|
||||||
private final Map<TopicPartition, OffsetAndMetadata> offsets;
|
private final Optional<Map<TopicPartition, OffsetAndMetadata>> offsets;
|
||||||
|
|
||||||
protected CommitEvent(final Type type, final Map<TopicPartition, OffsetAndMetadata> offsets, final long deadlineMs) {
|
protected CommitEvent(final Type type, final Optional<Map<TopicPartition, OffsetAndMetadata>> offsets, final long deadlineMs) {
|
||||||
super(type, deadlineMs);
|
super(type, deadlineMs);
|
||||||
this.offsets = validate(offsets);
|
this.offsets = validate(offsets);
|
||||||
}
|
}
|
||||||
|
@ -38,17 +39,21 @@ public abstract class CommitEvent extends CompletableApplicationEvent<Void> {
|
||||||
* Validates the offsets are not negative and then returns the given offset map as
|
* Validates the offsets are not negative and then returns the given offset map as
|
||||||
* {@link Collections#unmodifiableMap(Map) as unmodifiable}.
|
* {@link Collections#unmodifiableMap(Map) as unmodifiable}.
|
||||||
*/
|
*/
|
||||||
private static Map<TopicPartition, OffsetAndMetadata> validate(final Map<TopicPartition, OffsetAndMetadata> offsets) {
|
private static Optional<Map<TopicPartition, OffsetAndMetadata>> validate(final Optional<Map<TopicPartition, OffsetAndMetadata>> offsets) {
|
||||||
for (OffsetAndMetadata offsetAndMetadata : offsets.values()) {
|
if (!offsets.isPresent()) {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
for (OffsetAndMetadata offsetAndMetadata : offsets.get().values()) {
|
||||||
if (offsetAndMetadata.offset() < 0) {
|
if (offsetAndMetadata.offset() < 0) {
|
||||||
throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset());
|
throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return Collections.unmodifiableMap(offsets);
|
return Optional.of(Collections.unmodifiableMap(offsets.get()));
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<TopicPartition, OffsetAndMetadata> offsets() {
|
public Optional<Map<TopicPartition, OffsetAndMetadata>> offsets() {
|
||||||
return offsets;
|
return offsets;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,14 +20,15 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Event to commit offsets waiting for a response and retrying on expected retriable errors until
|
* Event to commit offsets waiting for a response and retrying on expected retriable errors until
|
||||||
* the timer expires.
|
* the timer expires. If no offsets are provided, this event will commit all consumed offsets.
|
||||||
*/
|
*/
|
||||||
public class SyncCommitEvent extends CommitEvent {
|
public class SyncCommitEvent extends CommitEvent {
|
||||||
|
|
||||||
public SyncCommitEvent(final Map<TopicPartition, OffsetAndMetadata> offsets, final long deadlineMs) {
|
public SyncCommitEvent(final Optional<Map<TopicPartition, OffsetAndMetadata>> offsets, final long deadlineMs) {
|
||||||
super(Type.COMMIT_SYNC, offsets, deadlineMs);
|
super(Type.COMMIT_SYNC, offsets, deadlineMs);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -54,7 +54,6 @@ import org.apache.kafka.clients.consumer.internals.events.TopicSubscriptionChang
|
||||||
import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent;
|
import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent;
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.KafkaException;
|
||||||
import org.apache.kafka.common.Metric;
|
import org.apache.kafka.common.Metric;
|
||||||
import org.apache.kafka.common.Node;
|
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.config.ConfigException;
|
import org.apache.kafka.common.config.ConfigException;
|
||||||
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
||||||
|
@ -185,8 +184,22 @@ public class AsyncKafkaConsumerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
private AsyncKafkaConsumer<String, String> newConsumer(Properties props) {
|
private AsyncKafkaConsumer<String, String> newConsumer(Properties props) {
|
||||||
|
// disable auto-commit by default, so we don't need to handle SyncCommitEvent for each case
|
||||||
|
if (!props.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
|
||||||
|
props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
|
||||||
|
}
|
||||||
final ConsumerConfig config = new ConsumerConfig(props);
|
final ConsumerConfig config = new ConsumerConfig(props);
|
||||||
return newConsumer(config);
|
return new AsyncKafkaConsumer<>(
|
||||||
|
config,
|
||||||
|
new StringDeserializer(),
|
||||||
|
new StringDeserializer(),
|
||||||
|
time,
|
||||||
|
(a, b, c, d, e, f, g) -> applicationEventHandler,
|
||||||
|
a -> backgroundEventReaper,
|
||||||
|
(a, b, c, d, e, f, g) -> fetchCollector,
|
||||||
|
(a, b, c, d) -> metadata,
|
||||||
|
backgroundEventQueue
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private AsyncKafkaConsumer<String, String> newConsumer(ConsumerConfig config) {
|
private AsyncKafkaConsumer<String, String> newConsumer(ConsumerConfig config) {
|
||||||
|
@ -209,10 +222,10 @@ public class AsyncKafkaConsumerTest {
|
||||||
ConsumerRebalanceListenerInvoker rebalanceListenerInvoker,
|
ConsumerRebalanceListenerInvoker rebalanceListenerInvoker,
|
||||||
SubscriptionState subscriptions,
|
SubscriptionState subscriptions,
|
||||||
String groupId,
|
String groupId,
|
||||||
String clientId) {
|
String clientId,
|
||||||
|
boolean autoCommitEnabled) {
|
||||||
long retryBackoffMs = 100L;
|
long retryBackoffMs = 100L;
|
||||||
int defaultApiTimeoutMs = 1000;
|
int defaultApiTimeoutMs = 1000;
|
||||||
boolean autoCommitEnabled = true;
|
|
||||||
return new AsyncKafkaConsumer<>(
|
return new AsyncKafkaConsumer<>(
|
||||||
new LogContext(),
|
new LogContext(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -281,8 +294,10 @@ public class AsyncKafkaConsumerTest {
|
||||||
final ArgumentCaptor<AsyncCommitEvent> commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitEvent.class);
|
final ArgumentCaptor<AsyncCommitEvent> commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitEvent.class);
|
||||||
verify(applicationEventHandler).add(commitEventCaptor.capture());
|
verify(applicationEventHandler).add(commitEventCaptor.capture());
|
||||||
final AsyncCommitEvent commitEvent = commitEventCaptor.getValue();
|
final AsyncCommitEvent commitEvent = commitEventCaptor.getValue();
|
||||||
assertEquals(offsets, commitEvent.offsets());
|
assertTrue(commitEvent.offsets().isPresent());
|
||||||
assertDoesNotThrow(() -> commitEvent.future().complete(null));
|
assertEquals(offsets, commitEvent.offsets().get());
|
||||||
|
|
||||||
|
commitEvent.future().complete(offsets);
|
||||||
assertDoesNotThrow(() -> consumer.commitAsync(offsets, null));
|
assertDoesNotThrow(() -> consumer.commitAsync(offsets, null));
|
||||||
|
|
||||||
// Clean-up. Close the consumer here as we know it will cause a TimeoutException to be thrown.
|
// Clean-up. Close the consumer here as we know it will cause a TimeoutException to be thrown.
|
||||||
|
@ -347,25 +362,6 @@ public class AsyncKafkaConsumerTest {
|
||||||
assertTrue((double) metric.metricValue() > 0);
|
assertTrue((double) metric.metricValue() > 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCommittedLeaderEpochUpdate() {
|
|
||||||
consumer = newConsumer();
|
|
||||||
final TopicPartition t0 = new TopicPartition("t0", 2);
|
|
||||||
final TopicPartition t1 = new TopicPartition("t0", 3);
|
|
||||||
final TopicPartition t2 = new TopicPartition("t0", 4);
|
|
||||||
HashMap<TopicPartition, OffsetAndMetadata> topicPartitionOffsets = new HashMap<>();
|
|
||||||
topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), ""));
|
|
||||||
topicPartitionOffsets.put(t1, null);
|
|
||||||
topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), ""));
|
|
||||||
completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets);
|
|
||||||
|
|
||||||
assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000)));
|
|
||||||
|
|
||||||
verify(metadata).updateLastSeenEpochIfNewer(t0, 2);
|
|
||||||
verify(metadata).updateLastSeenEpochIfNewer(t2, 3);
|
|
||||||
verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCommittedExceptionThrown() {
|
public void testCommittedExceptionThrown() {
|
||||||
consumer = newConsumer();
|
consumer = newConsumer();
|
||||||
|
@ -388,7 +384,6 @@ public class AsyncKafkaConsumerTest {
|
||||||
final TopicPartition tp = new TopicPartition(topicName, partition);
|
final TopicPartition tp = new TopicPartition(topicName, partition);
|
||||||
doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
|
doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
|
||||||
when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true);
|
when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true);
|
||||||
completeCommitSyncApplicationEventSuccessfully();
|
|
||||||
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
||||||
completeAssignmentChangeEventSuccessfully();
|
completeAssignmentChangeEventSuccessfully();
|
||||||
consumer.assign(singleton(tp));
|
consumer.assign(singleton(tp));
|
||||||
|
@ -410,7 +405,6 @@ public class AsyncKafkaConsumerTest {
|
||||||
return Fetch.empty();
|
return Fetch.empty();
|
||||||
}).doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
|
}).doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
|
||||||
when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true);
|
when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true);
|
||||||
completeCommitSyncApplicationEventSuccessfully();
|
|
||||||
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
||||||
completeAssignmentChangeEventSuccessfully();
|
completeAssignmentChangeEventSuccessfully();
|
||||||
consumer.assign(singleton(tp));
|
consumer.assign(singleton(tp));
|
||||||
|
@ -434,7 +428,6 @@ public class AsyncKafkaConsumerTest {
|
||||||
return Fetch.forPartition(tp, records, true, new OffsetAndMetadata(4, Optional.of(0), ""));
|
return Fetch.forPartition(tp, records, true, new OffsetAndMetadata(4, Optional.of(0), ""));
|
||||||
}).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class));
|
}).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class));
|
||||||
when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true);
|
when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true);
|
||||||
completeCommitSyncApplicationEventSuccessfully();
|
|
||||||
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
||||||
completeAssignmentChangeEventSuccessfully();
|
completeAssignmentChangeEventSuccessfully();
|
||||||
consumer.assign(singleton(tp));
|
consumer.assign(singleton(tp));
|
||||||
|
@ -492,7 +485,6 @@ public class AsyncKafkaConsumerTest {
|
||||||
doReturn(Fetch.forPartition(tp, records, true, new OffsetAndMetadata(4, Optional.of(0), "")))
|
doReturn(Fetch.forPartition(tp, records, true, new OffsetAndMetadata(4, Optional.of(0), "")))
|
||||||
.when(fetchCollector).collectFetch(any(FetchBuffer.class));
|
.when(fetchCollector).collectFetch(any(FetchBuffer.class));
|
||||||
when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true);
|
when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true);
|
||||||
completeCommitSyncApplicationEventSuccessfully();
|
|
||||||
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
||||||
completeAssignmentChangeEventSuccessfully();
|
completeAssignmentChangeEventSuccessfully();
|
||||||
consumer.assign(singleton(tp));
|
consumer.assign(singleton(tp));
|
||||||
|
@ -527,71 +519,6 @@ public class AsyncKafkaConsumerTest {
|
||||||
assertThrows(callbackException.getClass(), () -> consumer.commitSync());
|
assertThrows(callbackException.getClass(), () -> consumer.commitSync());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCommitSyncLeaderEpochUpdate() {
|
|
||||||
consumer = newConsumer();
|
|
||||||
final TopicPartition t0 = new TopicPartition("t0", 2);
|
|
||||||
final TopicPartition t1 = new TopicPartition("t0", 3);
|
|
||||||
HashMap<TopicPartition, OffsetAndMetadata> topicPartitionOffsets = new HashMap<>();
|
|
||||||
topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), ""));
|
|
||||||
topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), ""));
|
|
||||||
completeCommitSyncApplicationEventSuccessfully();
|
|
||||||
|
|
||||||
completeAssignmentChangeEventSuccessfully();
|
|
||||||
consumer.assign(Arrays.asList(t0, t1));
|
|
||||||
|
|
||||||
assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets));
|
|
||||||
|
|
||||||
verify(metadata).updateLastSeenEpochIfNewer(t0, 2);
|
|
||||||
verify(metadata).updateLastSeenEpochIfNewer(t1, 1);
|
|
||||||
verify(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitEvent.class));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCommitAsyncLeaderEpochUpdate() {
|
|
||||||
SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
|
|
||||||
consumer = newConsumer(
|
|
||||||
mock(FetchBuffer.class),
|
|
||||||
new ConsumerInterceptors<>(Collections.emptyList()),
|
|
||||||
mock(ConsumerRebalanceListenerInvoker.class),
|
|
||||||
subscriptions,
|
|
||||||
"group-id",
|
|
||||||
"client-id");
|
|
||||||
completeCommitSyncApplicationEventSuccessfully();
|
|
||||||
final TopicPartition t0 = new TopicPartition("t0", 2);
|
|
||||||
final TopicPartition t1 = new TopicPartition("t0", 3);
|
|
||||||
HashMap<TopicPartition, OffsetAndMetadata> topicPartitionOffsets = new HashMap<>();
|
|
||||||
topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), ""));
|
|
||||||
topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), ""));
|
|
||||||
when(metadata.currentLeader(t0)).thenReturn(
|
|
||||||
new LeaderAndEpoch(Optional.of(
|
|
||||||
new Node(1, "host", 9000)), Optional.of(1)));
|
|
||||||
when(metadata.currentLeader(t1)).thenReturn(
|
|
||||||
new LeaderAndEpoch(Optional.of(
|
|
||||||
new Node(1, "host", 9000)), Optional.of(1)));
|
|
||||||
completeAssignmentChangeEventSuccessfully();
|
|
||||||
consumer.assign(Arrays.asList(t0, t1));
|
|
||||||
completeSeekUnvalidatedEventSuccessfully();
|
|
||||||
consumer.seek(t0, 10);
|
|
||||||
consumer.seek(t1, 20);
|
|
||||||
|
|
||||||
MockCommitCallback callback = new MockCommitCallback();
|
|
||||||
assertDoesNotThrow(() -> consumer.commitAsync(topicPartitionOffsets, callback));
|
|
||||||
|
|
||||||
verify(metadata).updateLastSeenEpochIfNewer(t0, 2);
|
|
||||||
verify(metadata).updateLastSeenEpochIfNewer(t1, 1);
|
|
||||||
verify(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitEvent.class));
|
|
||||||
|
|
||||||
// Clean-Up. Close the consumer here as we know it will cause a TimeoutException to be thrown.
|
|
||||||
// If we get an error *other* than the TimeoutException, we'll fail the test.
|
|
||||||
try {
|
|
||||||
Exception e = assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO));
|
|
||||||
assertInstanceOf(TimeoutException.class, e.getCause());
|
|
||||||
} finally {
|
|
||||||
consumer = null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCommitSyncAwaitsCommitAsyncCompletionWithEmptyOffsets() {
|
public void testCommitSyncAwaitsCommitAsyncCompletionWithEmptyOffsets() {
|
||||||
final TopicPartition tp = new TopicPartition("foo", 0);
|
final TopicPartition tp = new TopicPartition("foo", 0);
|
||||||
|
@ -722,7 +649,8 @@ public class AsyncKafkaConsumerTest {
|
||||||
mock(ConsumerRebalanceListenerInvoker.class),
|
mock(ConsumerRebalanceListenerInvoker.class),
|
||||||
subscriptions,
|
subscriptions,
|
||||||
"group-id",
|
"group-id",
|
||||||
"client-id"));
|
"client-id",
|
||||||
|
false));
|
||||||
completeUnsubscribeApplicationEventSuccessfully();
|
completeUnsubscribeApplicationEventSuccessfully();
|
||||||
consumer.close(Duration.ZERO);
|
consumer.close(Duration.ZERO);
|
||||||
verifyUnsubscribeEvent(subscriptions);
|
verifyUnsubscribeEvent(subscriptions);
|
||||||
|
@ -739,7 +667,8 @@ public class AsyncKafkaConsumerTest {
|
||||||
mock(ConsumerRebalanceListenerInvoker.class),
|
mock(ConsumerRebalanceListenerInvoker.class),
|
||||||
subscriptions,
|
subscriptions,
|
||||||
"group-id",
|
"group-id",
|
||||||
"client-id"));
|
"client-id",
|
||||||
|
false));
|
||||||
doThrow(new KafkaException()).when(consumer).processBackgroundEvents(any(), any(), any());
|
doThrow(new KafkaException()).when(consumer).processBackgroundEvents(any(), any(), any());
|
||||||
assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO));
|
assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO));
|
||||||
verifyUnsubscribeEvent(subscriptions);
|
verifyUnsubscribeEvent(subscriptions);
|
||||||
|
@ -748,8 +677,7 @@ public class AsyncKafkaConsumerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAutoCommitSyncEnabled() {
|
public void testCommitSyncAllConsumed() {
|
||||||
completeCommitSyncApplicationEventSuccessfully();
|
|
||||||
SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
|
SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
|
||||||
consumer = newConsumer(
|
consumer = newConsumer(
|
||||||
mock(FetchBuffer.class),
|
mock(FetchBuffer.class),
|
||||||
|
@ -757,14 +685,19 @@ public class AsyncKafkaConsumerTest {
|
||||||
mock(ConsumerRebalanceListenerInvoker.class),
|
mock(ConsumerRebalanceListenerInvoker.class),
|
||||||
subscriptions,
|
subscriptions,
|
||||||
"group-id",
|
"group-id",
|
||||||
"client-id");
|
"client-id",
|
||||||
|
false);
|
||||||
completeTopicSubscriptionChangeEventSuccessfully();
|
completeTopicSubscriptionChangeEventSuccessfully();
|
||||||
consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class));
|
consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class));
|
||||||
subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0)));
|
subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0)));
|
||||||
completeSeekUnvalidatedEventSuccessfully();
|
completeSeekUnvalidatedEventSuccessfully();
|
||||||
subscriptions.seek(new TopicPartition("topic", 0), 100);
|
subscriptions.seek(new TopicPartition("topic", 0), 100);
|
||||||
consumer.commitSyncAllConsumed(time.timer(100));
|
consumer.commitSyncAllConsumed(time.timer(100));
|
||||||
verify(applicationEventHandler).add(any(SyncCommitEvent.class));
|
|
||||||
|
ArgumentCaptor<SyncCommitEvent> eventCaptor = ArgumentCaptor.forClass(SyncCommitEvent.class);
|
||||||
|
verify(applicationEventHandler).add(eventCaptor.capture());
|
||||||
|
SyncCommitEvent capturedEvent = eventCaptor.getValue();
|
||||||
|
assertFalse(capturedEvent.offsets().isPresent(), "Expected empty optional offsets");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -776,12 +709,15 @@ public class AsyncKafkaConsumerTest {
|
||||||
mock(ConsumerRebalanceListenerInvoker.class),
|
mock(ConsumerRebalanceListenerInvoker.class),
|
||||||
subscriptions,
|
subscriptions,
|
||||||
"group-id",
|
"group-id",
|
||||||
"client-id");
|
"client-id",
|
||||||
|
false);
|
||||||
completeTopicSubscriptionChangeEventSuccessfully();
|
completeTopicSubscriptionChangeEventSuccessfully();
|
||||||
consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class));
|
consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class));
|
||||||
subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0)));
|
subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0)));
|
||||||
completeSeekUnvalidatedEventSuccessfully();
|
completeSeekUnvalidatedEventSuccessfully();
|
||||||
subscriptions.seek(new TopicPartition("topic", 0), 100);
|
subscriptions.seek(new TopicPartition("topic", 0), 100);
|
||||||
|
completeUnsubscribeApplicationEventSuccessfully();
|
||||||
|
consumer.close();
|
||||||
verify(applicationEventHandler, never()).add(any(SyncCommitEvent.class));
|
verify(applicationEventHandler, never()).add(any(SyncCommitEvent.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1035,7 +971,9 @@ public class AsyncKafkaConsumerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testNoWakeupInCloseCommit() {
|
public void testNoWakeupInCloseCommit() {
|
||||||
TopicPartition tp = new TopicPartition("topic1", 0);
|
TopicPartition tp = new TopicPartition("topic1", 0);
|
||||||
consumer = newConsumer();
|
Properties props = requiredConsumerConfigAndGroupId("consumer-group");
|
||||||
|
props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, true);
|
||||||
|
consumer = newConsumer(props);
|
||||||
completeAssignmentChangeEventSuccessfully();
|
completeAssignmentChangeEventSuccessfully();
|
||||||
consumer.assign(Collections.singleton(tp));
|
consumer.assign(Collections.singleton(tp));
|
||||||
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
|
||||||
|
@ -1280,8 +1218,7 @@ public class AsyncKafkaConsumerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() {
|
public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() {
|
||||||
final String groupId = "consumerGroupA";
|
final String groupId = "consumerGroupA";
|
||||||
final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
|
consumer = newConsumer(requiredConsumerConfigAndGroupId(groupId));
|
||||||
consumer = newConsumer(config);
|
|
||||||
|
|
||||||
final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata();
|
final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata();
|
||||||
|
|
||||||
|
@ -1297,8 +1234,7 @@ public class AsyncKafkaConsumerTest {
|
||||||
final String groupInstanceId = "groupInstanceId1";
|
final String groupInstanceId = "groupInstanceId1";
|
||||||
final Properties props = requiredConsumerConfigAndGroupId(groupId);
|
final Properties props = requiredConsumerConfigAndGroupId(groupId);
|
||||||
props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId);
|
props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId);
|
||||||
final ConsumerConfig config = new ConsumerConfig(props);
|
consumer = newConsumer(props);
|
||||||
consumer = newConsumer(config);
|
|
||||||
|
|
||||||
final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata();
|
final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata();
|
||||||
|
|
||||||
|
@ -1333,9 +1269,8 @@ public class AsyncKafkaConsumerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testGroupMetadataUpdate() {
|
public void testGroupMetadataUpdate() {
|
||||||
final String groupId = "consumerGroupA";
|
final String groupId = "consumerGroupA";
|
||||||
final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
|
|
||||||
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
|
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
|
||||||
consumer = newConsumer(config);
|
consumer = newConsumer(requiredConsumerConfigAndGroupId(groupId));
|
||||||
final ConsumerGroupMetadata oldGroupMetadata = consumer.groupMetadata();
|
final ConsumerGroupMetadata oldGroupMetadata = consumer.groupMetadata();
|
||||||
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
|
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
|
||||||
final int expectedMemberEpoch = 42;
|
final int expectedMemberEpoch = 42;
|
||||||
|
@ -1355,9 +1290,8 @@ public class AsyncKafkaConsumerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testGroupMetadataIsResetAfterUnsubscribe() {
|
public void testGroupMetadataIsResetAfterUnsubscribe() {
|
||||||
final String groupId = "consumerGroupA";
|
final String groupId = "consumerGroupA";
|
||||||
final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
|
|
||||||
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
|
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
|
||||||
consumer = newConsumer(config);
|
consumer = newConsumer(requiredConsumerConfigAndGroupId(groupId));
|
||||||
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
|
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
|
||||||
consumer.subscribe(singletonList("topic"));
|
consumer.subscribe(singletonList("topic"));
|
||||||
final int memberEpoch = 42;
|
final int memberEpoch = 42;
|
||||||
|
@ -1479,8 +1413,7 @@ public class AsyncKafkaConsumerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testBackgroundError() {
|
public void testBackgroundError() {
|
||||||
final String groupId = "consumerGroupA";
|
final String groupId = "consumerGroupA";
|
||||||
final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
|
consumer = newConsumer(requiredConsumerConfigAndGroupId(groupId));
|
||||||
consumer = newConsumer(config);
|
|
||||||
|
|
||||||
final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition");
|
final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition");
|
||||||
final ErrorEvent errorEvent = new ErrorEvent(expectedException);
|
final ErrorEvent errorEvent = new ErrorEvent(expectedException);
|
||||||
|
@ -1495,8 +1428,7 @@ public class AsyncKafkaConsumerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testMultipleBackgroundErrors() {
|
public void testMultipleBackgroundErrors() {
|
||||||
final String groupId = "consumerGroupA";
|
final String groupId = "consumerGroupA";
|
||||||
final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
|
consumer = newConsumer(requiredConsumerConfigAndGroupId(groupId));
|
||||||
consumer = newConsumer(config);
|
|
||||||
|
|
||||||
final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition");
|
final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition");
|
||||||
final ErrorEvent errorEvent1 = new ErrorEvent(expectedException1);
|
final ErrorEvent errorEvent1 = new ErrorEvent(expectedException1);
|
||||||
|
@ -1589,7 +1521,8 @@ public class AsyncKafkaConsumerTest {
|
||||||
mock(ConsumerRebalanceListenerInvoker.class),
|
mock(ConsumerRebalanceListenerInvoker.class),
|
||||||
subscriptions,
|
subscriptions,
|
||||||
"group-id",
|
"group-id",
|
||||||
"client-id");
|
"client-id",
|
||||||
|
false);
|
||||||
final TopicPartition tp = new TopicPartition("topic", 0);
|
final TopicPartition tp = new TopicPartition("topic", 0);
|
||||||
final List<ConsumerRecord<String, String>> records = singletonList(
|
final List<ConsumerRecord<String, String>> records = singletonList(
|
||||||
new ConsumerRecord<>("topic", 0, 2, "key1", "value1"));
|
new ConsumerRecord<>("topic", 0, 2, "key1", "value1"));
|
||||||
|
|
|
@ -87,6 +87,7 @@ import static org.mockito.ArgumentMatchers.any;
|
||||||
import static org.mockito.ArgumentMatchers.anyLong;
|
import static org.mockito.ArgumentMatchers.anyLong;
|
||||||
import static org.mockito.ArgumentMatchers.eq;
|
import static org.mockito.ArgumentMatchers.eq;
|
||||||
import static org.mockito.Mockito.clearInvocations;
|
import static org.mockito.Mockito.clearInvocations;
|
||||||
|
import static org.mockito.Mockito.doReturn;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.never;
|
import static org.mockito.Mockito.never;
|
||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
|
@ -103,6 +104,7 @@ public class CommitRequestManagerTest {
|
||||||
private static final String DEFAULT_GROUP_INSTANCE_ID = "group-instance-id";
|
private static final String DEFAULT_GROUP_INSTANCE_ID = "group-instance-id";
|
||||||
private final Node mockedNode = new Node(1, "host1", 9092);
|
private final Node mockedNode = new Node(1, "host1", 9092);
|
||||||
private SubscriptionState subscriptionState;
|
private SubscriptionState subscriptionState;
|
||||||
|
private ConsumerMetadata metadata;
|
||||||
private LogContext logContext;
|
private LogContext logContext;
|
||||||
private MockTime time;
|
private MockTime time;
|
||||||
private CoordinatorRequestManager coordinatorRequestManager;
|
private CoordinatorRequestManager coordinatorRequestManager;
|
||||||
|
@ -118,6 +120,7 @@ public class CommitRequestManagerTest {
|
||||||
this.logContext = new LogContext();
|
this.logContext = new LogContext();
|
||||||
this.time = new MockTime(0);
|
this.time = new MockTime(0);
|
||||||
this.subscriptionState = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST);
|
this.subscriptionState = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST);
|
||||||
|
this.metadata = mock(ConsumerMetadata.class);
|
||||||
this.coordinatorRequestManager = mock(CoordinatorRequestManager.class);
|
this.coordinatorRequestManager = mock(CoordinatorRequestManager.class);
|
||||||
this.offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class);
|
this.offsetCommitCallbackInvoker = mock(OffsetCommitCallbackInvoker.class);
|
||||||
this.props = new Properties();
|
this.props = new Properties();
|
||||||
|
@ -142,7 +145,8 @@ public class CommitRequestManagerTest {
|
||||||
retryBackoffMs,
|
retryBackoffMs,
|
||||||
retryBackoffMaxMs,
|
retryBackoffMaxMs,
|
||||||
OptionalDouble.of(0),
|
OptionalDouble.of(0),
|
||||||
metrics);
|
metrics,
|
||||||
|
metadata);
|
||||||
|
|
||||||
commitRequestManager.onMemberEpochUpdated(Optional.of(1), Uuid.randomUuid().toString());
|
commitRequestManager.onMemberEpochUpdated(Optional.of(1), Uuid.randomUuid().toString());
|
||||||
Set<TopicPartition> requestedPartitions = Collections.singleton(new TopicPartition("topic-1", 1));
|
Set<TopicPartition> requestedPartitions = Collections.singleton(new TopicPartition("topic-1", 1));
|
||||||
|
@ -175,7 +179,7 @@ public class CommitRequestManagerTest {
|
||||||
|
|
||||||
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
|
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
|
||||||
offsets.put(new TopicPartition("t1", 0), new OffsetAndMetadata(0));
|
offsets.put(new TopicPartition("t1", 0), new OffsetAndMetadata(0));
|
||||||
commitRequestManager.commitAsync(offsets);
|
commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
assertPoll(false, 0, commitRequestManager);
|
assertPoll(false, 0, commitRequestManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -186,7 +190,7 @@ public class CommitRequestManagerTest {
|
||||||
|
|
||||||
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
|
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
|
||||||
offsets.put(new TopicPartition("t1", 0), new OffsetAndMetadata(0));
|
offsets.put(new TopicPartition("t1", 0), new OffsetAndMetadata(0));
|
||||||
commitRequestManager.commitAsync(offsets);
|
commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
assertPoll(false, 0, commitRequestManager);
|
assertPoll(false, 0, commitRequestManager);
|
||||||
assertPoll(true, 1, commitRequestManager);
|
assertPoll(true, 1, commitRequestManager);
|
||||||
}
|
}
|
||||||
|
@ -198,7 +202,7 @@ public class CommitRequestManagerTest {
|
||||||
|
|
||||||
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
|
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
|
||||||
offsets.put(new TopicPartition("t1", 0), new OffsetAndMetadata(0));
|
offsets.put(new TopicPartition("t1", 0), new OffsetAndMetadata(0));
|
||||||
commitRequestManager.commitAsync(offsets);
|
commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
assertPoll(1, commitRequestManager);
|
assertPoll(1, commitRequestManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -239,9 +243,9 @@ public class CommitRequestManagerTest {
|
||||||
|
|
||||||
// Add the requests to the CommitRequestManager and store their futures
|
// Add the requests to the CommitRequestManager and store their futures
|
||||||
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
|
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
|
||||||
commitManager.commitSync(offsets1, deadlineMs);
|
commitManager.commitSync(Optional.of(offsets1), deadlineMs);
|
||||||
commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)), deadlineMs);
|
commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)), deadlineMs);
|
||||||
commitManager.commitSync(offsets2, deadlineMs);
|
commitManager.commitSync(Optional.of(offsets2), deadlineMs);
|
||||||
commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 1)), deadlineMs);
|
commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 1)), deadlineMs);
|
||||||
|
|
||||||
// Poll the CommitRequestManager and verify that the inflightOffsetFetches size is correct
|
// Poll the CommitRequestManager and verify that the inflightOffsetFetches size is correct
|
||||||
|
@ -274,13 +278,146 @@ public class CommitRequestManagerTest {
|
||||||
Map<TopicPartition, OffsetAndMetadata> offsets = Collections.singletonMap(
|
Map<TopicPartition, OffsetAndMetadata> offsets = Collections.singletonMap(
|
||||||
new TopicPartition("topic", 1),
|
new TopicPartition("topic", 1),
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
commitRequestManager.commitAsync(offsets);
|
commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
assertEquals(1, commitRequestManager.unsentOffsetCommitRequests().size());
|
assertEquals(1, commitRequestManager.unsentOffsetCommitRequests().size());
|
||||||
assertEquals(1, commitRequestManager.poll(time.milliseconds()).unsentRequests.size());
|
assertEquals(1, commitRequestManager.poll(time.milliseconds()).unsentRequests.size());
|
||||||
assertTrue(commitRequestManager.unsentOffsetCommitRequests().isEmpty());
|
assertTrue(commitRequestManager.unsentOffsetCommitRequests().isEmpty());
|
||||||
assertEmptyPendingRequests(commitRequestManager);
|
assertEmptyPendingRequests(commitRequestManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitSync() {
|
||||||
|
subscriptionState = mock(SubscriptionState.class);
|
||||||
|
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
|
||||||
|
TopicPartition tp = new TopicPartition("topic", 1);
|
||||||
|
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(0, Optional.of(1), "");
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> offsets = Map.of(tp, offsetAndMetadata);
|
||||||
|
|
||||||
|
CommitRequestManager commitRequestManager = create(false, 100);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = commitRequestManager.commitSync(
|
||||||
|
Optional.of(offsets), time.milliseconds() + defaultApiTimeoutMs);
|
||||||
|
assertEquals(1, commitRequestManager.unsentOffsetCommitRequests().size());
|
||||||
|
List<NetworkClientDelegate.FutureCompletionHandler> pollResults = assertPoll(1, commitRequestManager);
|
||||||
|
pollResults.forEach(v -> v.onComplete(mockOffsetCommitResponse(
|
||||||
|
"topic",
|
||||||
|
1,
|
||||||
|
(short) 1,
|
||||||
|
Errors.NONE)));
|
||||||
|
|
||||||
|
verify(subscriptionState, never()).allConsumed();
|
||||||
|
verify(metadata).updateLastSeenEpochIfNewer(tp, 1);
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> commitOffsets = assertDoesNotThrow(() -> future.get());
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertEquals(offsets, commitOffsets);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitSyncWithEmptyOffsets() {
|
||||||
|
subscriptionState = mock(SubscriptionState.class);
|
||||||
|
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
|
||||||
|
TopicPartition tp = new TopicPartition("topic", 1);
|
||||||
|
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(0, Optional.of(1), "");
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> offsets = Map.of(tp, offsetAndMetadata);
|
||||||
|
doReturn(offsets).when(subscriptionState).allConsumed();
|
||||||
|
|
||||||
|
CommitRequestManager commitRequestManager = create(false, 100);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = commitRequestManager.commitSync(
|
||||||
|
Optional.empty(), time.milliseconds() + defaultApiTimeoutMs);
|
||||||
|
assertEquals(1, commitRequestManager.unsentOffsetCommitRequests().size());
|
||||||
|
List<NetworkClientDelegate.FutureCompletionHandler> pollResults = assertPoll(1, commitRequestManager);
|
||||||
|
pollResults.forEach(v -> v.onComplete(mockOffsetCommitResponse(
|
||||||
|
"topic",
|
||||||
|
1,
|
||||||
|
(short) 1,
|
||||||
|
Errors.NONE)));
|
||||||
|
|
||||||
|
verify(subscriptionState).allConsumed();
|
||||||
|
verify(metadata).updateLastSeenEpochIfNewer(tp, 1);
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> commitOffsets = assertDoesNotThrow(() -> future.get());
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertEquals(offsets, commitOffsets);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitSyncWithEmptyAllConsumedOffsets() {
|
||||||
|
subscriptionState = mock(SubscriptionState.class);
|
||||||
|
doReturn(Map.of()).when(subscriptionState).allConsumed();
|
||||||
|
|
||||||
|
CommitRequestManager commitRequestManager = create(true, 100);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = commitRequestManager.commitSync(
|
||||||
|
Optional.empty(), time.milliseconds() + defaultApiTimeoutMs);
|
||||||
|
|
||||||
|
verify(subscriptionState).allConsumed();
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> commitOffsets = assertDoesNotThrow(() -> future.get());
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertTrue(commitOffsets.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitAsync() {
|
||||||
|
subscriptionState = mock(SubscriptionState.class);
|
||||||
|
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
|
||||||
|
TopicPartition tp = new TopicPartition("topic", 1);
|
||||||
|
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(0, Optional.of(1), "");
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> offsets = Map.of(tp, offsetAndMetadata);
|
||||||
|
|
||||||
|
CommitRequestManager commitRequestManager = create(true, 100);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
|
assertEquals(1, commitRequestManager.unsentOffsetCommitRequests().size());
|
||||||
|
List<NetworkClientDelegate.FutureCompletionHandler> pollResults = assertPoll(1, commitRequestManager);
|
||||||
|
pollResults.forEach(v -> v.onComplete(mockOffsetCommitResponse(
|
||||||
|
"topic",
|
||||||
|
1,
|
||||||
|
(short) 1,
|
||||||
|
Errors.NONE)));
|
||||||
|
|
||||||
|
verify(subscriptionState, never()).allConsumed();
|
||||||
|
verify(metadata).updateLastSeenEpochIfNewer(tp, 1);
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> commitOffsets = assertDoesNotThrow(() -> future.get());
|
||||||
|
assertEquals(offsets, commitOffsets);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitAsyncWithEmptyOffsets() {
|
||||||
|
subscriptionState = mock(SubscriptionState.class);
|
||||||
|
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
|
||||||
|
TopicPartition tp = new TopicPartition("topic", 1);
|
||||||
|
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(0, Optional.of(1), "");
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> offsets = Map.of(tp, offsetAndMetadata);
|
||||||
|
doReturn(offsets).when(subscriptionState).allConsumed();
|
||||||
|
|
||||||
|
CommitRequestManager commitRequestManager = create(true, 100);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = commitRequestManager.commitAsync(Optional.empty());
|
||||||
|
assertEquals(1, commitRequestManager.unsentOffsetCommitRequests().size());
|
||||||
|
List<NetworkClientDelegate.FutureCompletionHandler> pollResults = assertPoll(1, commitRequestManager);
|
||||||
|
pollResults.forEach(v -> v.onComplete(mockOffsetCommitResponse(
|
||||||
|
"topic",
|
||||||
|
1,
|
||||||
|
(short) 1,
|
||||||
|
Errors.NONE)));
|
||||||
|
|
||||||
|
verify(subscriptionState).allConsumed();
|
||||||
|
verify(metadata).updateLastSeenEpochIfNewer(tp, 1);
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> commitOffsets = assertDoesNotThrow(() -> future.get());
|
||||||
|
assertEquals(offsets, commitOffsets);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitAsyncWithEmptyAllConsumedOffsets() {
|
||||||
|
subscriptionState = mock(SubscriptionState.class);
|
||||||
|
doReturn(Map.of()).when(subscriptionState).allConsumed();
|
||||||
|
|
||||||
|
CommitRequestManager commitRequestManager = create(true, 100);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = commitRequestManager.commitAsync(Optional.empty());
|
||||||
|
|
||||||
|
verify(subscriptionState).allConsumed();
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> commitOffsets = assertDoesNotThrow(() -> future.get());
|
||||||
|
assertTrue(commitOffsets.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
// This is the case of the async auto commit sent on calls to assign (async commit that
|
// This is the case of the async auto commit sent on calls to assign (async commit that
|
||||||
// should not be retried).
|
// should not be retried).
|
||||||
@Test
|
@Test
|
||||||
|
@ -333,7 +470,7 @@ public class CommitRequestManagerTest {
|
||||||
new TopicPartition("topic", 1),
|
new TopicPartition("topic", 1),
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
|
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
|
||||||
CompletableFuture<Void> commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult = commitRequestManager.commitSync(Optional.of(offsets), deadlineMs);
|
||||||
sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried(commitRequestManager, error, commitResult);
|
sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried(commitRequestManager, error, commitResult);
|
||||||
|
|
||||||
// We expect that request should have been retried on this sync commit.
|
// We expect that request should have been retried on this sync commit.
|
||||||
|
@ -359,7 +496,7 @@ public class CommitRequestManagerTest {
|
||||||
new TopicPartition("topic", 1),
|
new TopicPartition("topic", 1),
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
|
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
|
||||||
CompletableFuture<Void> commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult = commitRequestManager.commitSync(Optional.of(offsets), deadlineMs);
|
||||||
|
|
||||||
completeOffsetCommitRequestWithError(commitRequestManager, Errors.UNKNOWN_MEMBER_ID);
|
completeOffsetCommitRequestWithError(commitRequestManager, Errors.UNKNOWN_MEMBER_ID);
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
|
@ -378,8 +515,8 @@ public class CommitRequestManagerTest {
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
|
|
||||||
// Send commit request expected to be retried on retriable errors
|
// Send commit request expected to be retried on retriable errors
|
||||||
CompletableFuture<Void> commitResult = commitRequestManager.commitSync(
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult = commitRequestManager.commitSync(
|
||||||
offsets, time.milliseconds() + defaultApiTimeoutMs);
|
Optional.of(offsets), time.milliseconds() + defaultApiTimeoutMs);
|
||||||
completeOffsetCommitRequestWithError(commitRequestManager, Errors.STALE_MEMBER_EPOCH);
|
completeOffsetCommitRequestWithError(commitRequestManager, Errors.STALE_MEMBER_EPOCH);
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(0, res.unsentRequests.size());
|
assertEquals(0, res.unsentRequests.size());
|
||||||
|
@ -431,7 +568,7 @@ public class CommitRequestManagerTest {
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
|
|
||||||
// Async commit that won't be retried.
|
// Async commit that won't be retried.
|
||||||
CompletableFuture<Void> commitResult = commitRequestManager.commitAsync(offsets);
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult = commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
|
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(1, res.unsentRequests.size());
|
assertEquals(1, res.unsentRequests.size());
|
||||||
|
@ -781,7 +918,7 @@ public class CommitRequestManagerTest {
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
|
|
||||||
// Send async commit (not expected to be retried).
|
// Send async commit (not expected to be retried).
|
||||||
CompletableFuture<Void> commitResult = commitRequestManager.commitAsync(offsets);
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult = commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
completeOffsetCommitRequestWithError(commitRequestManager, error);
|
completeOffsetCommitRequestWithError(commitRequestManager, error);
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(0, res.unsentRequests.size());
|
assertEquals(0, res.unsentRequests.size());
|
||||||
|
@ -806,7 +943,7 @@ public class CommitRequestManagerTest {
|
||||||
|
|
||||||
// Send sync offset commit request that fails with retriable error.
|
// Send sync offset commit request that fails with retriable error.
|
||||||
long deadlineMs = time.milliseconds() + retryBackoffMs * 2;
|
long deadlineMs = time.milliseconds() + retryBackoffMs * 2;
|
||||||
CompletableFuture<Void> commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult = commitRequestManager.commitSync(Optional.of(offsets), deadlineMs);
|
||||||
completeOffsetCommitRequestWithError(commitRequestManager, Errors.REQUEST_TIMED_OUT);
|
completeOffsetCommitRequestWithError(commitRequestManager, Errors.REQUEST_TIMED_OUT);
|
||||||
|
|
||||||
// Request retried after backoff, and fails with retriable again. Should not complete yet
|
// Request retried after backoff, and fails with retriable again. Should not complete yet
|
||||||
|
@ -839,7 +976,7 @@ public class CommitRequestManagerTest {
|
||||||
|
|
||||||
// Send offset commit request that fails with retriable error.
|
// Send offset commit request that fails with retriable error.
|
||||||
long deadlineMs = time.milliseconds() + retryBackoffMs * 2;
|
long deadlineMs = time.milliseconds() + retryBackoffMs * 2;
|
||||||
CompletableFuture<Void> commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult = commitRequestManager.commitSync(Optional.of(offsets), deadlineMs);
|
||||||
completeOffsetCommitRequestWithError(commitRequestManager, error);
|
completeOffsetCommitRequestWithError(commitRequestManager, error);
|
||||||
|
|
||||||
// Sleep to expire the request timeout. Request should fail on the next poll with a
|
// Sleep to expire the request timeout. Request should fail on the next poll with a
|
||||||
|
@ -869,7 +1006,7 @@ public class CommitRequestManagerTest {
|
||||||
|
|
||||||
// Send async commit request that fails with retriable error (not expected to be retried).
|
// Send async commit request that fails with retriable error (not expected to be retried).
|
||||||
Errors retriableError = Errors.COORDINATOR_NOT_AVAILABLE;
|
Errors retriableError = Errors.COORDINATOR_NOT_AVAILABLE;
|
||||||
CompletableFuture<Void> commitResult = commitRequestManager.commitAsync(offsets);
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult = commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
completeOffsetCommitRequestWithError(commitRequestManager, retriableError);
|
completeOffsetCommitRequestWithError(commitRequestManager, retriableError);
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(0, res.unsentRequests.size());
|
assertEquals(0, res.unsentRequests.size());
|
||||||
|
@ -894,7 +1031,7 @@ public class CommitRequestManagerTest {
|
||||||
offsets.put(new TopicPartition("t1", 1), new OffsetAndMetadata(2));
|
offsets.put(new TopicPartition("t1", 1), new OffsetAndMetadata(2));
|
||||||
offsets.put(new TopicPartition("t1", 2), new OffsetAndMetadata(3));
|
offsets.put(new TopicPartition("t1", 2), new OffsetAndMetadata(3));
|
||||||
|
|
||||||
commitRequestManager.commitSync(offsets, time.milliseconds() + defaultApiTimeoutMs);
|
commitRequestManager.commitSync(Optional.of(offsets), time.milliseconds() + defaultApiTimeoutMs);
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(1, res.unsentRequests.size());
|
assertEquals(1, res.unsentRequests.size());
|
||||||
|
|
||||||
|
@ -919,7 +1056,7 @@ public class CommitRequestManagerTest {
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
|
|
||||||
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
|
long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
|
||||||
commitRequestManager.commitSync(offsets, deadlineMs);
|
commitRequestManager.commitSync(Optional.of(offsets), deadlineMs);
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(1, res.unsentRequests.size());
|
assertEquals(1, res.unsentRequests.size());
|
||||||
res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException());
|
res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException());
|
||||||
|
@ -1175,7 +1312,7 @@ public class CommitRequestManagerTest {
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
|
|
||||||
long commitCreationTimeMs = time.milliseconds();
|
long commitCreationTimeMs = time.milliseconds();
|
||||||
commitRequestManager.commitAsync(offsets);
|
commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
|
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(1, res.unsentRequests.size());
|
assertEquals(1, res.unsentRequests.size());
|
||||||
|
@ -1338,7 +1475,7 @@ public class CommitRequestManagerTest {
|
||||||
Map<TopicPartition, OffsetAndMetadata> offsets = Collections.singletonMap(new TopicPartition("topic", 1),
|
Map<TopicPartition, OffsetAndMetadata> offsets = Collections.singletonMap(new TopicPartition("topic", 1),
|
||||||
new OffsetAndMetadata(0));
|
new OffsetAndMetadata(0));
|
||||||
|
|
||||||
commitRequestManager.commitAsync(offsets);
|
commitRequestManager.commitAsync(Optional.of(offsets));
|
||||||
commitRequestManager.signalClose();
|
commitRequestManager.signalClose();
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(1, res.unsentRequests.size());
|
assertEquals(1, res.unsentRequests.size());
|
||||||
|
@ -1384,7 +1521,7 @@ public class CommitRequestManagerTest {
|
||||||
private void sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried(
|
private void sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried(
|
||||||
final CommitRequestManager commitRequestManager,
|
final CommitRequestManager commitRequestManager,
|
||||||
final Errors error,
|
final Errors error,
|
||||||
final CompletableFuture<Void> commitResult) {
|
final CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitResult) {
|
||||||
completeOffsetCommitRequestWithError(commitRequestManager, error);
|
completeOffsetCommitRequestWithError(commitRequestManager, error);
|
||||||
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(0, res.unsentRequests.size());
|
assertEquals(0, res.unsentRequests.size());
|
||||||
|
@ -1438,7 +1575,8 @@ public class CommitRequestManagerTest {
|
||||||
retryBackoffMs,
|
retryBackoffMs,
|
||||||
retryBackoffMaxMs,
|
retryBackoffMaxMs,
|
||||||
OptionalDouble.of(0),
|
OptionalDouble.of(0),
|
||||||
metrics));
|
metrics,
|
||||||
|
metadata));
|
||||||
}
|
}
|
||||||
|
|
||||||
private ClientResponse buildOffsetFetchClientResponse(
|
private ClientResponse buildOffsetFetchClientResponse(
|
||||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
import org.apache.kafka.clients.Metadata;
|
import org.apache.kafka.clients.Metadata;
|
||||||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
||||||
|
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||||
import org.apache.kafka.clients.consumer.internals.CommitRequestManager;
|
import org.apache.kafka.clients.consumer.internals.CommitRequestManager;
|
||||||
import org.apache.kafka.clients.consumer.internals.ConsumerHeartbeatRequestManager;
|
import org.apache.kafka.clients.consumer.internals.ConsumerHeartbeatRequestManager;
|
||||||
|
@ -32,6 +33,7 @@ import org.apache.kafka.clients.consumer.internals.RequestManagers;
|
||||||
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
||||||
import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager;
|
import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager;
|
||||||
import org.apache.kafka.common.Cluster;
|
import org.apache.kafka.common.Cluster;
|
||||||
|
import org.apache.kafka.common.KafkaException;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
import org.apache.kafka.common.utils.MockTime;
|
||||||
|
@ -45,7 +47,6 @@ import org.junit.jupiter.params.provider.ValueSource;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
@ -56,6 +57,7 @@ import java.util.regex.Pattern;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs;
|
import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs;
|
||||||
|
import static org.apache.kafka.test.TestUtils.assertFutureThrows;
|
||||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||||
|
@ -72,6 +74,7 @@ import static org.mockito.Mockito.times;
|
||||||
import static org.mockito.Mockito.verify;
|
import static org.mockito.Mockito.verify;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
@SuppressWarnings("ClassDataAbstractionCoupling")
|
||||||
public class ApplicationEventProcessorTest {
|
public class ApplicationEventProcessorTest {
|
||||||
private final Time time = new MockTime();
|
private final Time time = new MockTime();
|
||||||
private final CommitRequestManager commitRequestManager = mock(CommitRequestManager.class);
|
private final CommitRequestManager commitRequestManager = mock(CommitRequestManager.class);
|
||||||
|
@ -137,8 +140,6 @@ public class ApplicationEventProcessorTest {
|
||||||
return Stream.of(
|
return Stream.of(
|
||||||
Arguments.of(new PollEvent(100)),
|
Arguments.of(new PollEvent(100)),
|
||||||
Arguments.of(new CreateFetchRequestsEvent(calculateDeadlineMs(12345, 100))),
|
Arguments.of(new CreateFetchRequestsEvent(calculateDeadlineMs(12345, 100))),
|
||||||
Arguments.of(new AsyncCommitEvent(new HashMap<>())),
|
|
||||||
Arguments.of(new SyncCommitEvent(new HashMap<>(), 500)),
|
|
||||||
Arguments.of(new CheckAndUpdatePositionsEvent(500)),
|
Arguments.of(new CheckAndUpdatePositionsEvent(500)),
|
||||||
Arguments.of(new TopicMetadataEvent("topic", Long.MAX_VALUE)),
|
Arguments.of(new TopicMetadataEvent("topic", Long.MAX_VALUE)),
|
||||||
Arguments.of(new AssignmentChangeEvent(12345, 12345, Collections.emptyList())));
|
Arguments.of(new AssignmentChangeEvent(12345, 12345, Collections.emptyList())));
|
||||||
|
@ -202,14 +203,16 @@ public class ApplicationEventProcessorTest {
|
||||||
@Test
|
@Test
|
||||||
public void testSeekUnvalidatedEvent() {
|
public void testSeekUnvalidatedEvent() {
|
||||||
TopicPartition tp = new TopicPartition("topic", 0);
|
TopicPartition tp = new TopicPartition("topic", 0);
|
||||||
|
Optional<Integer> offsetEpoch = Optional.of(1);
|
||||||
SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(
|
SubscriptionState.FetchPosition position = new SubscriptionState.FetchPosition(
|
||||||
0, Optional.empty(), Metadata.LeaderAndEpoch.noLeaderOrEpoch());
|
0, offsetEpoch, Metadata.LeaderAndEpoch.noLeaderOrEpoch());
|
||||||
SeekUnvalidatedEvent event = new SeekUnvalidatedEvent(12345, tp, 0, Optional.empty());
|
SeekUnvalidatedEvent event = new SeekUnvalidatedEvent(12345, tp, 0, offsetEpoch);
|
||||||
|
|
||||||
setupProcessor(false);
|
setupProcessor(false);
|
||||||
doReturn(Metadata.LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(tp);
|
doReturn(Metadata.LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(tp);
|
||||||
doNothing().when(subscriptionState).seekUnvalidated(eq(tp), any());
|
doNothing().when(subscriptionState).seekUnvalidated(eq(tp), any());
|
||||||
processor.process(event);
|
processor.process(event);
|
||||||
|
verify(metadata).updateLastSeenEpochIfNewer(tp, offsetEpoch.get());
|
||||||
verify(metadata).currentLeader(tp);
|
verify(metadata).currentLeader(tp);
|
||||||
verify(subscriptionState).seekUnvalidated(tp, position);
|
verify(subscriptionState).seekUnvalidated(tp, position);
|
||||||
assertDoesNotThrow(() -> event.future().get());
|
assertDoesNotThrow(() -> event.future().get());
|
||||||
|
@ -262,6 +265,27 @@ public class ApplicationEventProcessorTest {
|
||||||
assertDoesNotThrow(() -> event.future().get());
|
assertDoesNotThrow(() -> event.future().get());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFetchCommittedOffsetsEvent() {
|
||||||
|
TopicPartition tp0 = new TopicPartition("topic", 0);
|
||||||
|
TopicPartition tp1 = new TopicPartition("topic", 1);
|
||||||
|
TopicPartition tp2 = new TopicPartition("topic", 2);
|
||||||
|
Set<TopicPartition> partitions = Set.of(tp0, tp1, tp2);
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> topicPartitionOffsets = Map.of(
|
||||||
|
tp0, new OffsetAndMetadata(10L, Optional.of(2), ""),
|
||||||
|
tp1, new OffsetAndMetadata(15L, Optional.empty(), ""),
|
||||||
|
tp2, new OffsetAndMetadata(20L, Optional.of(3), "")
|
||||||
|
);
|
||||||
|
FetchCommittedOffsetsEvent event = new FetchCommittedOffsetsEvent(partitions, 12345);
|
||||||
|
|
||||||
|
setupProcessor(true);
|
||||||
|
when(commitRequestManager.fetchOffsets(partitions, 12345)).thenReturn(CompletableFuture.completedFuture(topicPartitionOffsets));
|
||||||
|
processor.process(event);
|
||||||
|
|
||||||
|
verify(commitRequestManager).fetchOffsets(partitions, 12345);
|
||||||
|
assertEquals(topicPartitionOffsets, assertDoesNotThrow(() -> event.future().get()));
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTopicSubscriptionChangeEventWithIllegalSubscriptionState() {
|
public void testTopicSubscriptionChangeEventWithIllegalSubscriptionState() {
|
||||||
subscriptionState = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST);
|
subscriptionState = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST);
|
||||||
|
@ -360,6 +384,87 @@ public class ApplicationEventProcessorTest {
|
||||||
assertDoesNotThrow(() -> event2.future().get());
|
assertDoesNotThrow(() -> event2.future().get());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource("offsetsGenerator")
|
||||||
|
public void testSyncCommitEvent(Optional<Map<TopicPartition, OffsetAndMetadata>> offsets) {
|
||||||
|
SyncCommitEvent event = new SyncCommitEvent(offsets, 12345);
|
||||||
|
|
||||||
|
setupProcessor(true);
|
||||||
|
doReturn(CompletableFuture.completedFuture(offsets.orElse(Map.of()))).when(commitRequestManager).commitSync(offsets, 12345);
|
||||||
|
|
||||||
|
processor.process(event);
|
||||||
|
verify(commitRequestManager).commitSync(offsets, 12345);
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> committedOffsets = assertDoesNotThrow(() -> event.future().get());
|
||||||
|
assertEquals(offsets.orElse(Map.of()), committedOffsets);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSyncCommitEventWithoutCommitRequestManager() {
|
||||||
|
SyncCommitEvent event = new SyncCommitEvent(Optional.empty(), 12345);
|
||||||
|
|
||||||
|
setupProcessor(false);
|
||||||
|
processor.process(event);
|
||||||
|
assertFutureThrows(event.future(), KafkaException.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSyncCommitEventWithException() {
|
||||||
|
SyncCommitEvent event = new SyncCommitEvent(Optional.empty(), 12345);
|
||||||
|
|
||||||
|
setupProcessor(true);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = new CompletableFuture<>();
|
||||||
|
future.completeExceptionally(new IllegalStateException());
|
||||||
|
doReturn(future).when(commitRequestManager).commitSync(any(), anyLong());
|
||||||
|
processor.process(event);
|
||||||
|
|
||||||
|
verify(commitRequestManager).commitSync(Optional.empty(), 12345);
|
||||||
|
assertFutureThrows(event.future(), IllegalStateException.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource("offsetsGenerator")
|
||||||
|
public void testAsyncCommitEventWithOffsets(Optional<Map<TopicPartition, OffsetAndMetadata>> offsets) {
|
||||||
|
AsyncCommitEvent event = new AsyncCommitEvent(offsets);
|
||||||
|
|
||||||
|
setupProcessor(true);
|
||||||
|
doReturn(CompletableFuture.completedFuture(offsets.orElse(Map.of()))).when(commitRequestManager).commitAsync(offsets);
|
||||||
|
|
||||||
|
processor.process(event);
|
||||||
|
verify(commitRequestManager).commitAsync(offsets);
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> committedOffsets = assertDoesNotThrow(() -> event.future().get());
|
||||||
|
assertEquals(offsets.orElse(Map.of()), committedOffsets);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAsyncCommitEventWithoutCommitRequestManager() {
|
||||||
|
AsyncCommitEvent event = new AsyncCommitEvent(Optional.empty());
|
||||||
|
|
||||||
|
setupProcessor(false);
|
||||||
|
processor.process(event);
|
||||||
|
assertFutureThrows(event.future(), KafkaException.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAsyncCommitEventWithException() {
|
||||||
|
AsyncCommitEvent event = new AsyncCommitEvent(Optional.empty());
|
||||||
|
|
||||||
|
setupProcessor(true);
|
||||||
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future = new CompletableFuture<>();
|
||||||
|
future.completeExceptionally(new IllegalStateException());
|
||||||
|
doReturn(future).when(commitRequestManager).commitAsync(any());
|
||||||
|
processor.process(event);
|
||||||
|
|
||||||
|
verify(commitRequestManager).commitAsync(Optional.empty());
|
||||||
|
assertFutureThrows(event.future(), IllegalStateException.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Stream<Arguments> offsetsGenerator() {
|
||||||
|
return Stream.of(
|
||||||
|
Arguments.of(Optional.empty()),
|
||||||
|
Arguments.of(Optional.of(Map.of(new TopicPartition("topic", 0), new OffsetAndMetadata(10, Optional.of(1), ""))))
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
private List<NetworkClientDelegate.UnsentRequest> mockCommitResults() {
|
private List<NetworkClientDelegate.UnsentRequest> mockCommitResults() {
|
||||||
return Collections.singletonList(mock(NetworkClientDelegate.UnsentRequest.class));
|
return Collections.singletonList(mock(NetworkClientDelegate.UnsentRequest.class));
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue