Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@
import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.CompletableEvent;
import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper;
import org.apache.kafka.clients.consumer.internals.events.CompositePollEvent;
import org.apache.kafka.clients.consumer.internals.events.CompositePollResult;
import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent;
import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent;
import org.apache.kafka.clients.consumer.internals.events.CreateFetchRequestsEvent;
Expand All @@ -59,7 +61,6 @@
import org.apache.kafka.clients.consumer.internals.events.LeaveGroupOnCloseEvent;
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent;
import org.apache.kafka.clients.consumer.internals.events.PausePartitionsEvent;
import org.apache.kafka.clients.consumer.internals.events.PollEvent;
import org.apache.kafka.clients.consumer.internals.events.ResetOffsetEvent;
import org.apache.kafka.clients.consumer.internals.events.ResumePartitionsEvent;
import org.apache.kafka.clients.consumer.internals.events.SeekUnvalidatedEvent;
Expand Down Expand Up @@ -498,9 +499,13 @@ public AsyncKafkaConsumer(final ConsumerConfig config,
streamsRebalanceData
);
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext,
time,
metadata,
subscriptions,
requestManagersSupplier);
requestManagersSupplier,
backgroundEventHandler,
Optional.of(offsetCommitCallbackInvoker)
);
this.applicationEventHandler = applicationEventHandlerFactory.build(
logContext,
time,
Expand Down Expand Up @@ -687,9 +692,12 @@ public AsyncKafkaConsumer(final ConsumerConfig config,
);
Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(
logContext,
time,
metadata,
subscriptions,
requestManagersSupplier
requestManagersSupplier,
backgroundEventHandler,
Optional.of(offsetCommitCallbackInvoker)
);
this.applicationEventHandler = new ApplicationEventHandler(logContext,
time,
Expand Down Expand Up @@ -865,23 +873,39 @@ public ConsumerRecords<K, V> poll(final Duration timeout) {
}

do {
PollEvent event = new PollEvent(timer.currentTimeMs());
// Make sure to let the background thread know that we are still polling.
// This will trigger async auto-commits of consumed positions when hitting
// the interval time or reconciling new assignments
applicationEventHandler.add(event);
// Wait for reconciliation and auto-commit to be triggered, to ensure all commit requests
// retrieve the positions to commit before proceeding with fetching new records
ConsumerUtils.getResult(event.reconcileAndAutoCommit(), defaultApiTimeoutMs.toMillis());

// We must not allow wake-ups between polling for fetches and returning the records.
// If the polled fetches are not empty the consumed position has already been updated in the polling
// of the fetches. A wakeup between returned fetches and returning records would lead to never
// returning the records in the fetches. Thus, we trigger a possible wake-up before we poll fetches.
wakeupTrigger.maybeTriggerWakeup();

updateAssignmentMetadataIfNeeded(timer);
final Fetch<K, V> fetch = pollForFetches(timer);
long pollTimeMs = timer.currentTimeMs();
long deadlineMs = calculateDeadlineMs(timer);

log.debug("******** TEMP DEBUG ******** timeout: {}", timeout.toMillis());
log.debug("******** TEMP DEBUG ******** pollTimeMs: {}", pollTimeMs);
log.debug("******** TEMP DEBUG ******** deadlineMs: {}", deadlineMs);

ApplicationEvent.Type nextStep = ApplicationEvent.Type.POLL;

for (int i = 0; i < 10; i++) {
CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextStep);
applicationEventHandler.add(event);

CompositePollResult result = ConsumerUtils.getResult(event.future(), defaultApiTimeoutMs.toMillis());

if (result == CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS) {
offsetCommitCallbackInvoker.executeCallbacks();
nextStep = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA;
} else if (result == CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING) {
processBackgroundEvents();
nextStep = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS;
} else if (result == CompositePollResult.COMPLETE) {
break;
}
}

final Fetch<K, V> fetch = collectFetch();
if (!fetch.isEmpty()) {
// before returning the fetched records, we can send off the next round of fetches
// and avoid block waiting for their responses to enable pipelining while the user
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,9 @@ protected void maybeThrowAuthFailure(Node node) {
* @return Future on which the caller can wait to ensure that the requests have been created
*/
public CompletableFuture<Void> createFetchRequests() {
if (!fetchBuffer.isEmpty())
return CompletableFuture.completedFuture(null);

CompletableFuture<Void> future = new CompletableFuture<>();

if (pendingFetchRequestFuture != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,10 @@ public void enqueueInterceptorInvocation(final Map<TopicPartition, OffsetAndMeta
}
}

public int size() {
return callbackQueue.size();
}

public void enqueueUserCallbackInvocation(final OffsetCommitCallback callback,
final Map<TopicPartition, OffsetAndMetadata> offsets,
final Exception exception) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -299,9 +299,12 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) {
);
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(
logContext,
time,
metadata,
subscriptions,
requestManagersSupplier
requestManagersSupplier,
backgroundEventHandler,
Optional.empty()
);

this.applicationEventHandler = applicationEventHandlerFactory.build(
Expand Down Expand Up @@ -405,9 +408,12 @@ private void process(final ShareAcknowledgementCommitCallbackEvent event) {

final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(
logContext,
time,
metadata,
subscriptions,
requestManagersSupplier
requestManagersSupplier,
backgroundEventHandler,
Optional.empty()
);

this.applicationEventHandler = new ApplicationEventHandler(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
public abstract class ApplicationEvent {

public enum Type {
COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE,
COMMIT_ASYNC, COMMIT_SYNC, COMPOSITE_POLL, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE,
LIST_OFFSETS, CHECK_AND_UPDATE_POSITIONS, RESET_OFFSET, TOPIC_METADATA, ALL_TOPICS_METADATA,
TOPIC_SUBSCRIPTION_CHANGE, TOPIC_PATTERN_SUBSCRIPTION_CHANGE, TOPIC_RE2J_PATTERN_SUBSCRIPTION_CHANGE,
UPDATE_SUBSCRIPTION_METADATA, UNSUBSCRIBE,
Expand Down
Loading
Loading