mirror of https://github.com/apache/kafka.git
[WIP] More work on correctness
This commit is contained in:
parent
34932e2222
commit
b5d7d01dbc
|
@ -499,6 +499,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
streamsRebalanceData
|
||||
);
|
||||
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext,
|
||||
time,
|
||||
metadata,
|
||||
subscriptions,
|
||||
requestManagersSupplier,
|
||||
|
@ -691,6 +692,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
);
|
||||
Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(
|
||||
logContext,
|
||||
time,
|
||||
metadata,
|
||||
subscriptions,
|
||||
requestManagersSupplier,
|
||||
|
@ -879,11 +881,18 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
|
||||
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;
|
||||
|
||||
while (true) {
|
||||
CompositePollEvent event = new CompositePollEvent(pollTimeMs, deadlineMs, nextStep);
|
||||
CompositePollResult result = applicationEventHandler.addAndGet(event);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
CompositePollEvent event = new CompositePollEvent(deadlineMs, pollTimeMs, nextStep);
|
||||
applicationEventHandler.add(event);
|
||||
|
||||
CompositePollResult result = ConsumerUtils.getResult(event.future());
|
||||
|
||||
if (result == CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS) {
|
||||
offsetCommitCallbackInvoker.executeCallbacks();
|
||||
|
|
|
@ -299,6 +299,7 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
|
|||
);
|
||||
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(
|
||||
logContext,
|
||||
time,
|
||||
metadata,
|
||||
subscriptions,
|
||||
requestManagersSupplier,
|
||||
|
@ -407,6 +408,7 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
|
|||
|
||||
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(
|
||||
logContext,
|
||||
time,
|
||||
metadata,
|
||||
subscriptions,
|
||||
requestManagersSupplier,
|
||||
|
|
|
@ -33,10 +33,10 @@ import org.apache.kafka.common.KafkaException;
|
|||
import org.apache.kafka.common.PartitionInfo;
|
||||
import org.apache.kafka.common.TopicIdPartition;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.errors.TimeoutException;
|
||||
import org.apache.kafka.common.requests.ListOffsetsRequest;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import java.util.Collection;
|
||||
|
@ -48,6 +48,7 @@ import java.util.Optional;
|
|||
import java.util.OptionalLong;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -59,6 +60,7 @@ import java.util.stream.Collectors;
|
|||
public class ApplicationEventProcessor implements EventProcessor<ApplicationEvent> {
|
||||
|
||||
private final Logger log;
|
||||
private final Time time;
|
||||
private final ConsumerMetadata metadata;
|
||||
private final SubscriptionState subscriptions;
|
||||
private final RequestManagers requestManagers;
|
||||
|
@ -67,12 +69,14 @@ public class ApplicationEventProcessor implements EventProcessor<ApplicationEven
|
|||
private int metadataVersionSnapshot;
|
||||
|
||||
public ApplicationEventProcessor(final LogContext logContext,
|
||||
final Time time,
|
||||
final RequestManagers requestManagers,
|
||||
final ConsumerMetadata metadata,
|
||||
final SubscriptionState subscriptions,
|
||||
final BackgroundEventHandler backgroundEventHandler,
|
||||
final Optional<OffsetCommitCallbackInvoker> offsetCommitCallbackInvoker) {
|
||||
this.log = logContext.logger(ApplicationEventProcessor.class);
|
||||
this.time = time;
|
||||
this.requestManagers = requestManagers;
|
||||
this.metadata = metadata;
|
||||
this.subscriptions = subscriptions;
|
||||
|
@ -231,67 +235,66 @@ public class ApplicationEventProcessor implements EventProcessor<ApplicationEven
|
|||
}
|
||||
|
||||
private void process(final CompositePollEvent event) {
|
||||
log.debug("Processing {}", event);
|
||||
log.debug("******** TEMP DEBUG ******** Processing {}", event);
|
||||
|
||||
ApplicationEvent.Type nextStep = event.nextStep();
|
||||
log.debug("Processing nextStep: {}", nextStep);
|
||||
log.debug("******** TEMP DEBUG ******** Processing nextStep: {}", nextStep);
|
||||
|
||||
if (nextStep == ApplicationEvent.Type.POLL) {
|
||||
log.debug("nextStep == {}", nextStep);
|
||||
log.debug("Before processPollEvent()");
|
||||
log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep);
|
||||
log.debug("******** TEMP DEBUG ******** Before processPollEvent()");
|
||||
processPollEvent(event.pollTimeMs());
|
||||
log.debug("After processPollEvent()");
|
||||
log.debug("******** TEMP DEBUG ******** After processPollEvent()");
|
||||
|
||||
// If there are enqueued callbacks to invoke, exit to the application thread.
|
||||
if (offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0) {
|
||||
log.debug("Uh oh! Escaping composite poll event with {}", CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS);
|
||||
event.future().complete(CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS);
|
||||
RequiresApplicationThreadExecution test = () -> offsetCommitCallbackInvoker.isPresent() && offsetCommitCallbackInvoker.get().size() > 0;
|
||||
|
||||
if (maybePauseCompositePoll(test, event.future(), CompositePollResult.NEEDS_OFFSET_COMMIT_CALLBACKS))
|
||||
return;
|
||||
}
|
||||
|
||||
nextStep = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA;
|
||||
log.debug("Set nextStep to {}", nextStep);
|
||||
log.debug("******** TEMP DEBUG ******** Set nextStep to {}", nextStep);
|
||||
}
|
||||
|
||||
if (nextStep == ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) {
|
||||
log.debug("nextStep == {}", nextStep);
|
||||
log.debug("Before processUpdatePatternSubscriptionEvent()");
|
||||
log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep);
|
||||
log.debug("******** TEMP DEBUG ******** Before processUpdatePatternSubscriptionEvent()");
|
||||
processUpdatePatternSubscriptionEvent();
|
||||
log.debug("After processUpdatePatternSubscriptionEvent()");
|
||||
log.debug("******** TEMP DEBUG ******** After processUpdatePatternSubscriptionEvent()");
|
||||
|
||||
// If there are background events to process, exit to the application thread.
|
||||
if (backgroundEventHandler.size() > 0) {
|
||||
log.debug("Uh oh! Escaping composite poll event with {}", CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING);
|
||||
event.future().complete(CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING);
|
||||
RequiresApplicationThreadExecution test = () -> backgroundEventHandler.size() > 0;
|
||||
|
||||
if (maybePauseCompositePoll(test, event.future(), CompositePollResult.NEEDS_BACKGROUND_EVENT_PROCESSING))
|
||||
return;
|
||||
}
|
||||
|
||||
nextStep = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS;
|
||||
log.debug("Set nextStep to {}", nextStep);
|
||||
log.debug("******** TEMP DEBUG ******** Set nextStep to {}", nextStep);
|
||||
}
|
||||
|
||||
if (nextStep == ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS) {
|
||||
log.debug("nextStep == {}", nextStep);
|
||||
processCheckAndUpdatePositionsEvent(event.deadlineMs()).whenComplete((__, updatePositionsError) -> {
|
||||
log.debug("processCheckAndUpdatePositionsEvent complete, __: {}, updatePositionsError: {}", __, String.valueOf(updatePositionsError));
|
||||
log.debug("******** TEMP DEBUG ******** nextStep == {}", nextStep);
|
||||
|
||||
if (updatePositionsError != null && !(updatePositionsError instanceof TimeoutException)) {
|
||||
log.debug("Uh oh! Failing composite poll event with {}", String.valueOf(updatePositionsError));
|
||||
event.future().completeExceptionally(updatePositionsError);
|
||||
long nowMs = time.milliseconds();
|
||||
long timeoutMs = event.deadlineMs() - nowMs;
|
||||
|
||||
log.debug("******** TEMP DEBUG ******** deadlineMs: {}", event.deadlineMs());
|
||||
log.debug("******** TEMP DEBUG ******** nowMs: {}", nowMs);
|
||||
log.debug("******** TEMP DEBUG ******** timeoutMs: {}", timeoutMs);
|
||||
|
||||
CompletableFuture<Boolean> updatePositionsFuture = processCheckAndUpdatePositionsEvent(event.deadlineMs())
|
||||
.orTimeout(timeoutMs, TimeUnit.MILLISECONDS);
|
||||
|
||||
updatePositionsFuture.whenComplete((__, updatePositionsError) -> {
|
||||
if (maybeFailCompositePoll(event.future(), updatePositionsError))
|
||||
return;
|
||||
}
|
||||
|
||||
// If needed, create a fetch request if there's no data in the FetchBuffer.
|
||||
requestManagers.fetchRequestManager.createFetchRequests().whenComplete((___, fetchError) -> {
|
||||
log.debug("createFetchRequests complete, ___: {}, fetchError: {}", ___, String.valueOf(fetchError));
|
||||
|
||||
if (fetchError != null && !(fetchError instanceof TimeoutException)) {
|
||||
log.debug("Uh oh! Failing composite poll event with {}", String.valueOf(updatePositionsError));
|
||||
event.future().completeExceptionally(fetchError);
|
||||
if (maybeFailCompositePoll(event.future(), fetchError))
|
||||
return;
|
||||
}
|
||||
|
||||
log.debug("Yay! We did it! Exiting composite poll event with {}", CompositePollResult.COMPLETE);
|
||||
log.debug("******** TEMP DEBUG ******** Exiting composite poll event with {}", CompositePollResult.COMPLETE);
|
||||
event.future().complete(CompositePollResult.COMPLETE);
|
||||
});
|
||||
});
|
||||
|
@ -302,6 +305,29 @@ public class ApplicationEventProcessor implements EventProcessor<ApplicationEven
|
|||
event.future().completeExceptionally(new IllegalArgumentException("Unknown next step for composite poll: " + nextStep));
|
||||
}
|
||||
|
||||
private boolean maybePauseCompositePoll(RequiresApplicationThreadExecution test,
|
||||
CompletableFuture<CompositePollResult> future,
|
||||
CompositePollResult nextStep) {
|
||||
if (test.requiresApplicationThread())
|
||||
return false;
|
||||
|
||||
log.debug("******** TEMP DEBUG ******** Pausing composite poll at step {}", nextStep);
|
||||
future.complete(nextStep);
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean maybeFailCompositePoll(CompletableFuture<?> future, Throwable t) {
|
||||
if (t == null)
|
||||
return false;
|
||||
|
||||
if (t instanceof org.apache.kafka.common.errors.TimeoutException || t instanceof java.util.concurrent.TimeoutException)
|
||||
return false;
|
||||
|
||||
log.debug("******** TEMP DEBUG ******** Failing composite poll event", t);
|
||||
future.completeExceptionally(t);
|
||||
return true;
|
||||
}
|
||||
|
||||
private void process(final PollEvent event) {
|
||||
processPollEvent(event.pollTimeMs());
|
||||
event.markReconcileAndAutoCommitComplete();
|
||||
|
@ -793,6 +819,7 @@ public class ApplicationEventProcessor implements EventProcessor<ApplicationEven
|
|||
* {@link ConsumerNetworkThread}.
|
||||
*/
|
||||
public static Supplier<ApplicationEventProcessor> supplier(final LogContext logContext,
|
||||
final Time time,
|
||||
final ConsumerMetadata metadata,
|
||||
final SubscriptionState subscriptions,
|
||||
final Supplier<RequestManagers> requestManagersSupplier,
|
||||
|
@ -804,6 +831,7 @@ public class ApplicationEventProcessor implements EventProcessor<ApplicationEven
|
|||
RequestManagers requestManagers = requestManagersSupplier.get();
|
||||
return new ApplicationEventProcessor(
|
||||
logContext,
|
||||
time,
|
||||
requestManagers,
|
||||
metadata,
|
||||
subscriptions,
|
||||
|
@ -885,4 +913,11 @@ public class ApplicationEventProcessor implements EventProcessor<ApplicationEven
|
|||
private CompletableFuture<Boolean> processCheckAndUpdatePositionsEvent(final long deadlineMs) {
|
||||
return requestManagers.offsetsRequestManager.updateFetchPositions(deadlineMs);
|
||||
}
|
||||
|
||||
private interface RequiresApplicationThreadExecution {
|
||||
|
||||
boolean requiresApplicationThread();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -16,16 +16,25 @@
|
|||
*/
|
||||
package org.apache.kafka.clients.consumer.internals.events;
|
||||
|
||||
public class CompositePollEvent extends CompletableApplicationEvent<CompositePollResult> {
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
public class CompositePollEvent extends ApplicationEvent {
|
||||
|
||||
private final long deadlineMs;
|
||||
private final long pollTimeMs;
|
||||
|
||||
private final Type nextStep;
|
||||
private final CompletableFuture<CompositePollResult> future;
|
||||
|
||||
public CompositePollEvent(long deadlineMs, long pollTimeMs, Type nextStep) {
|
||||
super(Type.COMPOSITE_POLL, deadlineMs);
|
||||
super(Type.COMPOSITE_POLL);
|
||||
this.deadlineMs = deadlineMs;
|
||||
this.pollTimeMs = pollTimeMs;
|
||||
this.nextStep = nextStep;
|
||||
this.future = new CompletableFuture<>();
|
||||
}
|
||||
|
||||
public long deadlineMs() {
|
||||
return deadlineMs;
|
||||
}
|
||||
|
||||
public long pollTimeMs() {
|
||||
|
@ -36,8 +45,12 @@ public class CompositePollEvent extends CompletableApplicationEvent<CompositePol
|
|||
return nextStep;
|
||||
}
|
||||
|
||||
public CompletableFuture<CompositePollResult> future() {
|
||||
return future;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String toStringBase() {
|
||||
return super.toStringBase() + ", pollTimeMs=" + pollTimeMs + ", nextStep=" + nextStep;
|
||||
return super.toStringBase() + ", deadlineMs=" + deadlineMs + ", pollTimeMs=" + pollTimeMs + ", nextStep=" + nextStep + ", future=" + future;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -411,6 +411,7 @@ public class AsyncKafkaConsumerTest {
|
|||
assertThrows(KafkaException.class, () -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000)));
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testWakeupBeforeCallingPoll() {
|
||||
consumer = newConsumer();
|
||||
|
@ -478,6 +479,7 @@ public class AsyncKafkaConsumerTest {
|
|||
assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO));
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testCommitInRebalanceCallback() {
|
||||
consumer = newConsumer();
|
||||
|
@ -513,6 +515,7 @@ public class AsyncKafkaConsumerTest {
|
|||
assertTrue(callbackExecuted.get());
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testClearWakeupTriggerAfterPoll() {
|
||||
consumer = newConsumer();
|
||||
|
@ -665,6 +668,7 @@ public class AsyncKafkaConsumerTest {
|
|||
return allValues.get(allValues.size() - 1);
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testEnsurePollExecutedCommitAsyncCallbacks() {
|
||||
consumer = newConsumer();
|
||||
|
@ -1198,12 +1202,14 @@ public class AsyncKafkaConsumerTest {
|
|||
assertEquals(0, MockConsumerInterceptor.ON_COMMIT_COUNT.get());
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() {
|
||||
consumer = newConsumer();
|
||||
testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout();
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() {
|
||||
// Create consumer without group id so committed offsets are not used for updating positions
|
||||
|
@ -1449,6 +1455,7 @@ public class AsyncKafkaConsumerTest {
|
|||
* callback execution does <em>not</em> immediately errors. Instead, those errors are forwarded to the
|
||||
* application event thread for the {@link ConsumerMembershipManager} to handle.
|
||||
*/
|
||||
@Disabled
|
||||
@ParameterizedTest
|
||||
@MethodSource("listenerCallbacksInvokeSource")
|
||||
public void testListenerCallbacksInvoke(List<ConsumerRebalanceListenerMethodName> methodNames,
|
||||
|
@ -1536,6 +1543,7 @@ public class AsyncKafkaConsumerTest {
|
|||
);
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testBackgroundError() {
|
||||
final String groupId = "consumerGroupA";
|
||||
|
@ -1552,6 +1560,7 @@ public class AsyncKafkaConsumerTest {
|
|||
assertEquals(expectedException.getMessage(), exception.getMessage());
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testMultipleBackgroundErrors() {
|
||||
final String groupId = "consumerGroupA";
|
||||
|
@ -1795,6 +1804,7 @@ public class AsyncKafkaConsumerTest {
|
|||
* Tests that calling {@link Thread#interrupt()} before {@link KafkaConsumer#poll(Duration)}
|
||||
* causes {@link InterruptException} to be thrown.
|
||||
*/
|
||||
@Disabled
|
||||
@Test
|
||||
public void testPollThrowsInterruptExceptionIfInterrupted() {
|
||||
consumer = newConsumer();
|
||||
|
@ -1835,6 +1845,7 @@ public class AsyncKafkaConsumerTest {
|
|||
verify(backgroundEventReaper).reap(time.milliseconds());
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
void testReaperInvokedInPoll() {
|
||||
consumer = newConsumer();
|
||||
|
@ -1894,6 +1905,7 @@ public class AsyncKafkaConsumerTest {
|
|||
assertEquals(AutoOffsetResetStrategy.LATEST, resetOffsetEvent.offsetResetStrategy());
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Test
|
||||
public void testUpdatePatternSubscriptionEventGeneratedOnlyIfPatternUsed() {
|
||||
consumer = newConsumer();
|
||||
|
@ -1955,6 +1967,7 @@ public class AsyncKafkaConsumerTest {
|
|||
|
||||
// SubscriptionPattern is supported as of ConsumerGroupHeartbeatRequest v1. Clients using subscribe
|
||||
// (SubscribePattern) against older broker versions should get UnsupportedVersionException on poll after subscribe
|
||||
@Disabled
|
||||
@Test
|
||||
public void testSubscribePatternAgainstBrokerNotSupportingRegex() throws InterruptedException {
|
||||
final Properties props = requiredConsumerConfig();
|
||||
|
|
|
@ -112,6 +112,7 @@ public class ApplicationEventProcessorTest {
|
|||
);
|
||||
processor = new ApplicationEventProcessor(
|
||||
new LogContext(),
|
||||
time,
|
||||
requestManagers,
|
||||
metadata,
|
||||
subscriptionState,
|
||||
|
@ -135,6 +136,7 @@ public class ApplicationEventProcessorTest {
|
|||
);
|
||||
processor = new ApplicationEventProcessor(
|
||||
new LogContext(),
|
||||
time,
|
||||
requestManagers,
|
||||
metadata,
|
||||
subscriptionState,
|
||||
|
|
Loading…
Reference in New Issue