mirror of https://github.com/apache/kafka.git
KAFKA-14274 [6, 7]: Introduction of fetch request manager (#14406)
Changes: 1. Introduces FetchRequestManager that implements the RequestManager API for fetching messages from brokers. Unlike Fetcher, record decompression and deserialization is performed on the application thread inside CompletedFetch. 2. Restructured the code so that objects owned by the background thread are not instantiated until the background thread runs (via Supplier) to ensure that there are no references available to the application thread. 3. Ensuring resources are properly using Closeable and using IdempotentCloser to ensure they're only closed once. 4. Introduces ConsumerTestBuilder to reduce a lot of inconsistency in the way the objects were built up for tests. Reviewers: Philip Nee <pnee@confluent.io>, Lianet Magrans <lianetmr@gmail.com>, Jun Rao<junrao@gmail.com>
This commit is contained in:
parent
d144b7ee38
commit
2b233bfa5f
|
@ -205,6 +205,10 @@
|
||||||
|
|
||||||
<subpackage name="consumer">
|
<subpackage name="consumer">
|
||||||
<allow pkg="org.apache.kafka.clients.consumer" />
|
<allow pkg="org.apache.kafka.clients.consumer" />
|
||||||
|
|
||||||
|
<subpackage name="internals">
|
||||||
|
<allow pkg="org.apache.kafka.clients" />
|
||||||
|
</subpackage>
|
||||||
</subpackage>
|
</subpackage>
|
||||||
|
|
||||||
<subpackage name="producer">
|
<subpackage name="producer">
|
||||||
|
|
|
@ -61,6 +61,8 @@
|
||||||
files="AbstractRequest.java"/>
|
files="AbstractRequest.java"/>
|
||||||
<suppress checks="ClassFanOutComplexity"
|
<suppress checks="ClassFanOutComplexity"
|
||||||
files="AbstractResponse.java"/>
|
files="AbstractResponse.java"/>
|
||||||
|
<suppress checks="ClassFanOutComplexity"
|
||||||
|
files="PrototypeAsyncConsumer.java"/>
|
||||||
|
|
||||||
<suppress checks="MethodLength"
|
<suppress checks="MethodLength"
|
||||||
files="(KerberosLogin|RequestResponseTest|ConnectMetricsRegistry|KafkaConsumer|AbstractStickyAssignor|AbstractRequest|AbstractResponse).java"/>
|
files="(KerberosLogin|RequestResponseTest|ConnectMetricsRegistry|KafkaConsumer|AbstractStickyAssignor|AbstractRequest|AbstractResponse).java"/>
|
||||||
|
@ -68,7 +70,7 @@
|
||||||
<suppress checks="ParameterNumber"
|
<suppress checks="ParameterNumber"
|
||||||
files="(NetworkClient|FieldSpec|KafkaRaftClient).java"/>
|
files="(NetworkClient|FieldSpec|KafkaRaftClient).java"/>
|
||||||
<suppress checks="ParameterNumber"
|
<suppress checks="ParameterNumber"
|
||||||
files="(KafkaConsumer|ConsumerCoordinator).java"/>
|
files="(KafkaConsumer|PrototypeAsyncConsumer|ConsumerCoordinator).java"/>
|
||||||
<suppress checks="ParameterNumber"
|
<suppress checks="ParameterNumber"
|
||||||
files="(RecordAccumulator|Sender).java"/>
|
files="(RecordAccumulator|Sender).java"/>
|
||||||
<suppress checks="ParameterNumber"
|
<suppress checks="ParameterNumber"
|
||||||
|
@ -79,7 +81,7 @@
|
||||||
files="MemoryRecordsBuilder.java"/>
|
files="MemoryRecordsBuilder.java"/>
|
||||||
|
|
||||||
<suppress checks="ClassDataAbstractionCoupling"
|
<suppress checks="ClassDataAbstractionCoupling"
|
||||||
files="(KafkaConsumer|ConsumerCoordinator|AbstractFetch|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient|MockAdminClient|KafkaRaftClient|KafkaRaftClientTest).java"/>
|
files="(KafkaConsumer|PrototypeAsyncConsumer|ConsumerCoordinator|AbstractFetch|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient|MockAdminClient|KafkaRaftClient|KafkaRaftClientTest).java"/>
|
||||||
<suppress checks="ClassDataAbstractionCoupling"
|
<suppress checks="ClassDataAbstractionCoupling"
|
||||||
files="(Errors|SaslAuthenticatorTest|AgentTest|CoordinatorTest).java"/>
|
files="(Errors|SaslAuthenticatorTest|AgentTest|CoordinatorTest).java"/>
|
||||||
|
|
||||||
|
@ -108,10 +110,10 @@
|
||||||
|
|
||||||
<!-- Clients tests -->
|
<!-- Clients tests -->
|
||||||
<suppress checks="ClassDataAbstractionCoupling"
|
<suppress checks="ClassDataAbstractionCoupling"
|
||||||
files="(Sender|Fetcher|OffsetFetcher|KafkaConsumer|Metrics|RequestResponse|TransactionManager|KafkaAdminClient|Message|KafkaProducer)Test.java"/>
|
files="(Sender|Fetcher|FetchRequestManager|OffsetFetcher|KafkaConsumer|PrototypeAsyncConsumer|Metrics|RequestResponse|TransactionManager|KafkaAdminClient|Message|KafkaProducer)Test.java"/>
|
||||||
|
|
||||||
<suppress checks="ClassFanOutComplexity"
|
<suppress checks="ClassFanOutComplexity"
|
||||||
files="(ConsumerCoordinator|KafkaConsumer|RequestResponse|Fetcher|KafkaAdminClient|Message|KafkaProducer)Test.java"/>
|
files="(ConsumerCoordinator|KafkaConsumer|RequestResponse|Fetcher|FetchRequestManager|KafkaAdminClient|Message|KafkaProducer)Test.java"/>
|
||||||
|
|
||||||
<suppress checks="ClassFanOutComplexity"
|
<suppress checks="ClassFanOutComplexity"
|
||||||
files="MockAdminClient.java"/>
|
files="MockAdminClient.java"/>
|
||||||
|
@ -120,7 +122,7 @@
|
||||||
files="(OffsetFetcher|RequestResponse)Test.java"/>
|
files="(OffsetFetcher|RequestResponse)Test.java"/>
|
||||||
|
|
||||||
<suppress checks="JavaNCSS"
|
<suppress checks="JavaNCSS"
|
||||||
files="RequestResponseTest.java|FetcherTest.java|KafkaAdminClientTest.java"/>
|
files="RequestResponseTest.java|FetcherTest.java|FetchRequestManagerTest.java|KafkaAdminClientTest.java"/>
|
||||||
|
|
||||||
<suppress checks="NPathComplexity"
|
<suppress checks="NPathComplexity"
|
||||||
files="MemoryRecordsTest|MetricsTest|RequestResponseTest|TestSslUtils|AclAuthorizerBenchmark"/>
|
files="MemoryRecordsTest|MetricsTest|RequestResponseTest|TestSslUtils|AclAuthorizerBenchmark"/>
|
||||||
|
|
|
@ -78,14 +78,13 @@ import java.util.regex.Pattern;
|
||||||
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_JMX_PREFIX;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_JMX_PREFIX;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createConsumerNetworkClient;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createConsumerNetworkClient;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchConfig;
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createLogContext;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createLogContext;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredConsumerInterceptors;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredConsumerInterceptors;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredIsolationLevel;
|
|
||||||
import static org.apache.kafka.common.utils.Utils.closeQuietly;
|
import static org.apache.kafka.common.utils.Utils.closeQuietly;
|
||||||
import static org.apache.kafka.common.utils.Utils.isBlank;
|
import static org.apache.kafka.common.utils.Utils.isBlank;
|
||||||
import static org.apache.kafka.common.utils.Utils.join;
|
import static org.apache.kafka.common.utils.Utils.join;
|
||||||
|
@ -570,7 +569,6 @@ import static org.apache.kafka.common.utils.Utils.swallow;
|
||||||
public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
|
|
||||||
private static final long NO_CURRENT_THREAD = -1L;
|
private static final long NO_CURRENT_THREAD = -1L;
|
||||||
static final long DEFAULT_CLOSE_TIMEOUT_MS = 30 * 1000;
|
|
||||||
static final String DEFAULT_REASON = "rebalance enforced by user";
|
static final String DEFAULT_REASON = "rebalance enforced by user";
|
||||||
|
|
||||||
// Visible for testing
|
// Visible for testing
|
||||||
|
@ -715,7 +713,8 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
this.metadata.bootstrap(addresses);
|
this.metadata.bootstrap(addresses);
|
||||||
|
|
||||||
FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics);
|
FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics);
|
||||||
this.isolationLevel = configuredIsolationLevel(config);
|
FetchConfig fetchConfig = new FetchConfig(config);
|
||||||
|
this.isolationLevel = fetchConfig.isolationLevel;
|
||||||
|
|
||||||
ApiVersions apiVersions = new ApiVersions();
|
ApiVersions apiVersions = new ApiVersions();
|
||||||
this.client = createConsumerNetworkClient(config,
|
this.client = createConsumerNetworkClient(config,
|
||||||
|
@ -753,13 +752,13 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED),
|
config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED),
|
||||||
config.getString(ConsumerConfig.CLIENT_RACK_CONFIG));
|
config.getString(ConsumerConfig.CLIENT_RACK_CONFIG));
|
||||||
}
|
}
|
||||||
FetchConfig<K, V> fetchConfig = createFetchConfig(config, this.deserializers);
|
|
||||||
this.fetcher = new Fetcher<>(
|
this.fetcher = new Fetcher<>(
|
||||||
logContext,
|
logContext,
|
||||||
this.client,
|
this.client,
|
||||||
this.metadata,
|
this.metadata,
|
||||||
this.subscriptions,
|
this.subscriptions,
|
||||||
fetchConfig,
|
fetchConfig,
|
||||||
|
this.deserializers,
|
||||||
fetchMetricsManager,
|
fetchMetricsManager,
|
||||||
this.time);
|
this.time);
|
||||||
this.offsetFetcher = new OffsetFetcher(logContext,
|
this.offsetFetcher = new OffsetFetcher(logContext,
|
||||||
|
@ -2483,7 +2482,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
// coordinator lookup if there are partitions which have missing positions, so
|
// coordinator lookup if there are partitions which have missing positions, so
|
||||||
// a consumer with manually assigned partitions can avoid a coordinator dependence
|
// a consumer with manually assigned partitions can avoid a coordinator dependence
|
||||||
// by always ensuring that assigned partitions have an initial position.
|
// by always ensuring that assigned partitions have an initial position.
|
||||||
if (coordinator != null && !coordinator.refreshCommittedOffsetsIfNeeded(timer)) return false;
|
if (coordinator != null && !coordinator.initWithCommittedOffsetsIfNeeded(timer)) return false;
|
||||||
|
|
||||||
// If there are partitions still needing a position and a reset policy is defined,
|
// If there are partitions still needing a position and a reset policy is defined,
|
||||||
// request reset using the default policy. If no reset strategy is defined and there
|
// request reset using the default policy. If no reset strategy is defined and there
|
||||||
|
|
|
@ -46,7 +46,7 @@ import java.util.regex.Pattern;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static java.util.Collections.singleton;
|
import static java.util.Collections.singleton;
|
||||||
import static org.apache.kafka.clients.consumer.KafkaConsumer.DEFAULT_CLOSE_TIMEOUT_MS;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -18,10 +18,13 @@ package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.clients.ClientResponse;
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
import org.apache.kafka.clients.FetchSessionHandler;
|
import org.apache.kafka.clients.FetchSessionHandler;
|
||||||
|
import org.apache.kafka.clients.KafkaClient;
|
||||||
|
import org.apache.kafka.clients.NetworkClientUtils;
|
||||||
import org.apache.kafka.common.Cluster;
|
import org.apache.kafka.common.Cluster;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.Uuid;
|
import org.apache.kafka.common.Uuid;
|
||||||
|
import org.apache.kafka.common.errors.AuthenticationException;
|
||||||
import org.apache.kafka.common.internals.IdempotentCloser;
|
import org.apache.kafka.common.internals.IdempotentCloser;
|
||||||
import org.apache.kafka.common.message.FetchResponseData;
|
import org.apache.kafka.common.message.FetchResponseData;
|
||||||
import org.apache.kafka.common.protocol.ApiKeys;
|
import org.apache.kafka.common.protocol.ApiKeys;
|
||||||
|
@ -37,61 +40,73 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.helpers.MessageFormatter;
|
import org.slf4j.helpers.MessageFormatter;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.util.ArrayList;
|
import java.time.Duration;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.LinkedHashMap;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.FetchUtils.requestMetadataUpdate;
|
import static org.apache.kafka.clients.consumer.internals.FetchUtils.requestMetadataUpdate;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@code AbstractFetch} represents the basic state and logic for record fetching processing.
|
* {@code AbstractFetch} represents the basic state and logic for record fetching processing.
|
||||||
* @param <K> Type for the message key
|
|
||||||
* @param <V> Type for the message value
|
|
||||||
*/
|
*/
|
||||||
public abstract class AbstractFetch<K, V> implements Closeable {
|
public abstract class AbstractFetch implements Closeable {
|
||||||
|
|
||||||
private final Logger log;
|
private final Logger log;
|
||||||
|
private final IdempotentCloser idempotentCloser = new IdempotentCloser();
|
||||||
protected final LogContext logContext;
|
protected final LogContext logContext;
|
||||||
protected final ConsumerNetworkClient client;
|
|
||||||
protected final ConsumerMetadata metadata;
|
protected final ConsumerMetadata metadata;
|
||||||
protected final SubscriptionState subscriptions;
|
protected final SubscriptionState subscriptions;
|
||||||
protected final FetchConfig<K, V> fetchConfig;
|
protected final FetchConfig fetchConfig;
|
||||||
protected final Time time;
|
protected final Time time;
|
||||||
protected final FetchMetricsManager metricsManager;
|
protected final FetchMetricsManager metricsManager;
|
||||||
protected final FetchBuffer fetchBuffer;
|
protected final FetchBuffer fetchBuffer;
|
||||||
protected final BufferSupplier decompressionBufferSupplier;
|
protected final BufferSupplier decompressionBufferSupplier;
|
||||||
protected final Set<Integer> nodesWithPendingFetchRequests;
|
protected final Set<Integer> nodesWithPendingFetchRequests;
|
||||||
protected final IdempotentCloser idempotentCloser = new IdempotentCloser();
|
|
||||||
|
|
||||||
private final Map<Integer, FetchSessionHandler> sessionHandlers;
|
private final Map<Integer, FetchSessionHandler> sessionHandlers;
|
||||||
|
|
||||||
public AbstractFetch(final LogContext logContext,
|
public AbstractFetch(final LogContext logContext,
|
||||||
final ConsumerNetworkClient client,
|
|
||||||
final ConsumerMetadata metadata,
|
final ConsumerMetadata metadata,
|
||||||
final SubscriptionState subscriptions,
|
final SubscriptionState subscriptions,
|
||||||
final FetchConfig<K, V> fetchConfig,
|
final FetchConfig fetchConfig,
|
||||||
|
final FetchBuffer fetchBuffer,
|
||||||
final FetchMetricsManager metricsManager,
|
final FetchMetricsManager metricsManager,
|
||||||
final Time time) {
|
final Time time) {
|
||||||
this.log = logContext.logger(AbstractFetch.class);
|
this.log = logContext.logger(AbstractFetch.class);
|
||||||
this.logContext = logContext;
|
this.logContext = logContext;
|
||||||
this.client = client;
|
|
||||||
this.metadata = metadata;
|
this.metadata = metadata;
|
||||||
this.subscriptions = subscriptions;
|
this.subscriptions = subscriptions;
|
||||||
this.fetchConfig = fetchConfig;
|
this.fetchConfig = fetchConfig;
|
||||||
|
this.fetchBuffer = fetchBuffer;
|
||||||
this.decompressionBufferSupplier = BufferSupplier.create();
|
this.decompressionBufferSupplier = BufferSupplier.create();
|
||||||
this.fetchBuffer = new FetchBuffer(logContext);
|
|
||||||
this.sessionHandlers = new HashMap<>();
|
this.sessionHandlers = new HashMap<>();
|
||||||
this.nodesWithPendingFetchRequests = new HashSet<>();
|
this.nodesWithPendingFetchRequests = new HashSet<>();
|
||||||
this.metricsManager = metricsManager;
|
this.metricsManager = metricsManager;
|
||||||
this.time = time;
|
this.time = time;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if the node is disconnected and unavailable for immediate reconnection (i.e. if it is in
|
||||||
|
* reconnect backoff window following the disconnect).
|
||||||
|
*
|
||||||
|
* @param node {@link Node} to check for availability
|
||||||
|
* @see NetworkClientUtils#isUnavailable(KafkaClient, Node, Time)
|
||||||
|
*/
|
||||||
|
protected abstract boolean isUnavailable(Node node);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks for an authentication error on a given node and throws the exception if it exists.
|
||||||
|
*
|
||||||
|
* @param node {@link Node} to check for a previous {@link AuthenticationException}; if found it is thrown
|
||||||
|
* @see NetworkClientUtils#maybeThrowAuthFailure(KafkaClient, Node)
|
||||||
|
*/
|
||||||
|
protected abstract void maybeThrowAuthFailure(Node node);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return whether we have any completed fetches pending return to the user. This method is thread-safe. Has
|
* Return whether we have any completed fetches pending return to the user. This method is thread-safe. Has
|
||||||
* visibility for testing.
|
* visibility for testing.
|
||||||
|
@ -111,15 +126,15 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implements the core logic for a successful fetch request/response.
|
* Implements the core logic for a successful fetch response.
|
||||||
*
|
*
|
||||||
* @param fetchTarget {@link Node} from which the fetch data was requested
|
* @param fetchTarget {@link Node} from which the fetch data was requested
|
||||||
* @param data {@link FetchSessionHandler.FetchRequestData} that represents the session data
|
* @param data {@link FetchSessionHandler.FetchRequestData} that represents the session data
|
||||||
* @param resp {@link ClientResponse} from which the {@link FetchResponse} will be retrieved
|
* @param resp {@link ClientResponse} from which the {@link FetchResponse} will be retrieved
|
||||||
*/
|
*/
|
||||||
protected void handleFetchResponse(final Node fetchTarget,
|
protected void handleFetchSuccess(final Node fetchTarget,
|
||||||
final FetchSessionHandler.FetchRequestData data,
|
final FetchSessionHandler.FetchRequestData data,
|
||||||
final ClientResponse resp) {
|
final ClientResponse resp) {
|
||||||
try {
|
try {
|
||||||
final FetchResponse response = (FetchResponse) resp.responseBody();
|
final FetchResponse response = (FetchResponse) resp.responseBody();
|
||||||
final FetchSessionHandler handler = sessionHandler(fetchTarget.id());
|
final FetchSessionHandler handler = sessionHandler(fetchTarget.id());
|
||||||
|
@ -185,18 +200,20 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
|
|
||||||
metricsManager.recordLatency(resp.requestLatencyMs());
|
metricsManager.recordLatency(resp.requestLatencyMs());
|
||||||
} finally {
|
} finally {
|
||||||
log.debug("Removing pending request for node {}", fetchTarget);
|
removePendingFetchRequest(fetchTarget, data.metadata().sessionId());
|
||||||
nodesWithPendingFetchRequests.remove(fetchTarget.id());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implements the core logic for a failed fetch request/response.
|
* Implements the core logic for a failed fetch response.
|
||||||
*
|
*
|
||||||
* @param fetchTarget {@link Node} from which the fetch data was requested
|
* @param fetchTarget {@link Node} from which the fetch data was requested
|
||||||
* @param t {@link Throwable} representing the error that resulted in the failure
|
* @param data {@link FetchSessionHandler.FetchRequestData} from request
|
||||||
|
* @param t {@link Throwable} representing the error that resulted in the failure
|
||||||
*/
|
*/
|
||||||
protected void handleFetchResponse(final Node fetchTarget, final Throwable t) {
|
protected void handleFetchFailure(final Node fetchTarget,
|
||||||
|
final FetchSessionHandler.FetchRequestData data,
|
||||||
|
final Throwable t) {
|
||||||
try {
|
try {
|
||||||
final FetchSessionHandler handler = sessionHandler(fetchTarget.id());
|
final FetchSessionHandler handler = sessionHandler(fetchTarget.id());
|
||||||
|
|
||||||
|
@ -205,25 +222,32 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
handler.sessionTopicPartitions().forEach(subscriptions::clearPreferredReadReplica);
|
handler.sessionTopicPartitions().forEach(subscriptions::clearPreferredReadReplica);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
log.debug("Removing pending request for node {}", fetchTarget);
|
removePendingFetchRequest(fetchTarget, data.metadata().sessionId());
|
||||||
nodesWithPendingFetchRequests.remove(fetchTarget.id());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void handleCloseFetchSessionResponse(final Node fetchTarget,
|
protected void handleCloseFetchSessionSuccess(final Node fetchTarget,
|
||||||
final FetchSessionHandler.FetchRequestData data) {
|
final FetchSessionHandler.FetchRequestData data,
|
||||||
|
final ClientResponse ignored) {
|
||||||
int sessionId = data.metadata().sessionId();
|
int sessionId = data.metadata().sessionId();
|
||||||
|
removePendingFetchRequest(fetchTarget, sessionId);
|
||||||
log.debug("Successfully sent a close message for fetch session: {} to node: {}", sessionId, fetchTarget);
|
log.debug("Successfully sent a close message for fetch session: {} to node: {}", sessionId, fetchTarget);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void handleCloseFetchSessionResponse(final Node fetchTarget,
|
public void handleCloseFetchSessionFailure(final Node fetchTarget,
|
||||||
final FetchSessionHandler.FetchRequestData data,
|
final FetchSessionHandler.FetchRequestData data,
|
||||||
final Throwable t) {
|
final Throwable t) {
|
||||||
int sessionId = data.metadata().sessionId();
|
int sessionId = data.metadata().sessionId();
|
||||||
log.debug("Unable to a close message for fetch session: {} to node: {}. " +
|
removePendingFetchRequest(fetchTarget, sessionId);
|
||||||
|
log.debug("Unable to send a close message for fetch session: {} to node: {}. " +
|
||||||
"This may result in unnecessary fetch sessions at the broker.", sessionId, fetchTarget, t);
|
"This may result in unnecessary fetch sessions at the broker.", sessionId, fetchTarget, t);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void removePendingFetchRequest(Node fetchTarget, int sessionId) {
|
||||||
|
log.debug("Removing pending request for fetch session: {} for node: {}", sessionId, fetchTarget);
|
||||||
|
nodesWithPendingFetchRequests.remove(fetchTarget.id());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new {@link FetchRequest fetch request} in preparation for sending to the Kafka cluster.
|
* Creates a new {@link FetchRequest fetch request} in preparation for sending to the Kafka cluster.
|
||||||
*
|
*
|
||||||
|
@ -317,9 +341,9 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<Node, FetchSessionHandler.FetchRequestData> prepareCloseFetchSessionRequests() {
|
protected Map<Node, FetchSessionHandler.FetchRequestData> prepareCloseFetchSessionRequests() {
|
||||||
final Cluster cluster = metadata.fetch();
|
final Cluster cluster = metadata.fetch();
|
||||||
Map<Node, FetchSessionHandler.Builder> fetchable = new LinkedHashMap<>();
|
Map<Node, FetchSessionHandler.Builder> fetchable = new HashMap<>();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
sessionHandlers.forEach((fetchTargetNodeId, sessionHandler) -> {
|
sessionHandlers.forEach((fetchTargetNodeId, sessionHandler) -> {
|
||||||
|
@ -330,7 +354,7 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
// skip sending the close request.
|
// skip sending the close request.
|
||||||
final Node fetchTarget = cluster.nodeById(fetchTargetNodeId);
|
final Node fetchTarget = cluster.nodeById(fetchTargetNodeId);
|
||||||
|
|
||||||
if (fetchTarget == null || client.isUnavailable(fetchTarget)) {
|
if (fetchTarget == null || isUnavailable(fetchTarget)) {
|
||||||
log.debug("Skip sending close session request to broker {} since it is not reachable", fetchTarget);
|
log.debug("Skip sending close session request to broker {} since it is not reachable", fetchTarget);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -341,11 +365,7 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
sessionHandlers.clear();
|
sessionHandlers.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
Map<Node, FetchSessionHandler.FetchRequestData> reqs = new LinkedHashMap<>();
|
return fetchable.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().build()));
|
||||||
for (Map.Entry<Node, FetchSessionHandler.Builder> entry : fetchable.entrySet()) {
|
|
||||||
reqs.put(entry.getKey(), entry.getValue().build());
|
|
||||||
}
|
|
||||||
return reqs;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -356,7 +376,7 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
// Update metrics in case there was an assignment change
|
// Update metrics in case there was an assignment change
|
||||||
metricsManager.maybeUpdateAssignment(subscriptions);
|
metricsManager.maybeUpdateAssignment(subscriptions);
|
||||||
|
|
||||||
Map<Node, FetchSessionHandler.Builder> fetchable = new LinkedHashMap<>();
|
Map<Node, FetchSessionHandler.Builder> fetchable = new HashMap<>();
|
||||||
long currentTimeMs = time.milliseconds();
|
long currentTimeMs = time.milliseconds();
|
||||||
Map<String, Uuid> topicIds = metadata.topicIds();
|
Map<String, Uuid> topicIds = metadata.topicIds();
|
||||||
|
|
||||||
|
@ -377,8 +397,8 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
// Use the preferred read replica if set, otherwise the partition's leader
|
// Use the preferred read replica if set, otherwise the partition's leader
|
||||||
Node node = selectReadReplica(partition, leaderOpt.get(), currentTimeMs);
|
Node node = selectReadReplica(partition, leaderOpt.get(), currentTimeMs);
|
||||||
|
|
||||||
if (client.isUnavailable(node)) {
|
if (isUnavailable(node)) {
|
||||||
client.maybeThrowAuthFailure(node);
|
maybeThrowAuthFailure(node);
|
||||||
|
|
||||||
// If we try to send during the reconnect backoff window, then the request is just
|
// If we try to send during the reconnect backoff window, then the request is just
|
||||||
// going to be failed anyway before being sent, so skip sending the request for now
|
// going to be failed anyway before being sent, so skip sending the request for now
|
||||||
|
@ -405,51 +425,7 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Map<Node, FetchSessionHandler.FetchRequestData> reqs = new LinkedHashMap<>();
|
return fetchable.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().build()));
|
||||||
for (Map.Entry<Node, FetchSessionHandler.Builder> entry : fetchable.entrySet()) {
|
|
||||||
reqs.put(entry.getKey(), entry.getValue().build());
|
|
||||||
}
|
|
||||||
return reqs;
|
|
||||||
}
|
|
||||||
|
|
||||||
protected void maybeCloseFetchSessions(final Timer timer) {
|
|
||||||
final List<RequestFuture<ClientResponse>> requestFutures = new ArrayList<>();
|
|
||||||
Map<Node, FetchSessionHandler.FetchRequestData> fetchRequestMap = prepareCloseFetchSessionRequests();
|
|
||||||
|
|
||||||
for (Map.Entry<Node, FetchSessionHandler.FetchRequestData> entry : fetchRequestMap.entrySet()) {
|
|
||||||
final Node fetchTarget = entry.getKey();
|
|
||||||
final FetchSessionHandler.FetchRequestData data = entry.getValue();
|
|
||||||
final FetchRequest.Builder request = createFetchRequest(fetchTarget, data);
|
|
||||||
final RequestFuture<ClientResponse> responseFuture = client.send(fetchTarget, request);
|
|
||||||
|
|
||||||
responseFuture.addListener(new RequestFutureListener<ClientResponse>() {
|
|
||||||
@Override
|
|
||||||
public void onSuccess(ClientResponse value) {
|
|
||||||
handleCloseFetchSessionResponse(fetchTarget, data);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onFailure(RuntimeException e) {
|
|
||||||
handleCloseFetchSessionResponse(fetchTarget, data, e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
requestFutures.add(responseFuture);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until
|
|
||||||
// all requests have received a response.
|
|
||||||
while (timer.notExpired() && !requestFutures.stream().allMatch(RequestFuture::isDone)) {
|
|
||||||
client.poll(timer, null, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!requestFutures.stream().allMatch(RequestFuture::isDone)) {
|
|
||||||
// we ran out of time before completing all futures. It is ok since we don't want to block the shutdown
|
|
||||||
// here.
|
|
||||||
log.debug("All requests couldn't be sent in the specific timeout period {}ms. " +
|
|
||||||
"This may result in unnecessary fetch sessions at the broker. Consider increasing the timeout passed for " +
|
|
||||||
"KafkaConsumer.close(Duration timeout)", timer.timeoutMs());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Visible for testing
|
// Visible for testing
|
||||||
|
@ -467,20 +443,29 @@ public abstract class AbstractFetch<K, V> implements Closeable {
|
||||||
// Visible for testing
|
// Visible for testing
|
||||||
protected void closeInternal(Timer timer) {
|
protected void closeInternal(Timer timer) {
|
||||||
// we do not need to re-enable wake-ups since we are closing already
|
// we do not need to re-enable wake-ups since we are closing already
|
||||||
client.disableWakeups();
|
|
||||||
maybeCloseFetchSessions(timer);
|
|
||||||
Utils.closeQuietly(fetchBuffer, "fetchBuffer");
|
Utils.closeQuietly(fetchBuffer, "fetchBuffer");
|
||||||
Utils.closeQuietly(decompressionBufferSupplier, "decompressionBufferSupplier");
|
Utils.closeQuietly(decompressionBufferSupplier, "decompressionBufferSupplier");
|
||||||
}
|
}
|
||||||
|
|
||||||
public void close(final Timer timer) {
|
public void close(final Timer timer) {
|
||||||
idempotentCloser.close(() -> {
|
idempotentCloser.close(() -> closeInternal(timer));
|
||||||
closeInternal(timer);
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
close(time.timer(0));
|
close(time.timer(Duration.ZERO));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Defines the contract for handling fetch responses from brokers.
|
||||||
|
* @param <T> Type of response, usually either {@link ClientResponse} or {@link Throwable}
|
||||||
|
*/
|
||||||
|
@FunctionalInterface
|
||||||
|
protected interface ResponseHandler<T> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Handle the response from the given {@link Node target}
|
||||||
|
*/
|
||||||
|
void handle(Node target, FetchSessionHandler.FetchRequestData data, T response);
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -16,19 +16,28 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
import java.util.function.Supplier;
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent;
|
|
||||||
|
|
||||||
import java.util.Queue;
|
/**
|
||||||
|
* Simple {@link Supplier} that caches the initial creation of the object and stores it for later calls
|
||||||
|
* to {@link #get()}.
|
||||||
|
*
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* <em>Note</em>: this class is not thread safe! Use only in contexts which are designed/guaranteed to be
|
||||||
|
* single-threaded.
|
||||||
|
*/
|
||||||
|
public abstract class CachedSupplier<T> implements Supplier<T> {
|
||||||
|
|
||||||
public class ErrorEventHandler {
|
private T result;
|
||||||
private final Queue<BackgroundEvent> backgroundEventQueue;
|
|
||||||
|
|
||||||
public ErrorEventHandler(Queue<BackgroundEvent> backgroundEventQueue) {
|
protected abstract T create();
|
||||||
this.backgroundEventQueue = backgroundEventQueue;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void handle(Throwable e) {
|
@Override
|
||||||
backgroundEventQueue.add(new ErrorBackgroundEvent(e));
|
public T get() {
|
||||||
|
if (result == null)
|
||||||
|
result = create();
|
||||||
|
|
||||||
|
return result;
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -48,10 +48,12 @@ import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY;
|
||||||
|
|
||||||
public class CommitRequestManager implements RequestManager {
|
public class CommitRequestManager implements RequestManager {
|
||||||
|
|
||||||
// TODO: current in ConsumerConfig but inaccessible in the internal package.
|
// TODO: current in ConsumerConfig but inaccessible in the internal package.
|
||||||
private static final String THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED = "internal.throw.on.fetch.stable.offset.unsupported";
|
private static final String THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED = "internal.throw.on.fetch.stable.offset.unsupported";
|
||||||
// TODO: We will need to refactor the subscriptionState
|
|
||||||
private final SubscriptionState subscriptions;
|
private final SubscriptionState subscriptions;
|
||||||
private final LogContext logContext;
|
private final LogContext logContext;
|
||||||
private final Logger log;
|
private final Logger log;
|
||||||
|
@ -96,17 +98,14 @@ public class CommitRequestManager implements RequestManager {
|
||||||
@Override
|
@Override
|
||||||
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
|
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
|
||||||
// poll only when the coordinator node is known.
|
// poll only when the coordinator node is known.
|
||||||
if (!coordinatorRequestManager.coordinator().isPresent()) {
|
if (!coordinatorRequestManager.coordinator().isPresent())
|
||||||
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList());
|
return EMPTY;
|
||||||
}
|
|
||||||
|
|
||||||
maybeAutoCommit(this.subscriptions.allConsumed());
|
maybeAutoCommit(this.subscriptions.allConsumed());
|
||||||
if (!pendingRequests.hasUnsentRequests()) {
|
if (!pendingRequests.hasUnsentRequests())
|
||||||
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList());
|
return EMPTY;
|
||||||
}
|
|
||||||
|
|
||||||
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE,
|
return new NetworkClientDelegate.PollResult(pendingRequests.drain(currentTimeMs));
|
||||||
Collections.unmodifiableList(pendingRequests.drain(currentTimeMs)));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void maybeAutoCommit(final Map<TopicPartition, OffsetAndMetadata> offsets) {
|
public void maybeAutoCommit(final Map<TopicPartition, OffsetAndMetadata> offsets) {
|
||||||
|
|
|
@ -53,7 +53,7 @@ import java.util.Set;
|
||||||
/**
|
/**
|
||||||
* {@link CompletedFetch} represents a {@link RecordBatch batch} of {@link Record records} that was returned from the
|
* {@link CompletedFetch} represents a {@link RecordBatch batch} of {@link Record records} that was returned from the
|
||||||
* broker via a {@link FetchRequest}. It contains logic to maintain state between calls to
|
* broker via a {@link FetchRequest}. It contains logic to maintain state between calls to
|
||||||
* {@link #fetchRecords(FetchConfig, int)}.
|
* {@link #fetchRecords(FetchConfig, Deserializers, int)}.
|
||||||
*/
|
*/
|
||||||
public class CompletedFetch {
|
public class CompletedFetch {
|
||||||
|
|
||||||
|
@ -135,7 +135,8 @@ public class CompletedFetch {
|
||||||
/**
|
/**
|
||||||
* Draining a {@link CompletedFetch} will signal that the data has been consumed and the underlying resources
|
* Draining a {@link CompletedFetch} will signal that the data has been consumed and the underlying resources
|
||||||
* are closed. This is somewhat analogous to {@link Closeable#close() closing}, though no error will result if a
|
* are closed. This is somewhat analogous to {@link Closeable#close() closing}, though no error will result if a
|
||||||
* caller invokes {@link #fetchRecords(FetchConfig, int)}; an empty {@link List list} will be returned instead.
|
* caller invokes {@link #fetchRecords(FetchConfig, Deserializers, int)}; an empty {@link List list} will be
|
||||||
|
* returned instead.
|
||||||
*/
|
*/
|
||||||
void drain() {
|
void drain() {
|
||||||
if (!isConsumed) {
|
if (!isConsumed) {
|
||||||
|
@ -151,7 +152,7 @@ public class CompletedFetch {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private <K, V> void maybeEnsureValid(FetchConfig<K, V> fetchConfig, RecordBatch batch) {
|
private void maybeEnsureValid(FetchConfig fetchConfig, RecordBatch batch) {
|
||||||
if (fetchConfig.checkCrcs && batch.magic() >= RecordBatch.MAGIC_VALUE_V2) {
|
if (fetchConfig.checkCrcs && batch.magic() >= RecordBatch.MAGIC_VALUE_V2) {
|
||||||
try {
|
try {
|
||||||
batch.ensureValid();
|
batch.ensureValid();
|
||||||
|
@ -162,7 +163,7 @@ public class CompletedFetch {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private <K, V> void maybeEnsureValid(FetchConfig<K, V> fetchConfig, Record record) {
|
private void maybeEnsureValid(FetchConfig fetchConfig, Record record) {
|
||||||
if (fetchConfig.checkCrcs) {
|
if (fetchConfig.checkCrcs) {
|
||||||
try {
|
try {
|
||||||
record.ensureValid();
|
record.ensureValid();
|
||||||
|
@ -180,7 +181,7 @@ public class CompletedFetch {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private <K, V> Record nextFetchedRecord(FetchConfig<K, V> fetchConfig) {
|
private Record nextFetchedRecord(FetchConfig fetchConfig) {
|
||||||
while (true) {
|
while (true) {
|
||||||
if (records == null || !records.hasNext()) {
|
if (records == null || !records.hasNext()) {
|
||||||
maybeCloseRecordStream();
|
maybeCloseRecordStream();
|
||||||
|
@ -245,11 +246,14 @@ public class CompletedFetch {
|
||||||
* {@link Deserializer deserialization} of the {@link Record record's} key and value are performed in
|
* {@link Deserializer deserialization} of the {@link Record record's} key and value are performed in
|
||||||
* this step.
|
* this step.
|
||||||
*
|
*
|
||||||
* @param fetchConfig {@link FetchConfig Configuration} to use, including, but not limited to, {@link Deserializer}s
|
* @param fetchConfig {@link FetchConfig Configuration} to use
|
||||||
|
* @param deserializers {@link Deserializer}s to use to convert the raw bytes to the expected key and value types
|
||||||
* @param maxRecords The number of records to return; the number returned may be {@code 0 <= maxRecords}
|
* @param maxRecords The number of records to return; the number returned may be {@code 0 <= maxRecords}
|
||||||
* @return {@link ConsumerRecord Consumer records}
|
* @return {@link ConsumerRecord Consumer records}
|
||||||
*/
|
*/
|
||||||
<K, V> List<ConsumerRecord<K, V>> fetchRecords(FetchConfig<K, V> fetchConfig, int maxRecords) {
|
<K, V> List<ConsumerRecord<K, V>> fetchRecords(FetchConfig fetchConfig,
|
||||||
|
Deserializers<K, V> deserializers,
|
||||||
|
int maxRecords) {
|
||||||
// Error when fetching the next record before deserialization.
|
// Error when fetching the next record before deserialization.
|
||||||
if (corruptLastRecord)
|
if (corruptLastRecord)
|
||||||
throw new KafkaException("Received exception when fetching the next record from " + partition
|
throw new KafkaException("Received exception when fetching the next record from " + partition
|
||||||
|
@ -276,7 +280,7 @@ public class CompletedFetch {
|
||||||
|
|
||||||
Optional<Integer> leaderEpoch = maybeLeaderEpoch(currentBatch.partitionLeaderEpoch());
|
Optional<Integer> leaderEpoch = maybeLeaderEpoch(currentBatch.partitionLeaderEpoch());
|
||||||
TimestampType timestampType = currentBatch.timestampType();
|
TimestampType timestampType = currentBatch.timestampType();
|
||||||
ConsumerRecord<K, V> record = parseRecord(fetchConfig, partition, leaderEpoch, timestampType, lastRecord);
|
ConsumerRecord<K, V> record = parseRecord(deserializers, partition, leaderEpoch, timestampType, lastRecord);
|
||||||
records.add(record);
|
records.add(record);
|
||||||
recordsRead++;
|
recordsRead++;
|
||||||
bytesRead += lastRecord.sizeInBytes();
|
bytesRead += lastRecord.sizeInBytes();
|
||||||
|
@ -302,7 +306,7 @@ public class CompletedFetch {
|
||||||
/**
|
/**
|
||||||
* Parse the record entry, deserializing the key / value fields if necessary
|
* Parse the record entry, deserializing the key / value fields if necessary
|
||||||
*/
|
*/
|
||||||
<K, V> ConsumerRecord<K, V> parseRecord(FetchConfig<K, V> fetchConfig,
|
<K, V> ConsumerRecord<K, V> parseRecord(Deserializers<K, V> deserializers,
|
||||||
TopicPartition partition,
|
TopicPartition partition,
|
||||||
Optional<Integer> leaderEpoch,
|
Optional<Integer> leaderEpoch,
|
||||||
TimestampType timestampType,
|
TimestampType timestampType,
|
||||||
|
@ -312,16 +316,16 @@ public class CompletedFetch {
|
||||||
long timestamp = record.timestamp();
|
long timestamp = record.timestamp();
|
||||||
Headers headers = new RecordHeaders(record.headers());
|
Headers headers = new RecordHeaders(record.headers());
|
||||||
ByteBuffer keyBytes = record.key();
|
ByteBuffer keyBytes = record.key();
|
||||||
K key = keyBytes == null ? null : fetchConfig.deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes);
|
K key = keyBytes == null ? null : deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes);
|
||||||
ByteBuffer valueBytes = record.value();
|
ByteBuffer valueBytes = record.value();
|
||||||
V value = valueBytes == null ? null : fetchConfig.deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes);
|
V value = valueBytes == null ? null : deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes);
|
||||||
return new ConsumerRecord<>(partition.topic(), partition.partition(), offset,
|
return new ConsumerRecord<>(partition.topic(), partition.partition(), offset,
|
||||||
timestamp, timestampType,
|
timestamp, timestampType,
|
||||||
keyBytes == null ? ConsumerRecord.NULL_SIZE : keyBytes.remaining(),
|
keyBytes == null ? ConsumerRecord.NULL_SIZE : keyBytes.remaining(),
|
||||||
valueBytes == null ? ConsumerRecord.NULL_SIZE : valueBytes.remaining(),
|
valueBytes == null ? ConsumerRecord.NULL_SIZE : valueBytes.remaining(),
|
||||||
key, value, headers, leaderEpoch);
|
key, value, headers, leaderEpoch);
|
||||||
} catch (RuntimeException e) {
|
} catch (RuntimeException e) {
|
||||||
log.error("Deserializers with error: {}", fetchConfig.deserializers);
|
log.error("Deserializers with error: {}", deserializers);
|
||||||
throw new RecordDeserializationException(partition, record.offset(),
|
throw new RecordDeserializationException(partition, record.offset(),
|
||||||
"Error deserializing key/value for partition " + partition +
|
"Error deserializing key/value for partition " + partition +
|
||||||
" at offset " + record.offset() + ". If needed, please seek past the record to continue consumption.", e);
|
" at offset " + record.offset() + ". If needed, please seek past the record to continue consumption.", e);
|
||||||
|
|
|
@ -955,10 +955,16 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
|
||||||
* @param timer Timer bounding how long this method can block
|
* @param timer Timer bounding how long this method can block
|
||||||
* @return true iff the operation completed within the timeout
|
* @return true iff the operation completed within the timeout
|
||||||
*/
|
*/
|
||||||
public boolean refreshCommittedOffsetsIfNeeded(Timer timer) {
|
public boolean initWithCommittedOffsetsIfNeeded(Timer timer) {
|
||||||
final Set<TopicPartition> initializingPartitions = subscriptions.initializingPartitions();
|
final Set<TopicPartition> initializingPartitions = subscriptions.initializingPartitions();
|
||||||
final Map<TopicPartition, OffsetAndMetadata> offsets = fetchCommittedOffsets(initializingPartitions, timer);
|
final Map<TopicPartition, OffsetAndMetadata> offsets = fetchCommittedOffsets(initializingPartitions, timer);
|
||||||
return refreshCommittedOffsets(offsets, this.metadata, this.subscriptions);
|
|
||||||
|
// "offsets" will be null if the offset fetch requests did not receive responses within the given timeout
|
||||||
|
if (offsets == null)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
refreshCommittedOffsets(offsets, this.metadata, this.subscriptions);
|
||||||
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -0,0 +1,268 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.KafkaClient;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
||||||
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
import org.apache.kafka.common.errors.WakeupException;
|
||||||
|
import org.apache.kafka.common.internals.IdempotentCloser;
|
||||||
|
import org.apache.kafka.common.requests.AbstractRequest;
|
||||||
|
import org.apache.kafka.common.utils.KafkaThread;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
import org.apache.kafka.common.utils.Timer;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS;
|
||||||
|
import static org.apache.kafka.common.utils.Utils.closeQuietly;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Background thread runnable that consumes {@link ApplicationEvent} and produces {@link BackgroundEvent}. It
|
||||||
|
* uses an event loop to consume and produce events, and poll the network client to handle network IO.
|
||||||
|
*/
|
||||||
|
public class ConsumerNetworkThread extends KafkaThread implements Closeable {
|
||||||
|
|
||||||
|
private static final long MAX_POLL_TIMEOUT_MS = 5000;
|
||||||
|
private static final String BACKGROUND_THREAD_NAME = "consumer_background_thread";
|
||||||
|
private final Time time;
|
||||||
|
private final Logger log;
|
||||||
|
private final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier;
|
||||||
|
private final Supplier<NetworkClientDelegate> networkClientDelegateSupplier;
|
||||||
|
private final Supplier<RequestManagers> requestManagersSupplier;
|
||||||
|
private ApplicationEventProcessor applicationEventProcessor;
|
||||||
|
private NetworkClientDelegate networkClientDelegate;
|
||||||
|
private RequestManagers requestManagers;
|
||||||
|
private volatile boolean running;
|
||||||
|
private final IdempotentCloser closer = new IdempotentCloser();
|
||||||
|
private volatile Duration closeTimeout = Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS);
|
||||||
|
|
||||||
|
public ConsumerNetworkThread(LogContext logContext,
|
||||||
|
Time time,
|
||||||
|
Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier,
|
||||||
|
Supplier<NetworkClientDelegate> networkClientDelegateSupplier,
|
||||||
|
Supplier<RequestManagers> requestManagersSupplier) {
|
||||||
|
super(BACKGROUND_THREAD_NAME, true);
|
||||||
|
this.time = time;
|
||||||
|
this.log = logContext.logger(getClass());
|
||||||
|
this.applicationEventProcessorSupplier = applicationEventProcessorSupplier;
|
||||||
|
this.networkClientDelegateSupplier = networkClientDelegateSupplier;
|
||||||
|
this.requestManagersSupplier = requestManagersSupplier;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
closer.assertOpen("Consumer network thread is already closed");
|
||||||
|
running = true;
|
||||||
|
|
||||||
|
try {
|
||||||
|
log.debug("Consumer network thread started");
|
||||||
|
|
||||||
|
// Wait until we're securely in the background network thread to initialize these objects...
|
||||||
|
initializeResources();
|
||||||
|
|
||||||
|
while (running) {
|
||||||
|
try {
|
||||||
|
runOnce();
|
||||||
|
} catch (final WakeupException e) {
|
||||||
|
log.debug("WakeupException caught, consumer network thread won't be interrupted");
|
||||||
|
// swallow the wakeup exception to prevent killing the thread.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (final Throwable t) {
|
||||||
|
log.error("The consumer network thread failed due to unexpected error", t);
|
||||||
|
throw new KafkaException(t);
|
||||||
|
} finally {
|
||||||
|
cleanup();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void initializeResources() {
|
||||||
|
applicationEventProcessor = applicationEventProcessorSupplier.get();
|
||||||
|
networkClientDelegate = networkClientDelegateSupplier.get();
|
||||||
|
requestManagers = requestManagersSupplier.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Poll and process the {@link ApplicationEvent application events}. It performs the following tasks:
|
||||||
|
*
|
||||||
|
* <ol>
|
||||||
|
* <li>
|
||||||
|
* Drains and processes all the events from the application thread's application event queue via
|
||||||
|
* {@link ApplicationEventProcessor}
|
||||||
|
* </li>
|
||||||
|
* <li>
|
||||||
|
* Iterate through the {@link RequestManager} list and invoke {@link RequestManager#poll(long)} to get
|
||||||
|
* the {@link NetworkClientDelegate.UnsentRequest} list and the poll time for the network poll
|
||||||
|
* </li>
|
||||||
|
* <li>
|
||||||
|
* Stage each {@link AbstractRequest.Builder request} to be sent via
|
||||||
|
* {@link NetworkClientDelegate#addAll(List)}
|
||||||
|
* </li>
|
||||||
|
* <li>
|
||||||
|
* Poll the client via {@link KafkaClient#poll(long, long)} to send the requests, as well as
|
||||||
|
* retrieve any available responses
|
||||||
|
* </li>
|
||||||
|
* </ol>
|
||||||
|
*/
|
||||||
|
void runOnce() {
|
||||||
|
// If there are errors processing any events, the error will be thrown immediately. This will have
|
||||||
|
// the effect of closing the background thread.
|
||||||
|
applicationEventProcessor.process();
|
||||||
|
|
||||||
|
final long currentTimeMs = time.milliseconds();
|
||||||
|
final long pollWaitTimeMs = requestManagers.entries().stream()
|
||||||
|
.filter(Optional::isPresent)
|
||||||
|
.map(Optional::get)
|
||||||
|
.map(rm -> rm.poll(currentTimeMs))
|
||||||
|
.map(networkClientDelegate::addAll)
|
||||||
|
.reduce(MAX_POLL_TIMEOUT_MS, Math::min);
|
||||||
|
networkClientDelegate.poll(pollWaitTimeMs, currentTimeMs);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Performs any network I/O that is needed at the time of close for the consumer:
|
||||||
|
*
|
||||||
|
* <ol>
|
||||||
|
* <li>
|
||||||
|
* Iterate through the {@link RequestManager} list and invoke {@link RequestManager#pollOnClose()}
|
||||||
|
* to get the {@link NetworkClientDelegate.UnsentRequest} list and the poll time for the network poll
|
||||||
|
* </li>
|
||||||
|
* <li>
|
||||||
|
* Stage each {@link AbstractRequest.Builder request} to be sent via
|
||||||
|
* {@link NetworkClientDelegate#addAll(List)}
|
||||||
|
* </li>
|
||||||
|
* <li>
|
||||||
|
* {@link KafkaClient#poll(long, long) Poll the client} to send the requests, as well as
|
||||||
|
* retrieve any available responses
|
||||||
|
* </li>
|
||||||
|
* <li>
|
||||||
|
* Continuously {@link KafkaClient#poll(long, long) poll the client} as long as the
|
||||||
|
* {@link Timer#notExpired() timer hasn't expired} to retrieve the responses
|
||||||
|
* </li>
|
||||||
|
* </ol>
|
||||||
|
*/
|
||||||
|
// Visible for testing
|
||||||
|
static void runAtClose(final Collection<Optional<? extends RequestManager>> requestManagers,
|
||||||
|
final NetworkClientDelegate networkClientDelegate,
|
||||||
|
final Timer timer) {
|
||||||
|
// These are the optional outgoing requests at the
|
||||||
|
List<NetworkClientDelegate.PollResult> pollResults = requestManagers.stream()
|
||||||
|
.filter(Optional::isPresent)
|
||||||
|
.map(Optional::get)
|
||||||
|
.map(RequestManager::pollOnClose)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
long pollWaitTimeMs = pollResults.stream()
|
||||||
|
.map(networkClientDelegate::addAll)
|
||||||
|
.reduce(MAX_POLL_TIMEOUT_MS, Math::min);
|
||||||
|
pollWaitTimeMs = Math.min(pollWaitTimeMs, timer.remainingMs());
|
||||||
|
networkClientDelegate.poll(pollWaitTimeMs, timer.currentTimeMs());
|
||||||
|
timer.update();
|
||||||
|
|
||||||
|
List<Future<?>> requestFutures = pollResults.stream()
|
||||||
|
.flatMap(fads -> fads.unsentRequests.stream())
|
||||||
|
.map(NetworkClientDelegate.UnsentRequest::future)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
// Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until
|
||||||
|
// all requests have received a response.
|
||||||
|
while (timer.notExpired() && !requestFutures.stream().allMatch(Future::isDone)) {
|
||||||
|
networkClientDelegate.poll(timer.remainingMs(), timer.currentTimeMs());
|
||||||
|
timer.update();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isRunning() {
|
||||||
|
return running;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void wakeup() {
|
||||||
|
// The network client can be null if the initializeResources method has not yet been called.
|
||||||
|
if (networkClientDelegate != null)
|
||||||
|
networkClientDelegate.wakeup();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
close(closeTimeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close(final Duration timeout) {
|
||||||
|
Objects.requireNonNull(timeout, "Close timeout for consumer network thread must be non-null");
|
||||||
|
|
||||||
|
closer.close(
|
||||||
|
() -> closeInternal(timeout),
|
||||||
|
() -> log.warn("The consumer network thread was already closed")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Starts the closing process.
|
||||||
|
*
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* This method is called from the application thread, but our resources are owned by the network thread. As such,
|
||||||
|
* we don't actually close any of those resources here, immediately, on the application thread. Instead, we just
|
||||||
|
* update our internal state on the application thread. When the network thread next
|
||||||
|
* {@link #run() executes its loop}, it will notice that state, cease processing any further events, and begin
|
||||||
|
* {@link #cleanup() closing its resources}.
|
||||||
|
*
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* This method will wait (i.e. block the application thread) for up to the duration of the given timeout to give
|
||||||
|
* the network thread the time to close down cleanly.
|
||||||
|
*
|
||||||
|
* @param timeout Upper bound of time to wait for the network thread to close its resources
|
||||||
|
*/
|
||||||
|
private void closeInternal(final Duration timeout) {
|
||||||
|
long timeoutMs = timeout.toMillis();
|
||||||
|
log.trace("Signaling the consumer network thread to close in {}ms", timeoutMs);
|
||||||
|
running = false;
|
||||||
|
closeTimeout = timeout;
|
||||||
|
wakeup();
|
||||||
|
|
||||||
|
if (timeoutMs > 0) {
|
||||||
|
try {
|
||||||
|
join(timeoutMs);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
log.error("Interrupted while waiting for consumer network thread to complete", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void cleanup() {
|
||||||
|
log.trace("Closing the consumer network thread");
|
||||||
|
Timer timer = time.timer(closeTimeout);
|
||||||
|
runAtClose(requestManagers.entries(), networkClientDelegate, timer);
|
||||||
|
closeQuietly(requestManagers, "request managers");
|
||||||
|
closeQuietly(networkClientDelegate, "network client delegate");
|
||||||
|
closeQuietly(applicationEventProcessor, "application event processor");
|
||||||
|
log.debug("Closed the consumer network thread");
|
||||||
|
}
|
||||||
|
}
|
|
@ -56,6 +56,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
public final class ConsumerUtils {
|
public final class ConsumerUtils {
|
||||||
|
|
||||||
|
public static final long DEFAULT_CLOSE_TIMEOUT_MS = 30 * 1000;
|
||||||
public static final String CONSUMER_JMX_PREFIX = "kafka.consumer";
|
public static final String CONSUMER_JMX_PREFIX = "kafka.consumer";
|
||||||
public static final String CONSUMER_METRIC_GROUP_PREFIX = "consumer";
|
public static final String CONSUMER_METRIC_GROUP_PREFIX = "consumer";
|
||||||
|
|
||||||
|
@ -142,12 +143,6 @@ public final class ConsumerUtils {
|
||||||
return new FetchMetricsManager(metrics, metricsRegistry);
|
return new FetchMetricsManager(metrics, metricsRegistry);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <K, V> FetchConfig<K, V> createFetchConfig(ConsumerConfig config,
|
|
||||||
Deserializers<K, V> deserializers) {
|
|
||||||
IsolationLevel isolationLevel = configuredIsolationLevel(config);
|
|
||||||
return new FetchConfig<>(config, deserializers, isolationLevel);
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public static <K, V> List<ConsumerInterceptor<K, V>> configuredConsumerInterceptors(ConsumerConfig config) {
|
public static <K, V> List<ConsumerInterceptor<K, V>> configuredConsumerInterceptors(ConsumerConfig config) {
|
||||||
return (List<ConsumerInterceptor<K, V>>) ClientUtils.configuredInterceptors(config, ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, ConsumerInterceptor.class);
|
return (List<ConsumerInterceptor<K, V>>) ClientUtils.configuredInterceptors(config, ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, ConsumerInterceptor.class);
|
||||||
|
@ -168,14 +163,10 @@ public final class ConsumerUtils {
|
||||||
* committed offsets' metadata.
|
* committed offsets' metadata.
|
||||||
* @param subscriptions Subscription state to update, setting partitions' offsets to the
|
* @param subscriptions Subscription state to update, setting partitions' offsets to the
|
||||||
* committed offsets.
|
* committed offsets.
|
||||||
* @return False if null <code>offsetsAndMetadata</code> is provided, indicating that the
|
|
||||||
* refresh operation could not be performed. True in any other case.
|
|
||||||
*/
|
*/
|
||||||
public static boolean refreshCommittedOffsets(final Map<TopicPartition, OffsetAndMetadata> offsetsAndMetadata,
|
public static void refreshCommittedOffsets(final Map<TopicPartition, OffsetAndMetadata> offsetsAndMetadata,
|
||||||
final ConsumerMetadata metadata,
|
final ConsumerMetadata metadata,
|
||||||
final SubscriptionState subscriptions) {
|
final SubscriptionState subscriptions) {
|
||||||
if (offsetsAndMetadata == null) return false;
|
|
||||||
|
|
||||||
for (final Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsetsAndMetadata.entrySet()) {
|
for (final Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsetsAndMetadata.entrySet()) {
|
||||||
final TopicPartition tp = entry.getKey();
|
final TopicPartition tp = entry.getKey();
|
||||||
final OffsetAndMetadata offsetAndMetadata = entry.getValue();
|
final OffsetAndMetadata offsetAndMetadata = entry.getValue();
|
||||||
|
@ -200,7 +191,6 @@ public final class ConsumerUtils {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <T> T getResult(CompletableFuture<T> future, Timer timer) {
|
public static <T> T getResult(CompletableFuture<T> future, Timer timer) {
|
||||||
|
|
|
@ -16,6 +16,9 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent;
|
||||||
|
import org.apache.kafka.common.KafkaException;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
||||||
import org.apache.kafka.common.errors.RetriableException;
|
import org.apache.kafka.common.errors.RetriableException;
|
||||||
|
@ -28,10 +31,11 @@ import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is responsible for timing to send the next {@link FindCoordinatorRequest} based on the following criteria:
|
* This is responsible for timing to send the next {@link FindCoordinatorRequest} based on the following criteria:
|
||||||
* <p/>
|
* <p/>
|
||||||
|
@ -49,7 +53,7 @@ public class CoordinatorRequestManager implements RequestManager {
|
||||||
private static final long COORDINATOR_DISCONNECT_LOGGING_INTERVAL_MS = 60 * 1000;
|
private static final long COORDINATOR_DISCONNECT_LOGGING_INTERVAL_MS = 60 * 1000;
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final Logger log;
|
private final Logger log;
|
||||||
private final ErrorEventHandler nonRetriableErrorHandler;
|
private final BackgroundEventHandler backgroundEventHandler;
|
||||||
private final String groupId;
|
private final String groupId;
|
||||||
|
|
||||||
private final RequestState coordinatorRequestState;
|
private final RequestState coordinatorRequestState;
|
||||||
|
@ -62,13 +66,13 @@ public class CoordinatorRequestManager implements RequestManager {
|
||||||
final LogContext logContext,
|
final LogContext logContext,
|
||||||
final long retryBackoffMs,
|
final long retryBackoffMs,
|
||||||
final long retryBackoffMaxMs,
|
final long retryBackoffMaxMs,
|
||||||
final ErrorEventHandler errorHandler,
|
final BackgroundEventHandler errorHandler,
|
||||||
final String groupId
|
final String groupId
|
||||||
) {
|
) {
|
||||||
Objects.requireNonNull(groupId);
|
Objects.requireNonNull(groupId);
|
||||||
this.time = time;
|
this.time = time;
|
||||||
this.log = logContext.logger(this.getClass());
|
this.log = logContext.logger(this.getClass());
|
||||||
this.nonRetriableErrorHandler = errorHandler;
|
this.backgroundEventHandler = errorHandler;
|
||||||
this.groupId = groupId;
|
this.groupId = groupId;
|
||||||
this.coordinatorRequestState = new RequestState(
|
this.coordinatorRequestState = new RequestState(
|
||||||
logContext,
|
logContext,
|
||||||
|
@ -90,18 +94,15 @@ public class CoordinatorRequestManager implements RequestManager {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
|
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
|
||||||
if (this.coordinator != null) {
|
if (this.coordinator != null)
|
||||||
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList());
|
return EMPTY;
|
||||||
}
|
|
||||||
|
|
||||||
if (coordinatorRequestState.canSendRequest(currentTimeMs)) {
|
if (coordinatorRequestState.canSendRequest(currentTimeMs)) {
|
||||||
NetworkClientDelegate.UnsentRequest request = makeFindCoordinatorRequest(currentTimeMs);
|
NetworkClientDelegate.UnsentRequest request = makeFindCoordinatorRequest(currentTimeMs);
|
||||||
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.singletonList(request));
|
return new NetworkClientDelegate.PollResult(request);
|
||||||
}
|
}
|
||||||
|
|
||||||
return new NetworkClientDelegate.PollResult(
|
return new NetworkClientDelegate.PollResult(coordinatorRequestState.remainingBackoffMs(currentTimeMs));
|
||||||
coordinatorRequestState.remainingBackoffMs(currentTimeMs),
|
|
||||||
Collections.emptyList());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long currentTimeMs) {
|
private NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long currentTimeMs) {
|
||||||
|
@ -175,12 +176,13 @@ public class CoordinatorRequestManager implements RequestManager {
|
||||||
|
|
||||||
if (exception == Errors.GROUP_AUTHORIZATION_FAILED.exception()) {
|
if (exception == Errors.GROUP_AUTHORIZATION_FAILED.exception()) {
|
||||||
log.debug("FindCoordinator request failed due to authorization error {}", exception.getMessage());
|
log.debug("FindCoordinator request failed due to authorization error {}", exception.getMessage());
|
||||||
nonRetriableErrorHandler.handle(GroupAuthorizationException.forGroupId(this.groupId));
|
KafkaException groupAuthorizationException = GroupAuthorizationException.forGroupId(this.groupId);
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(groupAuthorizationException));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
log.warn("FindCoordinator request failed due to fatal exception", exception);
|
log.warn("FindCoordinator request failed due to fatal exception", exception);
|
||||||
nonRetriableErrorHandler.handle(exception);
|
backgroundEventHandler.add(new ErrorBackgroundEvent(exception));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -1,293 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
|
||||||
|
|
||||||
import org.apache.kafka.clients.ApiVersions;
|
|
||||||
import org.apache.kafka.clients.ClientUtils;
|
|
||||||
import org.apache.kafka.clients.GroupRebalanceConfig;
|
|
||||||
import org.apache.kafka.clients.NetworkClient;
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
|
||||||
import org.apache.kafka.common.KafkaException;
|
|
||||||
import org.apache.kafka.common.errors.WakeupException;
|
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
|
||||||
import org.apache.kafka.common.metrics.Sensor;
|
|
||||||
import org.apache.kafka.common.utils.KafkaThread;
|
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
|
||||||
import org.apache.kafka.common.utils.Time;
|
|
||||||
import org.apache.kafka.common.utils.Utils;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
|
|
||||||
import java.util.LinkedList;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.concurrent.BlockingQueue;
|
|
||||||
|
|
||||||
import static java.util.Objects.requireNonNull;
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION;
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredIsolationLevel;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Background thread runnable that consumes {@code ApplicationEvent} and
|
|
||||||
* produces {@code BackgroundEvent}. It uses an event loop to consume and
|
|
||||||
* produce events, and poll the network client to handle network IO.
|
|
||||||
* <p/>
|
|
||||||
* It holds a reference to the {@link SubscriptionState}, which is
|
|
||||||
* initialized by the polling thread.
|
|
||||||
* <p/>
|
|
||||||
* For processing application events that have been submitted to the
|
|
||||||
* {@link #applicationEventQueue}, this relies on an {@link ApplicationEventProcessor}. Processing includes generating requests and
|
|
||||||
* handling responses with the appropriate {@link RequestManager}. The network operations for
|
|
||||||
* actually sending the requests is delegated to the {@link NetworkClientDelegate}
|
|
||||||
* </li>
|
|
||||||
*/
|
|
||||||
public class DefaultBackgroundThread extends KafkaThread {
|
|
||||||
private static final long MAX_POLL_TIMEOUT_MS = 5000;
|
|
||||||
private static final String BACKGROUND_THREAD_NAME = "consumer_background_thread";
|
|
||||||
private final Time time;
|
|
||||||
private final Logger log;
|
|
||||||
private final BlockingQueue<ApplicationEvent> applicationEventQueue;
|
|
||||||
private final BlockingQueue<BackgroundEvent> backgroundEventQueue;
|
|
||||||
private final ConsumerMetadata metadata;
|
|
||||||
private final ConsumerConfig config;
|
|
||||||
// empty if groupId is null
|
|
||||||
private final ApplicationEventProcessor applicationEventProcessor;
|
|
||||||
private final NetworkClientDelegate networkClientDelegate;
|
|
||||||
private final ErrorEventHandler errorEventHandler;
|
|
||||||
private final GroupState groupState;
|
|
||||||
private boolean running;
|
|
||||||
|
|
||||||
private final RequestManagers requestManagers;
|
|
||||||
|
|
||||||
// Visible for testing
|
|
||||||
@SuppressWarnings("ParameterNumber")
|
|
||||||
DefaultBackgroundThread(final Time time,
|
|
||||||
final ConsumerConfig config,
|
|
||||||
final LogContext logContext,
|
|
||||||
final BlockingQueue<ApplicationEvent> applicationEventQueue,
|
|
||||||
final BlockingQueue<BackgroundEvent> backgroundEventQueue,
|
|
||||||
final ErrorEventHandler errorEventHandler,
|
|
||||||
final ApplicationEventProcessor processor,
|
|
||||||
final ConsumerMetadata metadata,
|
|
||||||
final NetworkClientDelegate networkClient,
|
|
||||||
final GroupState groupState,
|
|
||||||
final CoordinatorRequestManager coordinatorManager,
|
|
||||||
final CommitRequestManager commitRequestManager,
|
|
||||||
final OffsetsRequestManager offsetsRequestManager,
|
|
||||||
final TopicMetadataRequestManager topicMetadataRequestManager,
|
|
||||||
final HeartbeatRequestManager heartbeatRequestManager) {
|
|
||||||
super(BACKGROUND_THREAD_NAME, true);
|
|
||||||
this.time = time;
|
|
||||||
this.running = true;
|
|
||||||
this.log = logContext.logger(getClass());
|
|
||||||
this.applicationEventQueue = applicationEventQueue;
|
|
||||||
this.backgroundEventQueue = backgroundEventQueue;
|
|
||||||
this.applicationEventProcessor = processor;
|
|
||||||
this.config = config;
|
|
||||||
this.metadata = metadata;
|
|
||||||
this.networkClientDelegate = networkClient;
|
|
||||||
this.errorEventHandler = errorEventHandler;
|
|
||||||
this.groupState = groupState;
|
|
||||||
this.requestManagers = new RequestManagers(
|
|
||||||
offsetsRequestManager,
|
|
||||||
topicMetadataRequestManager,
|
|
||||||
Optional.ofNullable(coordinatorManager),
|
|
||||||
Optional.ofNullable(commitRequestManager),
|
|
||||||
Optional.ofNullable(heartbeatRequestManager));
|
|
||||||
}
|
|
||||||
|
|
||||||
public DefaultBackgroundThread(final Time time,
|
|
||||||
final ConsumerConfig config,
|
|
||||||
final GroupRebalanceConfig rebalanceConfig,
|
|
||||||
final LogContext logContext,
|
|
||||||
final BlockingQueue<ApplicationEvent> applicationEventQueue,
|
|
||||||
final BlockingQueue<BackgroundEvent> backgroundEventQueue,
|
|
||||||
final ConsumerMetadata metadata,
|
|
||||||
final SubscriptionState subscriptionState,
|
|
||||||
final ApiVersions apiVersions,
|
|
||||||
final Metrics metrics,
|
|
||||||
final Sensor fetcherThrottleTimeSensor) {
|
|
||||||
super(BACKGROUND_THREAD_NAME, true);
|
|
||||||
requireNonNull(config);
|
|
||||||
requireNonNull(rebalanceConfig);
|
|
||||||
requireNonNull(logContext);
|
|
||||||
requireNonNull(applicationEventQueue);
|
|
||||||
requireNonNull(backgroundEventQueue);
|
|
||||||
requireNonNull(metadata);
|
|
||||||
requireNonNull(subscriptionState);
|
|
||||||
try {
|
|
||||||
this.time = time;
|
|
||||||
this.log = logContext.logger(getClass());
|
|
||||||
this.applicationEventQueue = applicationEventQueue;
|
|
||||||
this.backgroundEventQueue = backgroundEventQueue;
|
|
||||||
this.config = config;
|
|
||||||
this.metadata = metadata;
|
|
||||||
final NetworkClient networkClient = ClientUtils.createNetworkClient(config,
|
|
||||||
metrics,
|
|
||||||
CONSUMER_METRIC_GROUP_PREFIX,
|
|
||||||
logContext,
|
|
||||||
apiVersions,
|
|
||||||
time,
|
|
||||||
CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION,
|
|
||||||
metadata,
|
|
||||||
fetcherThrottleTimeSensor);
|
|
||||||
this.networkClientDelegate = new NetworkClientDelegate(
|
|
||||||
this.time,
|
|
||||||
this.config,
|
|
||||||
logContext,
|
|
||||||
networkClient);
|
|
||||||
this.running = true;
|
|
||||||
this.errorEventHandler = new ErrorEventHandler(this.backgroundEventQueue);
|
|
||||||
this.groupState = new GroupState(rebalanceConfig);
|
|
||||||
long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
|
|
||||||
long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
|
|
||||||
final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
|
|
||||||
|
|
||||||
OffsetsRequestManager offsetsRequestManager =
|
|
||||||
new OffsetsRequestManager(
|
|
||||||
subscriptionState,
|
|
||||||
metadata,
|
|
||||||
configuredIsolationLevel(config),
|
|
||||||
time,
|
|
||||||
retryBackoffMs,
|
|
||||||
requestTimeoutMs,
|
|
||||||
apiVersions,
|
|
||||||
networkClientDelegate,
|
|
||||||
logContext);
|
|
||||||
CoordinatorRequestManager coordinatorRequestManager = null;
|
|
||||||
CommitRequestManager commitRequestManager = null;
|
|
||||||
TopicMetadataRequestManager topicMetadataRequestManger = new TopicMetadataRequestManager(
|
|
||||||
logContext,
|
|
||||||
config);
|
|
||||||
HeartbeatRequestManager heartbeatRequestManager = null;
|
|
||||||
|
|
||||||
// TODO: consolidate groupState and memberState
|
|
||||||
if (groupState.groupId != null) {
|
|
||||||
coordinatorRequestManager = new CoordinatorRequestManager(
|
|
||||||
this.time,
|
|
||||||
logContext,
|
|
||||||
retryBackoffMs,
|
|
||||||
retryBackoffMaxMs,
|
|
||||||
this.errorEventHandler,
|
|
||||||
groupState.groupId);
|
|
||||||
commitRequestManager = new CommitRequestManager(
|
|
||||||
this.time,
|
|
||||||
logContext,
|
|
||||||
subscriptionState,
|
|
||||||
config,
|
|
||||||
coordinatorRequestManager,
|
|
||||||
groupState);
|
|
||||||
MembershipManager membershipManager = new MembershipManagerImpl(groupState.groupId, logContext);
|
|
||||||
heartbeatRequestManager = new HeartbeatRequestManager(
|
|
||||||
this.time,
|
|
||||||
logContext,
|
|
||||||
config,
|
|
||||||
coordinatorRequestManager,
|
|
||||||
subscriptionState,
|
|
||||||
membershipManager,
|
|
||||||
errorEventHandler);
|
|
||||||
}
|
|
||||||
|
|
||||||
this.requestManagers = new RequestManagers(
|
|
||||||
offsetsRequestManager,
|
|
||||||
topicMetadataRequestManger,
|
|
||||||
Optional.ofNullable(coordinatorRequestManager),
|
|
||||||
Optional.ofNullable(commitRequestManager),
|
|
||||||
Optional.ofNullable(heartbeatRequestManager));
|
|
||||||
this.applicationEventProcessor = new ApplicationEventProcessor(
|
|
||||||
backgroundEventQueue,
|
|
||||||
requestManagers,
|
|
||||||
metadata);
|
|
||||||
} catch (final Exception e) {
|
|
||||||
close();
|
|
||||||
throw new KafkaException("Failed to construct background processor", e.getCause());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
try {
|
|
||||||
log.debug("Background thread started");
|
|
||||||
while (running) {
|
|
||||||
try {
|
|
||||||
runOnce();
|
|
||||||
} catch (final WakeupException e) {
|
|
||||||
log.debug("WakeupException caught, background thread won't be interrupted");
|
|
||||||
// swallow the wakeup exception to prevent killing the background thread.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (final Throwable t) {
|
|
||||||
log.error("The background thread failed due to unexpected error", t);
|
|
||||||
throw new KafkaException(t);
|
|
||||||
} finally {
|
|
||||||
close();
|
|
||||||
log.debug("{} closed", getClass());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Poll and process an {@link ApplicationEvent}. It performs the following tasks:
|
|
||||||
* 1. Drains and try to process all the requests in the queue.
|
|
||||||
* 2. Iterate through the registry, poll, and get the next poll time for the network poll
|
|
||||||
* 3. Poll the networkClient to send and retrieve the response.
|
|
||||||
*/
|
|
||||||
void runOnce() {
|
|
||||||
if (!applicationEventQueue.isEmpty()) {
|
|
||||||
LinkedList<ApplicationEvent> res = new LinkedList<>();
|
|
||||||
this.applicationEventQueue.drainTo(res);
|
|
||||||
|
|
||||||
for (ApplicationEvent event : res) {
|
|
||||||
log.debug("Consuming application event: {}", event);
|
|
||||||
Objects.requireNonNull(event);
|
|
||||||
applicationEventProcessor.process(event);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
final long currentTimeMs = time.milliseconds();
|
|
||||||
final long pollWaitTimeMs = requestManagers.entries().stream()
|
|
||||||
.filter(Optional::isPresent)
|
|
||||||
.map(m -> m.get().poll(currentTimeMs))
|
|
||||||
.map(this::handlePollResult)
|
|
||||||
.reduce(MAX_POLL_TIMEOUT_MS, Math::min);
|
|
||||||
networkClientDelegate.poll(pollWaitTimeMs, currentTimeMs);
|
|
||||||
}
|
|
||||||
|
|
||||||
long handlePollResult(NetworkClientDelegate.PollResult res) {
|
|
||||||
if (!res.unsentRequests.isEmpty()) {
|
|
||||||
networkClientDelegate.addAll(res.unsentRequests);
|
|
||||||
}
|
|
||||||
return res.timeUntilNextPollMs;
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean isRunning() {
|
|
||||||
return this.running;
|
|
||||||
}
|
|
||||||
|
|
||||||
public final void wakeup() {
|
|
||||||
networkClientDelegate.wakeup();
|
|
||||||
}
|
|
||||||
|
|
||||||
public final void close() {
|
|
||||||
this.running = false;
|
|
||||||
this.wakeup();
|
|
||||||
Utils.closeQuietly(networkClientDelegate, "network client utils");
|
|
||||||
Utils.closeQuietly(metadata, "consumer metadata client");
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,151 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
|
||||||
|
|
||||||
import org.apache.kafka.clients.ApiVersions;
|
|
||||||
import org.apache.kafka.clients.ClientUtils;
|
|
||||||
import org.apache.kafka.clients.GroupRebalanceConfig;
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.EventHandler;
|
|
||||||
import org.apache.kafka.common.internals.ClusterResourceListeners;
|
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
|
||||||
import org.apache.kafka.common.metrics.Sensor;
|
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
|
||||||
import org.apache.kafka.common.utils.Time;
|
|
||||||
import org.apache.kafka.common.utils.Timer;
|
|
||||||
|
|
||||||
import java.net.InetSocketAddress;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.concurrent.BlockingQueue;
|
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* An {@code EventHandler} that uses a single background thread to consume {@code ApplicationEvent} and produce
|
|
||||||
* {@code BackgroundEvent} from the {@link DefaultBackgroundThread}.
|
|
||||||
*/
|
|
||||||
public class DefaultEventHandler implements EventHandler {
|
|
||||||
|
|
||||||
private final BlockingQueue<ApplicationEvent> applicationEventQueue;
|
|
||||||
private final BlockingQueue<BackgroundEvent> backgroundEventQueue;
|
|
||||||
private final DefaultBackgroundThread backgroundThread;
|
|
||||||
|
|
||||||
public DefaultEventHandler(final ConsumerConfig config,
|
|
||||||
final GroupRebalanceConfig groupRebalanceConfig,
|
|
||||||
final LogContext logContext,
|
|
||||||
final SubscriptionState subscriptionState,
|
|
||||||
final ApiVersions apiVersions,
|
|
||||||
final Metrics metrics,
|
|
||||||
final ClusterResourceListeners clusterResourceListeners,
|
|
||||||
final Sensor fetcherThrottleTimeSensor) {
|
|
||||||
this(Time.SYSTEM,
|
|
||||||
config,
|
|
||||||
groupRebalanceConfig,
|
|
||||||
logContext,
|
|
||||||
new LinkedBlockingQueue<>(),
|
|
||||||
new LinkedBlockingQueue<>(),
|
|
||||||
subscriptionState,
|
|
||||||
apiVersions,
|
|
||||||
metrics,
|
|
||||||
clusterResourceListeners,
|
|
||||||
fetcherThrottleTimeSensor);
|
|
||||||
}
|
|
||||||
|
|
||||||
public DefaultEventHandler(final Time time,
|
|
||||||
final ConsumerConfig config,
|
|
||||||
final GroupRebalanceConfig groupRebalanceConfig,
|
|
||||||
final LogContext logContext,
|
|
||||||
final BlockingQueue<ApplicationEvent> applicationEventQueue,
|
|
||||||
final BlockingQueue<BackgroundEvent> backgroundEventQueue,
|
|
||||||
final SubscriptionState subscriptionState,
|
|
||||||
final ApiVersions apiVersions,
|
|
||||||
final Metrics metrics,
|
|
||||||
final ClusterResourceListeners clusterResourceListeners,
|
|
||||||
final Sensor fetcherThrottleTimeSensor) {
|
|
||||||
this.applicationEventQueue = applicationEventQueue;
|
|
||||||
this.backgroundEventQueue = backgroundEventQueue;
|
|
||||||
|
|
||||||
// Bootstrap a metadata object with the bootstrap server IP address, which will be used once for the
|
|
||||||
// subsequent metadata refresh once the background thread has started up.
|
|
||||||
final ConsumerMetadata metadata = new ConsumerMetadata(config,
|
|
||||||
subscriptionState,
|
|
||||||
logContext,
|
|
||||||
clusterResourceListeners);
|
|
||||||
final List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config);
|
|
||||||
metadata.bootstrap(addresses);
|
|
||||||
|
|
||||||
this.backgroundThread = new DefaultBackgroundThread(
|
|
||||||
time,
|
|
||||||
config,
|
|
||||||
groupRebalanceConfig,
|
|
||||||
logContext,
|
|
||||||
this.applicationEventQueue,
|
|
||||||
this.backgroundEventQueue,
|
|
||||||
metadata,
|
|
||||||
subscriptionState,
|
|
||||||
apiVersions,
|
|
||||||
metrics,
|
|
||||||
fetcherThrottleTimeSensor);
|
|
||||||
backgroundThread.start();
|
|
||||||
}
|
|
||||||
|
|
||||||
// VisibleForTesting
|
|
||||||
DefaultEventHandler(final DefaultBackgroundThread backgroundThread,
|
|
||||||
final BlockingQueue<ApplicationEvent> applicationEventQueue,
|
|
||||||
final BlockingQueue<BackgroundEvent> backgroundEventQueue) {
|
|
||||||
this.backgroundThread = backgroundThread;
|
|
||||||
this.applicationEventQueue = applicationEventQueue;
|
|
||||||
this.backgroundEventQueue = backgroundEventQueue;
|
|
||||||
backgroundThread.start();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Optional<BackgroundEvent> poll() {
|
|
||||||
return Optional.ofNullable(backgroundEventQueue.poll());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isEmpty() {
|
|
||||||
return backgroundEventQueue.isEmpty();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean add(final ApplicationEvent event) {
|
|
||||||
backgroundThread.wakeup();
|
|
||||||
return applicationEventQueue.add(event);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public <T> T addAndGet(final CompletableApplicationEvent<T> event, final Timer timer) {
|
|
||||||
Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null");
|
|
||||||
Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null");
|
|
||||||
add(event);
|
|
||||||
return event.get(timer);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void close() {
|
|
||||||
try {
|
|
||||||
backgroundThread.close();
|
|
||||||
} catch (final Exception e) {
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -75,4 +75,12 @@ public class Deserializers<K, V> implements AutoCloseable {
|
||||||
throw new KafkaException("Failed to close deserializers", exception);
|
throw new KafkaException("Failed to close deserializers", exception);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "Deserializers{" +
|
||||||
|
"keyDeserializer=" + keyDeserializer +
|
||||||
|
", valueDeserializer=" + valueDeserializer +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,15 +17,21 @@
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.InterruptException;
|
||||||
import org.apache.kafka.common.internals.IdempotentCloser;
|
import org.apache.kafka.common.internals.IdempotentCloser;
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.apache.kafka.common.utils.Timer;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
import java.io.Closeable;
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.locks.Condition;
|
||||||
|
import java.util.concurrent.locks.Lock;
|
||||||
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -35,12 +41,15 @@ import java.util.function.Predicate;
|
||||||
*
|
*
|
||||||
* <p/>
|
* <p/>
|
||||||
*
|
*
|
||||||
* <em>Note</em>: this class is not thread-safe and is intended to only be used from a single thread.
|
* <em>Note</em>: this class is thread-safe with the intention that {@link CompletedFetch the data} will be
|
||||||
|
* "produced" by a background thread and consumed by the application thread.
|
||||||
*/
|
*/
|
||||||
public class FetchBuffer implements Closeable {
|
public class FetchBuffer implements AutoCloseable {
|
||||||
|
|
||||||
private final Logger log;
|
private final Logger log;
|
||||||
private final ConcurrentLinkedQueue<CompletedFetch> completedFetches;
|
private final ConcurrentLinkedQueue<CompletedFetch> completedFetches;
|
||||||
|
private final Lock lock;
|
||||||
|
private final Condition notEmptyCondition;
|
||||||
private final IdempotentCloser idempotentCloser = new IdempotentCloser();
|
private final IdempotentCloser idempotentCloser = new IdempotentCloser();
|
||||||
|
|
||||||
private CompletedFetch nextInLineFetch;
|
private CompletedFetch nextInLineFetch;
|
||||||
|
@ -48,6 +57,8 @@ public class FetchBuffer implements Closeable {
|
||||||
public FetchBuffer(final LogContext logContext) {
|
public FetchBuffer(final LogContext logContext) {
|
||||||
this.log = logContext.logger(FetchBuffer.class);
|
this.log = logContext.logger(FetchBuffer.class);
|
||||||
this.completedFetches = new ConcurrentLinkedQueue<>();
|
this.completedFetches = new ConcurrentLinkedQueue<>();
|
||||||
|
this.lock = new ReentrantLock();
|
||||||
|
this.notEmptyCondition = lock.newCondition();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -56,7 +67,12 @@ public class FetchBuffer implements Closeable {
|
||||||
* @return {@code true} if the buffer is empty, {@code false} otherwise
|
* @return {@code true} if the buffer is empty, {@code false} otherwise
|
||||||
*/
|
*/
|
||||||
boolean isEmpty() {
|
boolean isEmpty() {
|
||||||
return completedFetches.isEmpty();
|
try {
|
||||||
|
lock.lock();
|
||||||
|
return completedFetches.isEmpty();
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -66,31 +82,107 @@ public class FetchBuffer implements Closeable {
|
||||||
* @return {@code true} if there are completed fetches that match the {@link Predicate}, {@code false} otherwise
|
* @return {@code true} if there are completed fetches that match the {@link Predicate}, {@code false} otherwise
|
||||||
*/
|
*/
|
||||||
boolean hasCompletedFetches(Predicate<CompletedFetch> predicate) {
|
boolean hasCompletedFetches(Predicate<CompletedFetch> predicate) {
|
||||||
return completedFetches.stream().anyMatch(predicate);
|
try {
|
||||||
|
lock.lock();
|
||||||
|
return completedFetches.stream().anyMatch(predicate);
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void add(CompletedFetch completedFetch) {
|
void add(CompletedFetch completedFetch) {
|
||||||
completedFetches.add(completedFetch);
|
try {
|
||||||
|
lock.lock();
|
||||||
|
completedFetches.add(completedFetch);
|
||||||
|
notEmptyCondition.signalAll();
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void addAll(Collection<CompletedFetch> completedFetches) {
|
void addAll(Collection<CompletedFetch> completedFetches) {
|
||||||
this.completedFetches.addAll(completedFetches);
|
if (completedFetches == null || completedFetches.isEmpty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
try {
|
||||||
|
lock.lock();
|
||||||
|
this.completedFetches.addAll(completedFetches);
|
||||||
|
notEmptyCondition.signalAll();
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
CompletedFetch nextInLineFetch() {
|
CompletedFetch nextInLineFetch() {
|
||||||
return nextInLineFetch;
|
try {
|
||||||
|
lock.lock();
|
||||||
|
return nextInLineFetch;
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void setNextInLineFetch(CompletedFetch completedFetch) {
|
void setNextInLineFetch(CompletedFetch nextInLineFetch) {
|
||||||
this.nextInLineFetch = completedFetch;
|
try {
|
||||||
|
lock.lock();
|
||||||
|
this.nextInLineFetch = nextInLineFetch;
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
CompletedFetch peek() {
|
CompletedFetch peek() {
|
||||||
return completedFetches.peek();
|
try {
|
||||||
|
lock.lock();
|
||||||
|
return completedFetches.peek();
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
CompletedFetch poll() {
|
CompletedFetch poll() {
|
||||||
return completedFetches.poll();
|
try {
|
||||||
|
lock.lock();
|
||||||
|
return completedFetches.poll();
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Allows the caller to await presence of data in the buffer. The method will block, returning only
|
||||||
|
* under one of the following conditions:
|
||||||
|
*
|
||||||
|
* <ol>
|
||||||
|
* <li>The buffer was already non-empty on entry</li>
|
||||||
|
* <li>The buffer was populated during the wait</li>
|
||||||
|
* <li>The remaining time on the {@link Timer timer} elapsed</li>
|
||||||
|
* <li>The thread was interrupted</li>
|
||||||
|
* </ol>
|
||||||
|
*
|
||||||
|
* @param timer Timer that provides time to wait
|
||||||
|
*/
|
||||||
|
void awaitNotEmpty(Timer timer) {
|
||||||
|
try {
|
||||||
|
lock.lock();
|
||||||
|
|
||||||
|
while (isEmpty()) {
|
||||||
|
// Update the timer before we head into the loop in case it took a while to get the lock.
|
||||||
|
timer.update();
|
||||||
|
|
||||||
|
if (timer.isExpired())
|
||||||
|
break;
|
||||||
|
|
||||||
|
if (!notEmptyCondition.await(timer.remainingMs(), TimeUnit.MILLISECONDS)) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new InterruptException("Timeout waiting for results from fetching records", e);
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
timer.update();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -100,12 +192,22 @@ public class FetchBuffer implements Closeable {
|
||||||
* @param partitions {@link Set} of {@link TopicPartition}s for which any buffered data should be kept
|
* @param partitions {@link Set} of {@link TopicPartition}s for which any buffered data should be kept
|
||||||
*/
|
*/
|
||||||
void retainAll(final Set<TopicPartition> partitions) {
|
void retainAll(final Set<TopicPartition> partitions) {
|
||||||
completedFetches.removeIf(cf -> maybeDrain(partitions, cf));
|
try {
|
||||||
|
lock.lock();
|
||||||
|
|
||||||
if (maybeDrain(partitions, nextInLineFetch))
|
completedFetches.removeIf(cf -> maybeDrain(partitions, cf));
|
||||||
nextInLineFetch = null;
|
|
||||||
|
if (maybeDrain(partitions, nextInLineFetch))
|
||||||
|
nextInLineFetch = null;
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Drains (i.e. <em>removes</em>) the contents of the given {@link CompletedFetch} as its data should not
|
||||||
|
* be returned to the user.
|
||||||
|
*/
|
||||||
private boolean maybeDrain(final Set<TopicPartition> partitions, final CompletedFetch completedFetch) {
|
private boolean maybeDrain(final Set<TopicPartition> partitions, final CompletedFetch completedFetch) {
|
||||||
if (completedFetch != null && !partitions.contains(completedFetch.partition)) {
|
if (completedFetch != null && !partitions.contains(completedFetch.partition)) {
|
||||||
log.debug("Removing {} from buffered fetch data as it is not in the set of partitions to retain ({})", completedFetch.partition, partitions);
|
log.debug("Removing {} from buffered fetch data as it is not in the set of partitions to retain ({})", completedFetch.partition, partitions);
|
||||||
|
@ -122,28 +224,33 @@ public class FetchBuffer implements Closeable {
|
||||||
* @return {@link TopicPartition Partition} set
|
* @return {@link TopicPartition Partition} set
|
||||||
*/
|
*/
|
||||||
Set<TopicPartition> bufferedPartitions() {
|
Set<TopicPartition> bufferedPartitions() {
|
||||||
final Set<TopicPartition> partitions = new HashSet<>();
|
try {
|
||||||
|
lock.lock();
|
||||||
|
|
||||||
if (nextInLineFetch != null && !nextInLineFetch.isConsumed()) {
|
final Set<TopicPartition> partitions = new HashSet<>();
|
||||||
partitions.add(nextInLineFetch.partition);
|
|
||||||
|
if (nextInLineFetch != null && !nextInLineFetch.isConsumed()) {
|
||||||
|
partitions.add(nextInLineFetch.partition);
|
||||||
|
}
|
||||||
|
|
||||||
|
completedFetches.forEach(cf -> partitions.add(cf.partition));
|
||||||
|
return partitions;
|
||||||
|
} finally {
|
||||||
|
lock.unlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
completedFetches.forEach(cf -> partitions.add(cf.partition));
|
|
||||||
return partitions;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
idempotentCloser.close(() -> {
|
try {
|
||||||
log.debug("Closing the fetch buffer");
|
lock.lock();
|
||||||
|
|
||||||
if (nextInLineFetch != null) {
|
idempotentCloser.close(
|
||||||
nextInLineFetch.drain();
|
() -> retainAll(Collections.emptySet()),
|
||||||
nextInLineFetch = null;
|
() -> log.warn("The fetch buffer was already closed")
|
||||||
}
|
);
|
||||||
|
} finally {
|
||||||
completedFetches.forEach(CompletedFetch::drain);
|
lock.unlock();
|
||||||
completedFetches.clear();
|
}
|
||||||
}, () -> log.warn("The fetch buffer was previously closed"));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -54,20 +54,23 @@ public class FetchCollector<K, V> {
|
||||||
private final Logger log;
|
private final Logger log;
|
||||||
private final ConsumerMetadata metadata;
|
private final ConsumerMetadata metadata;
|
||||||
private final SubscriptionState subscriptions;
|
private final SubscriptionState subscriptions;
|
||||||
private final FetchConfig<K, V> fetchConfig;
|
private final FetchConfig fetchConfig;
|
||||||
|
private final Deserializers<K, V> deserializers;
|
||||||
private final FetchMetricsManager metricsManager;
|
private final FetchMetricsManager metricsManager;
|
||||||
private final Time time;
|
private final Time time;
|
||||||
|
|
||||||
public FetchCollector(final LogContext logContext,
|
public FetchCollector(final LogContext logContext,
|
||||||
final ConsumerMetadata metadata,
|
final ConsumerMetadata metadata,
|
||||||
final SubscriptionState subscriptions,
|
final SubscriptionState subscriptions,
|
||||||
final FetchConfig<K, V> fetchConfig,
|
final FetchConfig fetchConfig,
|
||||||
|
final Deserializers<K, V> deserializers,
|
||||||
final FetchMetricsManager metricsManager,
|
final FetchMetricsManager metricsManager,
|
||||||
final Time time) {
|
final Time time) {
|
||||||
this.log = logContext.logger(FetchCollector.class);
|
this.log = logContext.logger(FetchCollector.class);
|
||||||
this.metadata = metadata;
|
this.metadata = metadata;
|
||||||
this.subscriptions = subscriptions;
|
this.subscriptions = subscriptions;
|
||||||
this.fetchConfig = fetchConfig;
|
this.fetchConfig = fetchConfig;
|
||||||
|
this.deserializers = deserializers;
|
||||||
this.metricsManager = metricsManager;
|
this.metricsManager = metricsManager;
|
||||||
this.time = time;
|
this.time = time;
|
||||||
}
|
}
|
||||||
|
@ -162,7 +165,9 @@ public class FetchCollector<K, V> {
|
||||||
throw new IllegalStateException("Missing position for fetchable partition " + tp);
|
throw new IllegalStateException("Missing position for fetchable partition " + tp);
|
||||||
|
|
||||||
if (nextInLineFetch.nextFetchOffset() == position.offset) {
|
if (nextInLineFetch.nextFetchOffset() == position.offset) {
|
||||||
List<ConsumerRecord<K, V>> partRecords = nextInLineFetch.fetchRecords(fetchConfig, fetchConfig.maxPollRecords);
|
List<ConsumerRecord<K, V>> partRecords = nextInLineFetch.fetchRecords(fetchConfig,
|
||||||
|
deserializers,
|
||||||
|
fetchConfig.maxPollRecords);
|
||||||
|
|
||||||
log.trace("Returning {} fetched records at offset {} for assigned partition {}",
|
log.trace("Returning {} fetched records at offset {} for assigned partition {}",
|
||||||
partRecords.size(), position, tp);
|
partRecords.size(), position, tp);
|
||||||
|
|
|
@ -19,57 +19,30 @@ package org.apache.kafka.clients.consumer.internals;
|
||||||
import org.apache.kafka.clients.consumer.Consumer;
|
import org.apache.kafka.clients.consumer.Consumer;
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||||
import org.apache.kafka.common.IsolationLevel;
|
import org.apache.kafka.common.IsolationLevel;
|
||||||
import org.apache.kafka.common.serialization.Deserializer;
|
|
||||||
|
|
||||||
import java.util.Objects;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredIsolationLevel;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@link FetchConfig} represents the static configuration for fetching records from Kafka. It is simply a way
|
* {@link FetchConfig} represents the static configuration for fetching records from Kafka. It is simply a way
|
||||||
* to bundle the immutable settings that were presented at the time the {@link Consumer} was created for later use by
|
* to bundle the immutable settings that were presented at the time the {@link Consumer} was created for later use by
|
||||||
* classes like {@link Fetcher}, {@link CompletedFetch}, etc.
|
* classes like {@link Fetcher}, {@link CompletedFetch}, etc.
|
||||||
*
|
|
||||||
* <p/>
|
|
||||||
*
|
|
||||||
* In most cases, the values stored and returned by {@link FetchConfig} will be those stored in the following
|
|
||||||
* {@link ConsumerConfig consumer configuration} settings:
|
|
||||||
*
|
|
||||||
* <ul>
|
|
||||||
* <li>{@link #minBytes}: {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG}</li>
|
|
||||||
* <li>{@link #maxBytes}: {@link ConsumerConfig#FETCH_MAX_BYTES_CONFIG}</li>
|
|
||||||
* <li>{@link #maxWaitMs}: {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}</li>
|
|
||||||
* <li>{@link #fetchSize}: {@link ConsumerConfig#MAX_PARTITION_FETCH_BYTES_CONFIG}</li>
|
|
||||||
* <li>{@link #maxPollRecords}: {@link ConsumerConfig#MAX_POLL_RECORDS_CONFIG}</li>
|
|
||||||
* <li>{@link #checkCrcs}: {@link ConsumerConfig#CHECK_CRCS_CONFIG}</li>
|
|
||||||
* <li>{@link #clientRackId}: {@link ConsumerConfig#CLIENT_RACK_CONFIG}</li>
|
|
||||||
* <li>{@link #deserializers}:
|
|
||||||
* {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG}/{@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
|
|
||||||
* </li>
|
|
||||||
* <li>{@link #isolationLevel}: {@link ConsumerConfig#ISOLATION_LEVEL_CONFIG}</li>
|
|
||||||
* </ul>
|
|
||||||
*
|
|
||||||
* However, there are places in the code where additional logic is used to determine these fetch-related configuration
|
|
||||||
* values. In those cases, the values are calculated outside of this class and simply passed in when constructed.
|
|
||||||
*
|
|
||||||
* <p/>
|
|
||||||
*
|
|
||||||
* Note: the {@link Deserializer deserializers} used for the key and value are not closed by this class. They should be
|
|
||||||
* closed by the creator of the {@link FetchConfig}.
|
|
||||||
*
|
|
||||||
* @param <K> Type used to {@link Deserializer deserialize} the message/record key
|
|
||||||
* @param <V> Type used to {@link Deserializer deserialize} the message/record value
|
|
||||||
*/
|
*/
|
||||||
public class FetchConfig<K, V> {
|
public class FetchConfig {
|
||||||
|
|
||||||
final int minBytes;
|
public final int minBytes;
|
||||||
final int maxBytes;
|
public final int maxBytes;
|
||||||
final int maxWaitMs;
|
public final int maxWaitMs;
|
||||||
final int fetchSize;
|
public final int fetchSize;
|
||||||
final int maxPollRecords;
|
public final int maxPollRecords;
|
||||||
final boolean checkCrcs;
|
public final boolean checkCrcs;
|
||||||
final String clientRackId;
|
public final String clientRackId;
|
||||||
final Deserializers<K, V> deserializers;
|
public final IsolationLevel isolationLevel;
|
||||||
final IsolationLevel isolationLevel;
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs a new {@link FetchConfig} using explicitly provided values. This is provided here for tests that
|
||||||
|
* want to exercise different scenarios can construct specific configuration values rather than going through
|
||||||
|
* the hassle of constructing a {@link ConsumerConfig}.
|
||||||
|
*/
|
||||||
public FetchConfig(int minBytes,
|
public FetchConfig(int minBytes,
|
||||||
int maxBytes,
|
int maxBytes,
|
||||||
int maxWaitMs,
|
int maxWaitMs,
|
||||||
|
@ -77,7 +50,6 @@ public class FetchConfig<K, V> {
|
||||||
int maxPollRecords,
|
int maxPollRecords,
|
||||||
boolean checkCrcs,
|
boolean checkCrcs,
|
||||||
String clientRackId,
|
String clientRackId,
|
||||||
Deserializers<K, V> deserializers,
|
|
||||||
IsolationLevel isolationLevel) {
|
IsolationLevel isolationLevel) {
|
||||||
this.minBytes = minBytes;
|
this.minBytes = minBytes;
|
||||||
this.maxBytes = maxBytes;
|
this.maxBytes = maxBytes;
|
||||||
|
@ -86,13 +58,27 @@ public class FetchConfig<K, V> {
|
||||||
this.maxPollRecords = maxPollRecords;
|
this.maxPollRecords = maxPollRecords;
|
||||||
this.checkCrcs = checkCrcs;
|
this.checkCrcs = checkCrcs;
|
||||||
this.clientRackId = clientRackId;
|
this.clientRackId = clientRackId;
|
||||||
this.deserializers = Objects.requireNonNull(deserializers, "Message deserializers provided to FetchConfig should not be null");
|
|
||||||
this.isolationLevel = isolationLevel;
|
this.isolationLevel = isolationLevel;
|
||||||
}
|
}
|
||||||
|
|
||||||
public FetchConfig(ConsumerConfig config,
|
/**
|
||||||
Deserializers<K, V> deserializers,
|
* Constructs a new {@link FetchConfig} using values from the given {@link ConsumerConfig consumer configuration}
|
||||||
IsolationLevel isolationLevel) {
|
* settings:
|
||||||
|
*
|
||||||
|
* <ul>
|
||||||
|
* <li>{@link #minBytes}: {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG}</li>
|
||||||
|
* <li>{@link #maxBytes}: {@link ConsumerConfig#FETCH_MAX_BYTES_CONFIG}</li>
|
||||||
|
* <li>{@link #maxWaitMs}: {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}</li>
|
||||||
|
* <li>{@link #fetchSize}: {@link ConsumerConfig#MAX_PARTITION_FETCH_BYTES_CONFIG}</li>
|
||||||
|
* <li>{@link #maxPollRecords}: {@link ConsumerConfig#MAX_POLL_RECORDS_CONFIG}</li>
|
||||||
|
* <li>{@link #checkCrcs}: {@link ConsumerConfig#CHECK_CRCS_CONFIG}</li>
|
||||||
|
* <li>{@link #clientRackId}: {@link ConsumerConfig#CLIENT_RACK_CONFIG}</li>
|
||||||
|
* <li>{@link #isolationLevel}: {@link ConsumerConfig#ISOLATION_LEVEL_CONFIG}</li>
|
||||||
|
* </ul>
|
||||||
|
*
|
||||||
|
* @param config Consumer configuration
|
||||||
|
*/
|
||||||
|
public FetchConfig(ConsumerConfig config) {
|
||||||
this.minBytes = config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG);
|
this.minBytes = config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG);
|
||||||
this.maxBytes = config.getInt(ConsumerConfig.FETCH_MAX_BYTES_CONFIG);
|
this.maxBytes = config.getInt(ConsumerConfig.FETCH_MAX_BYTES_CONFIG);
|
||||||
this.maxWaitMs = config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG);
|
this.maxWaitMs = config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG);
|
||||||
|
@ -100,8 +86,7 @@ public class FetchConfig<K, V> {
|
||||||
this.maxPollRecords = config.getInt(ConsumerConfig.MAX_POLL_RECORDS_CONFIG);
|
this.maxPollRecords = config.getInt(ConsumerConfig.MAX_POLL_RECORDS_CONFIG);
|
||||||
this.checkCrcs = config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG);
|
this.checkCrcs = config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG);
|
||||||
this.clientRackId = config.getString(ConsumerConfig.CLIENT_RACK_CONFIG);
|
this.clientRackId = config.getString(ConsumerConfig.CLIENT_RACK_CONFIG);
|
||||||
this.deserializers = Objects.requireNonNull(deserializers, "Message deserializers provided to FetchConfig should not be null");
|
this.isolationLevel = configuredIsolationLevel(config);
|
||||||
this.isolationLevel = isolationLevel;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -114,7 +99,6 @@ public class FetchConfig<K, V> {
|
||||||
", maxPollRecords=" + maxPollRecords +
|
", maxPollRecords=" + maxPollRecords +
|
||||||
", checkCrcs=" + checkCrcs +
|
", checkCrcs=" + checkCrcs +
|
||||||
", clientRackId='" + clientRackId + '\'' +
|
", clientRackId='" + clientRackId + '\'' +
|
||||||
", deserializers=" + deserializers +
|
|
||||||
", isolationLevel=" + isolationLevel +
|
", isolationLevel=" + isolationLevel +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,117 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.function.BiConsumer;
|
||||||
|
import java.util.function.Predicate;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
|
import org.apache.kafka.clients.FetchSessionHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest;
|
||||||
|
import org.apache.kafka.common.Node;
|
||||||
|
import org.apache.kafka.common.requests.FetchRequest;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@code FetchRequestManager} is responsible for generating {@link FetchRequest} that represent the
|
||||||
|
* {@link SubscriptionState#fetchablePartitions(Predicate)} based on the user's topic subscription/partition
|
||||||
|
* assignment.
|
||||||
|
*/
|
||||||
|
public class FetchRequestManager extends AbstractFetch implements RequestManager {
|
||||||
|
|
||||||
|
private final NetworkClientDelegate networkClientDelegate;
|
||||||
|
|
||||||
|
FetchRequestManager(final LogContext logContext,
|
||||||
|
final Time time,
|
||||||
|
final ConsumerMetadata metadata,
|
||||||
|
final SubscriptionState subscriptions,
|
||||||
|
final FetchConfig fetchConfig,
|
||||||
|
final FetchBuffer fetchBuffer,
|
||||||
|
final FetchMetricsManager metricsManager,
|
||||||
|
final NetworkClientDelegate networkClientDelegate) {
|
||||||
|
super(logContext, metadata, subscriptions, fetchConfig, fetchBuffer, metricsManager, time);
|
||||||
|
this.networkClientDelegate = networkClientDelegate;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean isUnavailable(Node node) {
|
||||||
|
return networkClientDelegate.isUnavailable(node);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void maybeThrowAuthFailure(Node node) {
|
||||||
|
networkClientDelegate.maybeThrowAuthFailure(node);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public PollResult poll(long currentTimeMs) {
|
||||||
|
return pollInternal(
|
||||||
|
prepareFetchRequests(),
|
||||||
|
this::handleFetchSuccess,
|
||||||
|
this::handleFetchFailure
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public PollResult pollOnClose() {
|
||||||
|
return pollInternal(
|
||||||
|
prepareCloseFetchSessionRequests(),
|
||||||
|
this::handleCloseFetchSessionSuccess,
|
||||||
|
this::handleCloseFetchSessionFailure
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates the {@link PollResult poll result} that contains a list of zero or more
|
||||||
|
* {@link FetchRequest.Builder fetch requests}.
|
||||||
|
*
|
||||||
|
* @param fetchRequests {@link Map} of {@link Node nodes} to their {@link FetchSessionHandler.FetchRequestData}
|
||||||
|
* @param successHandler {@link ResponseHandler Handler for successful responses}
|
||||||
|
* @param errorHandler {@link ResponseHandler Handler for failure responses}
|
||||||
|
* @return {@link PollResult}
|
||||||
|
*/
|
||||||
|
private PollResult pollInternal(Map<Node, FetchSessionHandler.FetchRequestData> fetchRequests,
|
||||||
|
ResponseHandler<ClientResponse> successHandler,
|
||||||
|
ResponseHandler<Throwable> errorHandler) {
|
||||||
|
List<UnsentRequest> requests = fetchRequests.entrySet().stream().map(entry -> {
|
||||||
|
final Node fetchTarget = entry.getKey();
|
||||||
|
final FetchSessionHandler.FetchRequestData data = entry.getValue();
|
||||||
|
final FetchRequest.Builder request = createFetchRequest(fetchTarget, data);
|
||||||
|
final BiConsumer<ClientResponse, Throwable> responseHandler = (clientResponse, error) -> {
|
||||||
|
if (error != null)
|
||||||
|
errorHandler.handle(fetchTarget, data, error);
|
||||||
|
else
|
||||||
|
successHandler.handle(fetchTarget, data, clientResponse);
|
||||||
|
};
|
||||||
|
|
||||||
|
return new UnsentRequest(request, fetchTarget, responseHandler);
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
|
return new PollResult(requests);
|
||||||
|
}
|
||||||
|
}
|
|
@ -16,16 +16,23 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.ClientRequest;
|
||||||
import org.apache.kafka.clients.ClientResponse;
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
import org.apache.kafka.clients.FetchSessionHandler;
|
import org.apache.kafka.clients.FetchSessionHandler;
|
||||||
|
import org.apache.kafka.clients.NetworkClient;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.internals.IdempotentCloser;
|
||||||
import org.apache.kafka.common.requests.FetchRequest;
|
import org.apache.kafka.common.requests.FetchRequest;
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
import org.apache.kafka.common.utils.Timer;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -47,66 +54,147 @@ import java.util.Map;
|
||||||
* on a different thread.</li>
|
* on a different thread.</li>
|
||||||
* </ul>
|
* </ul>
|
||||||
*/
|
*/
|
||||||
public class Fetcher<K, V> extends AbstractFetch<K, V> {
|
public class Fetcher<K, V> extends AbstractFetch {
|
||||||
|
|
||||||
|
private final Logger log;
|
||||||
|
private final ConsumerNetworkClient client;
|
||||||
private final FetchCollector<K, V> fetchCollector;
|
private final FetchCollector<K, V> fetchCollector;
|
||||||
|
|
||||||
public Fetcher(LogContext logContext,
|
public Fetcher(LogContext logContext,
|
||||||
ConsumerNetworkClient client,
|
ConsumerNetworkClient client,
|
||||||
ConsumerMetadata metadata,
|
ConsumerMetadata metadata,
|
||||||
SubscriptionState subscriptions,
|
SubscriptionState subscriptions,
|
||||||
FetchConfig<K, V> fetchConfig,
|
FetchConfig fetchConfig,
|
||||||
|
Deserializers<K, V> deserializers,
|
||||||
FetchMetricsManager metricsManager,
|
FetchMetricsManager metricsManager,
|
||||||
Time time) {
|
Time time) {
|
||||||
super(logContext, client, metadata, subscriptions, fetchConfig, metricsManager, time);
|
super(logContext, metadata, subscriptions, fetchConfig, new FetchBuffer(logContext), metricsManager, time);
|
||||||
|
this.log = logContext.logger(Fetcher.class);
|
||||||
|
this.client = client;
|
||||||
this.fetchCollector = new FetchCollector<>(logContext,
|
this.fetchCollector = new FetchCollector<>(logContext,
|
||||||
metadata,
|
metadata,
|
||||||
subscriptions,
|
subscriptions,
|
||||||
fetchConfig,
|
fetchConfig,
|
||||||
|
deserializers,
|
||||||
metricsManager,
|
metricsManager,
|
||||||
time);
|
time);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean isUnavailable(Node node) {
|
||||||
|
return client.isUnavailable(node);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void maybeThrowAuthFailure(Node node) {
|
||||||
|
client.maybeThrowAuthFailure(node);
|
||||||
|
}
|
||||||
|
|
||||||
public void clearBufferedDataForUnassignedPartitions(Collection<TopicPartition> assignedPartitions) {
|
public void clearBufferedDataForUnassignedPartitions(Collection<TopicPartition> assignedPartitions) {
|
||||||
fetchBuffer.retainAll(new HashSet<>(assignedPartitions));
|
fetchBuffer.retainAll(new HashSet<>(assignedPartitions));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set-up a fetch request for any node that we have assigned partitions for which doesn't already have
|
* Set up a fetch request for any node that we have assigned partitions for which doesn't already have
|
||||||
* an in-flight fetch or pending fetch data.
|
* an in-flight fetch or pending fetch data.
|
||||||
* @return number of fetches sent
|
* @return number of fetches sent
|
||||||
*/
|
*/
|
||||||
public synchronized int sendFetches() {
|
public synchronized int sendFetches() {
|
||||||
Map<Node, FetchSessionHandler.FetchRequestData> fetchRequestMap = prepareFetchRequests();
|
final Map<Node, FetchSessionHandler.FetchRequestData> fetchRequests = prepareFetchRequests();
|
||||||
|
sendFetchesInternal(
|
||||||
for (Map.Entry<Node, FetchSessionHandler.FetchRequestData> entry : fetchRequestMap.entrySet()) {
|
fetchRequests,
|
||||||
final Node fetchTarget = entry.getKey();
|
(fetchTarget, data, clientResponse) -> {
|
||||||
final FetchSessionHandler.FetchRequestData data = entry.getValue();
|
|
||||||
final FetchRequest.Builder request = createFetchRequest(fetchTarget, data);
|
|
||||||
RequestFutureListener<ClientResponse> listener = new RequestFutureListener<ClientResponse>() {
|
|
||||||
@Override
|
|
||||||
public void onSuccess(ClientResponse resp) {
|
|
||||||
synchronized (Fetcher.this) {
|
synchronized (Fetcher.this) {
|
||||||
handleFetchResponse(fetchTarget, data, resp);
|
handleFetchSuccess(fetchTarget, data, clientResponse);
|
||||||
}
|
}
|
||||||
}
|
},
|
||||||
|
(fetchTarget, data, error) -> {
|
||||||
@Override
|
|
||||||
public void onFailure(RuntimeException e) {
|
|
||||||
synchronized (Fetcher.this) {
|
synchronized (Fetcher.this) {
|
||||||
handleFetchResponse(fetchTarget, e);
|
handleFetchFailure(fetchTarget, data, error);
|
||||||
}
|
}
|
||||||
}
|
});
|
||||||
};
|
return fetchRequests.size();
|
||||||
|
}
|
||||||
|
|
||||||
final RequestFuture<ClientResponse> future = client.send(fetchTarget, request);
|
protected void maybeCloseFetchSessions(final Timer timer) {
|
||||||
future.addListener(listener);
|
final List<RequestFuture<ClientResponse>> requestFutures = sendFetchesInternal(
|
||||||
|
prepareCloseFetchSessionRequests(),
|
||||||
|
this::handleCloseFetchSessionSuccess,
|
||||||
|
this::handleCloseFetchSessionFailure
|
||||||
|
);
|
||||||
|
|
||||||
|
// Poll to ensure that request has been written to the socket. Wait until either the timer has expired or until
|
||||||
|
// all requests have received a response.
|
||||||
|
while (timer.notExpired() && !requestFutures.stream().allMatch(RequestFuture::isDone)) {
|
||||||
|
client.poll(timer, null, true);
|
||||||
|
timer.update();
|
||||||
}
|
}
|
||||||
|
|
||||||
return fetchRequestMap.size();
|
if (!requestFutures.stream().allMatch(RequestFuture::isDone)) {
|
||||||
|
// we ran out of time before completing all futures. It is ok since we don't want to block the shutdown
|
||||||
|
// here.
|
||||||
|
log.debug("All requests couldn't be sent in the specific timeout period {}ms. " +
|
||||||
|
"This may result in unnecessary fetch sessions at the broker. Consider increasing the timeout passed for " +
|
||||||
|
"KafkaConsumer.close(Duration timeout)", timer.timeoutMs());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public Fetch<K, V> collectFetch() {
|
public Fetch<K, V> collectFetch() {
|
||||||
return fetchCollector.collectFetch(fetchBuffer);
|
return fetchCollector.collectFetch(fetchBuffer);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method is called by {@link #close(Timer)} which is guarded by the {@link IdempotentCloser}) such as to only
|
||||||
|
* be executed once the first time that any of the {@link #close()} methods are called. Subclasses can override
|
||||||
|
* this method without the need for extra synchronization at the instance level.
|
||||||
|
*
|
||||||
|
* @param timer Timer to enforce time limit
|
||||||
|
*/
|
||||||
|
// Visible for testing
|
||||||
|
protected void closeInternal(Timer timer) {
|
||||||
|
// we do not need to re-enable wake-ups since we are closing already
|
||||||
|
client.disableWakeups();
|
||||||
|
maybeCloseFetchSessions(timer);
|
||||||
|
super.closeInternal(timer);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates the {@link FetchRequest.Builder fetch request},
|
||||||
|
* {@link NetworkClient#send(ClientRequest, long) enqueues/sends it, and adds the {@link RequestFuture callback}
|
||||||
|
* for the response.
|
||||||
|
*
|
||||||
|
* @param fetchRequests {@link Map} of {@link Node nodes} to their
|
||||||
|
* {@link FetchSessionHandler.FetchRequestData request data}
|
||||||
|
* @param successHandler {@link ResponseHandler Handler for successful responses}
|
||||||
|
* @param errorHandler {@link ResponseHandler Handler for failure responses}
|
||||||
|
* @return List of {@link RequestFuture callbacks}
|
||||||
|
*/
|
||||||
|
private List<RequestFuture<ClientResponse>> sendFetchesInternal(Map<Node, FetchSessionHandler.FetchRequestData> fetchRequests,
|
||||||
|
ResponseHandler<ClientResponse> successHandler,
|
||||||
|
ResponseHandler<Throwable> errorHandler) {
|
||||||
|
final List<RequestFuture<ClientResponse>> requestFutures = new ArrayList<>();
|
||||||
|
|
||||||
|
for (Map.Entry<Node, FetchSessionHandler.FetchRequestData> entry : fetchRequests.entrySet()) {
|
||||||
|
final Node fetchTarget = entry.getKey();
|
||||||
|
final FetchSessionHandler.FetchRequestData data = entry.getValue();
|
||||||
|
final FetchRequest.Builder request = createFetchRequest(fetchTarget, data);
|
||||||
|
final RequestFuture<ClientResponse> responseFuture = client.send(fetchTarget, request);
|
||||||
|
|
||||||
|
responseFuture.addListener(new RequestFutureListener<ClientResponse>() {
|
||||||
|
@Override
|
||||||
|
public void onSuccess(ClientResponse resp) {
|
||||||
|
successHandler.handle(fetchTarget, data, resp);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(RuntimeException e) {
|
||||||
|
errorHandler.handle(fetchTarget, data, e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
requestFutures.add(responseFuture);
|
||||||
|
}
|
||||||
|
|
||||||
|
return requestFutures;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -18,6 +18,8 @@ package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.clients.CommonClientConfigs;
|
import org.apache.kafka.clients.CommonClientConfigs;
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent;
|
||||||
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
||||||
import org.apache.kafka.common.errors.RetriableException;
|
import org.apache.kafka.common.errors.RetriableException;
|
||||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||||
|
@ -58,6 +60,7 @@ import java.util.Collections;
|
||||||
* <p>See {@link HeartbeatRequestState} for more details.</p>
|
* <p>See {@link HeartbeatRequestState} for more details.</p>
|
||||||
*/
|
*/
|
||||||
public class HeartbeatRequestManager implements RequestManager {
|
public class HeartbeatRequestManager implements RequestManager {
|
||||||
|
|
||||||
private final Logger logger;
|
private final Logger logger;
|
||||||
private final Time time;
|
private final Time time;
|
||||||
|
|
||||||
|
@ -90,22 +93,22 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
/**
|
/**
|
||||||
* ErrorEventHandler allows the background thread to propagate errors back to the user
|
* ErrorEventHandler allows the background thread to propagate errors back to the user
|
||||||
*/
|
*/
|
||||||
private final ErrorEventHandler nonRetriableErrorHandler;
|
private final BackgroundEventHandler backgroundEventHandler;
|
||||||
|
|
||||||
public HeartbeatRequestManager(
|
public HeartbeatRequestManager(
|
||||||
final Time time,
|
|
||||||
final LogContext logContext,
|
final LogContext logContext,
|
||||||
|
final Time time,
|
||||||
final ConsumerConfig config,
|
final ConsumerConfig config,
|
||||||
final CoordinatorRequestManager coordinatorRequestManager,
|
final CoordinatorRequestManager coordinatorRequestManager,
|
||||||
final SubscriptionState subscriptions,
|
final SubscriptionState subscriptions,
|
||||||
final MembershipManager membershipManager,
|
final MembershipManager membershipManager,
|
||||||
final ErrorEventHandler nonRetriableErrorHandler) {
|
final BackgroundEventHandler backgroundEventHandler) {
|
||||||
this.coordinatorRequestManager = coordinatorRequestManager;
|
this.coordinatorRequestManager = coordinatorRequestManager;
|
||||||
this.time = time;
|
this.time = time;
|
||||||
this.logger = logContext.logger(getClass());
|
this.logger = logContext.logger(getClass());
|
||||||
this.subscriptions = subscriptions;
|
this.subscriptions = subscriptions;
|
||||||
this.membershipManager = membershipManager;
|
this.membershipManager = membershipManager;
|
||||||
this.nonRetriableErrorHandler = nonRetriableErrorHandler;
|
this.backgroundEventHandler = backgroundEventHandler;
|
||||||
this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
|
this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
|
||||||
long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
|
long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
|
||||||
long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
|
long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
|
||||||
|
@ -122,7 +125,7 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
final SubscriptionState subscriptions,
|
final SubscriptionState subscriptions,
|
||||||
final MembershipManager membershipManager,
|
final MembershipManager membershipManager,
|
||||||
final HeartbeatRequestState heartbeatRequestState,
|
final HeartbeatRequestState heartbeatRequestState,
|
||||||
final ErrorEventHandler nonRetriableErrorHandler) {
|
final BackgroundEventHandler backgroundEventHandler) {
|
||||||
this.logger = logContext.logger(this.getClass());
|
this.logger = logContext.logger(this.getClass());
|
||||||
this.time = time;
|
this.time = time;
|
||||||
this.subscriptions = subscriptions;
|
this.subscriptions = subscriptions;
|
||||||
|
@ -130,7 +133,7 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
this.coordinatorRequestManager = coordinatorRequestManager;
|
this.coordinatorRequestManager = coordinatorRequestManager;
|
||||||
this.heartbeatRequestState = heartbeatRequestState;
|
this.heartbeatRequestState = heartbeatRequestState;
|
||||||
this.membershipManager = membershipManager;
|
this.membershipManager = membershipManager;
|
||||||
this.nonRetriableErrorHandler = nonRetriableErrorHandler;
|
this.backgroundEventHandler = backgroundEventHandler;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -146,18 +149,14 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
|
public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
|
||||||
if (!coordinatorRequestManager.coordinator().isPresent() || !membershipManager.shouldSendHeartbeat()) {
|
if (!coordinatorRequestManager.coordinator().isPresent() || !membershipManager.shouldSendHeartbeat())
|
||||||
return new NetworkClientDelegate.PollResult(
|
return NetworkClientDelegate.PollResult.EMPTY;
|
||||||
Long.MAX_VALUE, Collections.emptyList());
|
|
||||||
}
|
|
||||||
|
|
||||||
// TODO: We will need to send a heartbeat response after partitions being revoke. This needs to be
|
// TODO: We will need to send a heartbeat response after partitions being revoke. This needs to be
|
||||||
// implemented either with or after the partition reconciliation logic.
|
// implemented either with or after the partition reconciliation logic.
|
||||||
if (!heartbeatRequestState.canSendRequest(currentTimeMs)) {
|
if (!heartbeatRequestState.canSendRequest(currentTimeMs))
|
||||||
return new NetworkClientDelegate.PollResult(
|
return new NetworkClientDelegate.PollResult(heartbeatRequestState.nextHeartbeatMs(currentTimeMs));
|
||||||
heartbeatRequestState.nextHeartbeatMs(currentTimeMs),
|
|
||||||
Collections.emptyList());
|
|
||||||
}
|
|
||||||
this.heartbeatRequestState.onSendAttempt(currentTimeMs);
|
this.heartbeatRequestState.onSendAttempt(currentTimeMs);
|
||||||
NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest();
|
NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest();
|
||||||
return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(request));
|
return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(request));
|
||||||
|
@ -198,7 +197,7 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
this.heartbeatRequestState.onFailedAttempt(responseTimeMs);
|
this.heartbeatRequestState.onFailedAttempt(responseTimeMs);
|
||||||
if (exception instanceof RetriableException) {
|
if (exception instanceof RetriableException) {
|
||||||
String message = String.format("GroupHeartbeatRequest failed because of the retriable exception. " +
|
String message = String.format("GroupHeartbeatRequest failed because of the retriable exception. " +
|
||||||
"Will retry in %s ms: {}",
|
"Will retry in %s ms: %s",
|
||||||
heartbeatRequestState.remainingBackoffMs(responseTimeMs),
|
heartbeatRequestState.remainingBackoffMs(responseTimeMs),
|
||||||
exception.getMessage());
|
exception.getMessage());
|
||||||
logger.debug(message);
|
logger.debug(message);
|
||||||
|
@ -223,12 +222,13 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
final long currentTimeMs) {
|
final long currentTimeMs) {
|
||||||
Errors error = Errors.forCode(response.data().errorCode());
|
Errors error = Errors.forCode(response.data().errorCode());
|
||||||
String errorMessage = response.data().errorMessage();
|
String errorMessage = response.data().errorMessage();
|
||||||
|
String message;
|
||||||
// TODO: upon encountering a fatal/fenced error, trigger onPartitionLost logic to give up the current
|
// TODO: upon encountering a fatal/fenced error, trigger onPartitionLost logic to give up the current
|
||||||
// assignments.
|
// assignments.
|
||||||
switch (error) {
|
switch (error) {
|
||||||
case NOT_COORDINATOR:
|
case NOT_COORDINATOR:
|
||||||
// the manager should retry immediately when the coordinator node becomes available again
|
// the manager should retry immediately when the coordinator node becomes available again
|
||||||
String message = String.format("GroupHeartbeatRequest failed because the group coordinator %s is incorrect. " +
|
message = String.format("GroupHeartbeatRequest failed because the group coordinator %s is incorrect. " +
|
||||||
"Will attempt to find the coordinator again and retry",
|
"Will attempt to find the coordinator again and retry",
|
||||||
coordinatorRequestManager.coordinator());
|
coordinatorRequestManager.coordinator());
|
||||||
logInfo(message, response, currentTimeMs);
|
logInfo(message, response, currentTimeMs);
|
||||||
|
@ -274,7 +274,7 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case FENCED_MEMBER_EPOCH:
|
case FENCED_MEMBER_EPOCH:
|
||||||
message = String.format("GroupHeartbeatRequest failed because member epoch %s is invalid. " +
|
message = String.format("GroupHeartbeatRequest failed because member ID %s with epoch %s is invalid. " +
|
||||||
"Will abandon all partitions and rejoin the group",
|
"Will abandon all partitions and rejoin the group",
|
||||||
membershipManager.memberId(), membershipManager.memberEpoch());
|
membershipManager.memberId(), membershipManager.memberEpoch());
|
||||||
logInfo(message, response, currentTimeMs);
|
logInfo(message, response, currentTimeMs);
|
||||||
|
@ -282,7 +282,7 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case UNKNOWN_MEMBER_ID:
|
case UNKNOWN_MEMBER_ID:
|
||||||
message = String.format("GroupHeartbeatRequest failed because member id %s is invalid. " +
|
message = String.format("GroupHeartbeatRequest failed because member of unknown ID %s with epoch %s is invalid. " +
|
||||||
"Will abandon all partitions and rejoin the group",
|
"Will abandon all partitions and rejoin the group",
|
||||||
membershipManager.memberId(), membershipManager.memberEpoch());
|
membershipManager.memberId(), membershipManager.memberEpoch());
|
||||||
logInfo(message, response, currentTimeMs);
|
logInfo(message, response, currentTimeMs);
|
||||||
|
@ -307,7 +307,7 @@ public class HeartbeatRequestManager implements RequestManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void handleFatalFailure(Throwable error) {
|
private void handleFatalFailure(Throwable error) {
|
||||||
nonRetriableErrorHandler.handle(error);
|
backgroundEventHandler.add(new ErrorBackgroundEvent(error));
|
||||||
membershipManager.transitionToFailed();
|
membershipManager.transitionToFailed();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -16,8 +16,10 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.ApiVersions;
|
||||||
import org.apache.kafka.clients.ClientRequest;
|
import org.apache.kafka.clients.ClientRequest;
|
||||||
import org.apache.kafka.clients.ClientResponse;
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
|
import org.apache.kafka.clients.ClientUtils;
|
||||||
import org.apache.kafka.clients.KafkaClient;
|
import org.apache.kafka.clients.KafkaClient;
|
||||||
import org.apache.kafka.clients.NetworkClientUtils;
|
import org.apache.kafka.clients.NetworkClientUtils;
|
||||||
import org.apache.kafka.clients.RequestCompletionHandler;
|
import org.apache.kafka.clients.RequestCompletionHandler;
|
||||||
|
@ -26,6 +28,7 @@ import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.errors.AuthenticationException;
|
import org.apache.kafka.common.errors.AuthenticationException;
|
||||||
import org.apache.kafka.common.errors.DisconnectException;
|
import org.apache.kafka.common.errors.DisconnectException;
|
||||||
import org.apache.kafka.common.errors.TimeoutException;
|
import org.apache.kafka.common.errors.TimeoutException;
|
||||||
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
import org.apache.kafka.common.requests.AbstractRequest;
|
import org.apache.kafka.common.requests.AbstractRequest;
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
@ -35,27 +38,29 @@ import org.slf4j.Logger;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayDeque;
|
import java.util.ArrayDeque;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Queue;
|
import java.util.Queue;
|
||||||
import java.util.Set;
|
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION;
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A wrapper around the {@link org.apache.kafka.clients.NetworkClient} to handle network poll and send operations.
|
* A wrapper around the {@link org.apache.kafka.clients.NetworkClient} to handle network poll and send operations.
|
||||||
*/
|
*/
|
||||||
public class NetworkClientDelegate implements AutoCloseable {
|
public class NetworkClientDelegate implements AutoCloseable {
|
||||||
|
|
||||||
private final KafkaClient client;
|
private final KafkaClient client;
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final Logger log;
|
private final Logger log;
|
||||||
private final int requestTimeoutMs;
|
private final int requestTimeoutMs;
|
||||||
private final Queue<UnsentRequest> unsentRequests;
|
private final Queue<UnsentRequest> unsentRequests;
|
||||||
private final long retryBackoffMs;
|
private final long retryBackoffMs;
|
||||||
private final Set<Node> tryConnectNodes;
|
|
||||||
|
|
||||||
public NetworkClientDelegate(
|
public NetworkClientDelegate(
|
||||||
final Time time,
|
final Time time,
|
||||||
|
@ -68,9 +73,40 @@ public class NetworkClientDelegate implements AutoCloseable {
|
||||||
this.unsentRequests = new ArrayDeque<>();
|
this.unsentRequests = new ArrayDeque<>();
|
||||||
this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
|
this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
|
||||||
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
|
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
|
||||||
this.tryConnectNodes = new HashSet<>();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Visible for testing
|
||||||
|
Queue<UnsentRequest> unsentRequests() {
|
||||||
|
return unsentRequests;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if the node is disconnected and unavailable for immediate reconnection (i.e. if it is in
|
||||||
|
* reconnect backoff window following the disconnect).
|
||||||
|
*
|
||||||
|
* @param node {@link Node} to check for availability
|
||||||
|
* @see NetworkClientUtils#isUnavailable(KafkaClient, Node, Time)
|
||||||
|
*/
|
||||||
|
public boolean isUnavailable(Node node) {
|
||||||
|
return NetworkClientUtils.isUnavailable(client, node, time);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks for an authentication error on a given node and throws the exception if it exists.
|
||||||
|
*
|
||||||
|
* @param node {@link Node} to check for a previous {@link AuthenticationException}; if found it is thrown
|
||||||
|
* @see NetworkClientUtils#maybeThrowAuthFailure(KafkaClient, Node)
|
||||||
|
*/
|
||||||
|
public void maybeThrowAuthFailure(Node node) {
|
||||||
|
NetworkClientUtils.maybeThrowAuthFailure(client, node);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initiate a connection if currently possible. This is only really useful for resetting
|
||||||
|
* the failed status of a socket.
|
||||||
|
*
|
||||||
|
* @param node The node to connect to
|
||||||
|
*/
|
||||||
public void tryConnect(Node node) {
|
public void tryConnect(Node node) {
|
||||||
NetworkClientUtils.tryConnect(client, node, time);
|
NetworkClientUtils.tryConnect(client, node, time);
|
||||||
}
|
}
|
||||||
|
@ -81,7 +117,6 @@ public class NetworkClientDelegate implements AutoCloseable {
|
||||||
*
|
*
|
||||||
* @param timeoutMs timeout time
|
* @param timeoutMs timeout time
|
||||||
* @param currentTimeMs current time
|
* @param currentTimeMs current time
|
||||||
* @return a list of client response
|
|
||||||
*/
|
*/
|
||||||
public void poll(final long timeoutMs, final long currentTimeMs) {
|
public void poll(final long timeoutMs, final long currentTimeMs) {
|
||||||
trySend(currentTimeMs);
|
trySend(currentTimeMs);
|
||||||
|
@ -119,8 +154,7 @@ public class NetworkClientDelegate implements AutoCloseable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean doSend(final UnsentRequest r,
|
boolean doSend(final UnsentRequest r, final long currentTimeMs) {
|
||||||
final long currentTimeMs) {
|
|
||||||
Node node = r.node.orElse(client.leastLoadedNode(currentTimeMs));
|
Node node = r.node.orElse(client.leastLoadedNode(currentTimeMs));
|
||||||
if (node == null || nodeUnavailable(node)) {
|
if (node == null || nodeUnavailable(node)) {
|
||||||
log.debug("No broker available to send the request: {}. Retrying.", r);
|
log.debug("No broker available to send the request: {}. Retrying.", r);
|
||||||
|
@ -137,7 +171,7 @@ public class NetworkClientDelegate implements AutoCloseable {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkDisconnects(final long currentTimeMs) {
|
protected void checkDisconnects(final long currentTimeMs) {
|
||||||
// Check the connection of the unsent request. Disconnect the disconnected node if it is unable to be connected.
|
// Check the connection of the unsent request. Disconnect the disconnected node if it is unable to be connected.
|
||||||
Iterator<UnsentRequest> iter = unsentRequests.iterator();
|
Iterator<UnsentRequest> iter = unsentRequests.iterator();
|
||||||
while (iter.hasNext()) {
|
while (iter.hasNext()) {
|
||||||
|
@ -190,26 +224,47 @@ public class NetworkClientDelegate implements AutoCloseable {
|
||||||
this.client.close();
|
this.client.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public long addAll(PollResult pollResult) {
|
||||||
|
addAll(pollResult.unsentRequests);
|
||||||
|
return pollResult.timeUntilNextPollMs;
|
||||||
|
}
|
||||||
|
|
||||||
public void addAll(final List<UnsentRequest> requests) {
|
public void addAll(final List<UnsentRequest> requests) {
|
||||||
requests.forEach(u -> {
|
if (!requests.isEmpty()) {
|
||||||
u.setTimer(this.time, this.requestTimeoutMs);
|
requests.forEach(ur -> ur.setTimer(time, requestTimeoutMs));
|
||||||
});
|
unsentRequests.addAll(requests);
|
||||||
this.unsentRequests.addAll(requests);
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class PollResult {
|
public static class PollResult {
|
||||||
|
|
||||||
|
public static final long WAIT_FOREVER = Long.MAX_VALUE;
|
||||||
|
public static final PollResult EMPTY = new PollResult(WAIT_FOREVER);
|
||||||
public final long timeUntilNextPollMs;
|
public final long timeUntilNextPollMs;
|
||||||
public final List<UnsentRequest> unsentRequests;
|
public final List<UnsentRequest> unsentRequests;
|
||||||
|
|
||||||
public PollResult(final long timeMsTillNextPoll, final List<UnsentRequest> unsentRequests) {
|
public PollResult(final long timeUntilNextPollMs, final List<UnsentRequest> unsentRequests) {
|
||||||
this.timeUntilNextPollMs = timeMsTillNextPoll;
|
this.timeUntilNextPollMs = timeUntilNextPollMs;
|
||||||
this.unsentRequests = Collections.unmodifiableList(unsentRequests);
|
this.unsentRequests = Collections.unmodifiableList(unsentRequests);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public PollResult(final List<UnsentRequest> unsentRequests) {
|
||||||
|
this(WAIT_FOREVER, unsentRequests);
|
||||||
|
}
|
||||||
|
|
||||||
|
public PollResult(final UnsentRequest unsentRequest) {
|
||||||
|
this(Collections.singletonList(unsentRequest));
|
||||||
|
}
|
||||||
|
|
||||||
|
public PollResult(final long timeUntilNextPollMs) {
|
||||||
|
this(timeUntilNextPollMs, Collections.emptyList());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class UnsentRequest {
|
public static class UnsentRequest {
|
||||||
private final AbstractRequest.Builder<?> requestBuilder;
|
private final AbstractRequest.Builder<?> requestBuilder;
|
||||||
private final FutureCompletionHandler handler;
|
private final FutureCompletionHandler handler;
|
||||||
private Optional<Node> node; // empty if random node can be chosen
|
private final Optional<Node> node; // empty if random node can be chosen
|
||||||
private Timer timer;
|
private Timer timer;
|
||||||
|
|
||||||
public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder,
|
public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder,
|
||||||
|
@ -227,6 +282,12 @@ public class NetworkClientDelegate implements AutoCloseable {
|
||||||
this.handler.future().whenComplete(callback);
|
this.handler.future().whenComplete(callback);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder,
|
||||||
|
final Node node,
|
||||||
|
final BiConsumer<ClientResponse, Throwable> callback) {
|
||||||
|
this(requestBuilder, Optional.of(node), callback);
|
||||||
|
}
|
||||||
|
|
||||||
public void setTimer(final Time time, final long requestTimeoutMs) {
|
public void setTimer(final Time time, final long requestTimeoutMs) {
|
||||||
this.timer = time.timer(requestTimeoutMs);
|
this.timer = time.timer(requestTimeoutMs);
|
||||||
}
|
}
|
||||||
|
@ -243,6 +304,10 @@ public class NetworkClientDelegate implements AutoCloseable {
|
||||||
return requestBuilder;
|
return requestBuilder;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Optional<Node> node() {
|
||||||
|
return node;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "UnsentRequest{" +
|
return "UnsentRequest{" +
|
||||||
|
@ -291,4 +356,32 @@ public class NetworkClientDelegate implements AutoCloseable {
|
||||||
return future;
|
return future;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a {@link Supplier} for deferred creation during invocation by
|
||||||
|
* {@link ConsumerNetworkThread}.
|
||||||
|
*/
|
||||||
|
public static Supplier<NetworkClientDelegate> supplier(final Time time,
|
||||||
|
final LogContext logContext,
|
||||||
|
final ConsumerMetadata metadata,
|
||||||
|
final ConsumerConfig config,
|
||||||
|
final ApiVersions apiVersions,
|
||||||
|
final Metrics metrics,
|
||||||
|
final FetchMetricsManager fetchMetricsManager) {
|
||||||
|
return new CachedSupplier<NetworkClientDelegate>() {
|
||||||
|
@Override
|
||||||
|
protected NetworkClientDelegate create() {
|
||||||
|
KafkaClient client = ClientUtils.createNetworkClient(config,
|
||||||
|
metrics,
|
||||||
|
CONSUMER_METRIC_GROUP_PREFIX,
|
||||||
|
logContext,
|
||||||
|
apiVersions,
|
||||||
|
time,
|
||||||
|
CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION,
|
||||||
|
metadata,
|
||||||
|
fetchMetricsManager.throttleTimeSensor());
|
||||||
|
return new NetworkClientDelegate(time, config, logContext, client);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.kafka.clients.consumer.internals;
|
||||||
import org.apache.kafka.clients.ApiVersions;
|
import org.apache.kafka.clients.ApiVersions;
|
||||||
import org.apache.kafka.clients.NodeApiVersions;
|
import org.apache.kafka.clients.NodeApiVersions;
|
||||||
import org.apache.kafka.clients.consumer.LogTruncationException;
|
import org.apache.kafka.clients.consumer.LogTruncationException;
|
||||||
|
import org.apache.kafka.clients.consumer.NoOffsetForPartitionException;
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
||||||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||||
|
@ -262,7 +263,7 @@ class OffsetFetcherUtils {
|
||||||
else if (strategy == OffsetResetStrategy.LATEST)
|
else if (strategy == OffsetResetStrategy.LATEST)
|
||||||
return ListOffsetsRequest.LATEST_TIMESTAMP;
|
return ListOffsetsRequest.LATEST_TIMESTAMP;
|
||||||
else
|
else
|
||||||
return null;
|
throw new NoOffsetForPartitionException(partition);
|
||||||
}
|
}
|
||||||
|
|
||||||
static Set<String> topicsForPartitions(Collection<TopicPartition> partitions) {
|
static Set<String> topicsForPartitions(Collection<TopicPartition> partitions) {
|
||||||
|
|
|
@ -24,6 +24,8 @@ import org.apache.kafka.clients.consumer.LogTruncationException;
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
||||||
import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetData;
|
import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetData;
|
||||||
import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetResult;
|
import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetResult;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent;
|
||||||
import org.apache.kafka.common.ClusterResource;
|
import org.apache.kafka.common.ClusterResource;
|
||||||
import org.apache.kafka.common.ClusterResourceListener;
|
import org.apache.kafka.common.ClusterResourceListener;
|
||||||
import org.apache.kafka.common.IsolationLevel;
|
import org.apache.kafka.common.IsolationLevel;
|
||||||
|
@ -83,6 +85,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final ApiVersions apiVersions;
|
private final ApiVersions apiVersions;
|
||||||
private final NetworkClientDelegate networkClientDelegate;
|
private final NetworkClientDelegate networkClientDelegate;
|
||||||
|
private final BackgroundEventHandler backgroundEventHandler;
|
||||||
|
|
||||||
@SuppressWarnings("this-escape")
|
@SuppressWarnings("this-escape")
|
||||||
public OffsetsRequestManager(final SubscriptionState subscriptionState,
|
public OffsetsRequestManager(final SubscriptionState subscriptionState,
|
||||||
|
@ -93,6 +96,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis
|
||||||
final long requestTimeoutMs,
|
final long requestTimeoutMs,
|
||||||
final ApiVersions apiVersions,
|
final ApiVersions apiVersions,
|
||||||
final NetworkClientDelegate networkClientDelegate,
|
final NetworkClientDelegate networkClientDelegate,
|
||||||
|
final BackgroundEventHandler backgroundEventHandler,
|
||||||
final LogContext logContext) {
|
final LogContext logContext) {
|
||||||
requireNonNull(subscriptionState);
|
requireNonNull(subscriptionState);
|
||||||
requireNonNull(metadata);
|
requireNonNull(metadata);
|
||||||
|
@ -100,6 +104,7 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis
|
||||||
requireNonNull(time);
|
requireNonNull(time);
|
||||||
requireNonNull(apiVersions);
|
requireNonNull(apiVersions);
|
||||||
requireNonNull(networkClientDelegate);
|
requireNonNull(networkClientDelegate);
|
||||||
|
requireNonNull(backgroundEventHandler);
|
||||||
requireNonNull(logContext);
|
requireNonNull(logContext);
|
||||||
|
|
||||||
this.metadata = metadata;
|
this.metadata = metadata;
|
||||||
|
@ -112,11 +117,12 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis
|
||||||
this.requestTimeoutMs = requestTimeoutMs;
|
this.requestTimeoutMs = requestTimeoutMs;
|
||||||
this.apiVersions = apiVersions;
|
this.apiVersions = apiVersions;
|
||||||
this.networkClientDelegate = networkClientDelegate;
|
this.networkClientDelegate = networkClientDelegate;
|
||||||
|
this.backgroundEventHandler = backgroundEventHandler;
|
||||||
this.offsetFetcherUtils = new OffsetFetcherUtils(logContext, metadata, subscriptionState,
|
this.offsetFetcherUtils = new OffsetFetcherUtils(logContext, metadata, subscriptionState,
|
||||||
time, retryBackoffMs, apiVersions);
|
time, retryBackoffMs, apiVersions);
|
||||||
// Register the cluster metadata update callback. Note this only relies on the
|
// Register the cluster metadata update callback. Note this only relies on the
|
||||||
// requestsToRetry initialized above, and won't be invoked until all managers are
|
// requestsToRetry initialized above, and won't be invoked until all managers are
|
||||||
// initialized and the background thread started.
|
// initialized and the network thread started.
|
||||||
this.metadata.addClusterUpdateListener(this);
|
this.metadata.addClusterUpdateListener(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -127,10 +133,10 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
|
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
|
||||||
NetworkClientDelegate.PollResult pollResult =
|
// Copy the outgoing request list and clear it.
|
||||||
new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>(requestsToSend));
|
List<NetworkClientDelegate.UnsentRequest> unsentRequests = new ArrayList<>(requestsToSend);
|
||||||
this.requestsToSend.clear();
|
requestsToSend.clear();
|
||||||
return pollResult;
|
return new NetworkClientDelegate.PollResult(unsentRequests);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -188,7 +194,14 @@ public class OffsetsRequestManager implements RequestManager, ClusterResourceLis
|
||||||
* this function (ex. {@link org.apache.kafka.common.errors.TopicAuthorizationException})
|
* this function (ex. {@link org.apache.kafka.common.errors.TopicAuthorizationException})
|
||||||
*/
|
*/
|
||||||
public void resetPositionsIfNeeded() {
|
public void resetPositionsIfNeeded() {
|
||||||
Map<TopicPartition, Long> offsetResetTimestamps = offsetFetcherUtils.getOffsetResetTimestamp();
|
Map<TopicPartition, Long> offsetResetTimestamps;
|
||||||
|
|
||||||
|
try {
|
||||||
|
offsetResetTimestamps = offsetFetcherUtils.getOffsetResetTimestamp();
|
||||||
|
} catch (Exception e) {
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(e));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
if (offsetResetTimestamps.isEmpty())
|
if (offsetResetTimestamps.isEmpty())
|
||||||
return;
|
return;
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -16,13 +16,50 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.Consumer;
|
||||||
import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult;
|
import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@code PollResult} consist of {@code UnsentRequest} if there are requests to send; otherwise, return the time till
|
* {@code PollResult} consist of {@code UnsentRequest} if there are requests to send; otherwise, return the time till
|
||||||
* the next poll event.
|
* the next poll event.
|
||||||
*/
|
*/
|
||||||
public interface RequestManager {
|
public interface RequestManager {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* During normal operation of the {@link Consumer}, a request manager may need to send out network requests.
|
||||||
|
* Implementations can return {@link PollResult their need for network I/O} by returning the requests here.
|
||||||
|
* This method is called within a single-threaded context from
|
||||||
|
* {@link ConsumerNetworkThread the consumer's network I/O thread}. As such, there should be no need for
|
||||||
|
* synchronization protection in this method's implementation.
|
||||||
|
*
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* <em>Note</em>: no network I/O occurs in this method. The method itself should not block for any reason. This
|
||||||
|
* method is called from the consumer's network I/O thread, so quick execution of this method in <em>all</em>
|
||||||
|
* request managers is critical to ensure that we can heartbeat in a timely fashion.
|
||||||
|
*
|
||||||
|
* @param currentTimeMs The current system time at which the method was called; useful for determining if
|
||||||
|
* time-sensitive operations should be performed
|
||||||
|
*/
|
||||||
PollResult poll(long currentTimeMs);
|
PollResult poll(long currentTimeMs);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* On shutdown of the {@link Consumer}, a request manager may need to send out network requests. Implementations
|
||||||
|
* can signal that by returning the {@link PollResult close} requests here. Like {@link #poll(long)}, this method
|
||||||
|
* is called within a single-threaded context from {@link ConsumerNetworkThread the consumer's network I/O thread}.
|
||||||
|
* As such, there should be no need for synchronization protection in this method's implementation.
|
||||||
|
*
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* <em>Note</em>: no network I/O occurs in this method. The method itself should not block for any reason. This
|
||||||
|
* method is called as an (indirect) result of {@link Consumer#close() the consumer's close method} being invoked.
|
||||||
|
* (Note that it is still invoked on the consumer's network I/O thread). Quick execution of this method in
|
||||||
|
* <em>all</em> request managers is critical to ensure that we can complete as many of the consumer's shutdown
|
||||||
|
* tasks as possible within the user-provided timeout.
|
||||||
|
*/
|
||||||
|
default PollResult pollOnClose() {
|
||||||
|
return EMPTY;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,10 +16,25 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.ApiVersions;
|
||||||
|
import org.apache.kafka.clients.GroupRebalanceConfig;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
|
||||||
|
import org.apache.kafka.common.internals.IdempotentCloser;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
|
import static org.apache.kafka.common.utils.Utils.closeQuietly;
|
||||||
|
|
||||||
import static java.util.Objects.requireNonNull;
|
import static java.util.Objects.requireNonNull;
|
||||||
|
|
||||||
|
@ -28,25 +43,31 @@ import static java.util.Objects.requireNonNull;
|
||||||
* This allows callers to both use the specific {@link RequestManager} instance, or to iterate over the list via
|
* This allows callers to both use the specific {@link RequestManager} instance, or to iterate over the list via
|
||||||
* the {@link #entries()} method.
|
* the {@link #entries()} method.
|
||||||
*/
|
*/
|
||||||
public class RequestManagers {
|
public class RequestManagers implements Closeable {
|
||||||
|
|
||||||
|
private final Logger log;
|
||||||
public final Optional<CoordinatorRequestManager> coordinatorRequestManager;
|
public final Optional<CoordinatorRequestManager> coordinatorRequestManager;
|
||||||
public final Optional<CommitRequestManager> commitRequestManager;
|
public final Optional<CommitRequestManager> commitRequestManager;
|
||||||
private final Optional<HeartbeatRequestManager> heartbeatRequestManager;
|
public final Optional<HeartbeatRequestManager> heartbeatRequestManager;
|
||||||
public final OffsetsRequestManager offsetsRequestManager;
|
public final OffsetsRequestManager offsetsRequestManager;
|
||||||
public final TopicMetadataRequestManager topicMetadataRequestManager;
|
public final TopicMetadataRequestManager topicMetadataRequestManager;
|
||||||
|
public final FetchRequestManager fetchRequestManager;
|
||||||
private final List<Optional<? extends RequestManager>> entries;
|
private final List<Optional<? extends RequestManager>> entries;
|
||||||
|
private final IdempotentCloser closer = new IdempotentCloser();
|
||||||
|
|
||||||
public RequestManagers(OffsetsRequestManager offsetsRequestManager,
|
public RequestManagers(LogContext logContext,
|
||||||
|
OffsetsRequestManager offsetsRequestManager,
|
||||||
TopicMetadataRequestManager topicMetadataRequestManager,
|
TopicMetadataRequestManager topicMetadataRequestManager,
|
||||||
|
FetchRequestManager fetchRequestManager,
|
||||||
Optional<CoordinatorRequestManager> coordinatorRequestManager,
|
Optional<CoordinatorRequestManager> coordinatorRequestManager,
|
||||||
Optional<CommitRequestManager> commitRequestManager,
|
Optional<CommitRequestManager> commitRequestManager,
|
||||||
Optional<HeartbeatRequestManager> heartbeatRequestManager) {
|
Optional<HeartbeatRequestManager> heartbeatRequestManager) {
|
||||||
this.offsetsRequestManager = requireNonNull(offsetsRequestManager,
|
this.log = logContext.logger(RequestManagers.class);
|
||||||
"OffsetsRequestManager cannot be null");
|
this.offsetsRequestManager = requireNonNull(offsetsRequestManager, "OffsetsRequestManager cannot be null");
|
||||||
this.coordinatorRequestManager = coordinatorRequestManager;
|
this.coordinatorRequestManager = coordinatorRequestManager;
|
||||||
this.commitRequestManager = commitRequestManager;
|
this.commitRequestManager = commitRequestManager;
|
||||||
this.topicMetadataRequestManager = topicMetadataRequestManager;
|
this.topicMetadataRequestManager = topicMetadataRequestManager;
|
||||||
|
this.fetchRequestManager = fetchRequestManager;
|
||||||
this.heartbeatRequestManager = heartbeatRequestManager;
|
this.heartbeatRequestManager = heartbeatRequestManager;
|
||||||
|
|
||||||
List<Optional<? extends RequestManager>> list = new ArrayList<>();
|
List<Optional<? extends RequestManager>> list = new ArrayList<>();
|
||||||
|
@ -55,10 +76,111 @@ public class RequestManagers {
|
||||||
list.add(heartbeatRequestManager);
|
list.add(heartbeatRequestManager);
|
||||||
list.add(Optional.of(offsetsRequestManager));
|
list.add(Optional.of(offsetsRequestManager));
|
||||||
list.add(Optional.of(topicMetadataRequestManager));
|
list.add(Optional.of(topicMetadataRequestManager));
|
||||||
|
list.add(Optional.of(fetchRequestManager));
|
||||||
entries = Collections.unmodifiableList(list);
|
entries = Collections.unmodifiableList(list);
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<Optional<? extends RequestManager>> entries() {
|
public List<Optional<? extends RequestManager>> entries() {
|
||||||
return entries;
|
return entries;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
closer.close(
|
||||||
|
() -> {
|
||||||
|
log.debug("Closing RequestManagers");
|
||||||
|
|
||||||
|
entries.stream()
|
||||||
|
.filter(Optional::isPresent)
|
||||||
|
.map(Optional::get)
|
||||||
|
.filter(rm -> rm instanceof Closeable)
|
||||||
|
.map(rm -> (Closeable) rm)
|
||||||
|
.forEach(c -> closeQuietly(c, c.getClass().getSimpleName()));
|
||||||
|
log.debug("RequestManagers has been closed");
|
||||||
|
},
|
||||||
|
() -> log.debug("RequestManagers was already closed")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a {@link Supplier} for deferred creation during invocation by
|
||||||
|
* {@link ConsumerNetworkThread}.
|
||||||
|
*/
|
||||||
|
public static Supplier<RequestManagers> supplier(final Time time,
|
||||||
|
final LogContext logContext,
|
||||||
|
final BlockingQueue<BackgroundEvent> backgroundEventQueue,
|
||||||
|
final ConsumerMetadata metadata,
|
||||||
|
final SubscriptionState subscriptions,
|
||||||
|
final FetchBuffer fetchBuffer,
|
||||||
|
final ConsumerConfig config,
|
||||||
|
final GroupRebalanceConfig groupRebalanceConfig,
|
||||||
|
final ApiVersions apiVersions,
|
||||||
|
final FetchMetricsManager fetchMetricsManager,
|
||||||
|
final Supplier<NetworkClientDelegate> networkClientDelegateSupplier) {
|
||||||
|
return new CachedSupplier<RequestManagers>() {
|
||||||
|
@Override
|
||||||
|
protected RequestManagers create() {
|
||||||
|
final NetworkClientDelegate networkClientDelegate = networkClientDelegateSupplier.get();
|
||||||
|
final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(logContext, backgroundEventQueue);
|
||||||
|
final FetchConfig fetchConfig = new FetchConfig(config);
|
||||||
|
long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
|
||||||
|
long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
|
||||||
|
final int requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
|
||||||
|
final OffsetsRequestManager listOffsets = new OffsetsRequestManager(subscriptions,
|
||||||
|
metadata,
|
||||||
|
fetchConfig.isolationLevel,
|
||||||
|
time,
|
||||||
|
retryBackoffMs,
|
||||||
|
requestTimeoutMs,
|
||||||
|
apiVersions,
|
||||||
|
networkClientDelegate,
|
||||||
|
backgroundEventHandler,
|
||||||
|
logContext);
|
||||||
|
final FetchRequestManager fetch = new FetchRequestManager(logContext,
|
||||||
|
time,
|
||||||
|
metadata,
|
||||||
|
subscriptions,
|
||||||
|
fetchConfig,
|
||||||
|
fetchBuffer,
|
||||||
|
fetchMetricsManager,
|
||||||
|
networkClientDelegate);
|
||||||
|
final TopicMetadataRequestManager topic = new TopicMetadataRequestManager(
|
||||||
|
logContext,
|
||||||
|
config);
|
||||||
|
HeartbeatRequestManager heartbeatRequestManager = null;
|
||||||
|
CoordinatorRequestManager coordinator = null;
|
||||||
|
CommitRequestManager commit = null;
|
||||||
|
|
||||||
|
if (groupRebalanceConfig != null && groupRebalanceConfig.groupId != null) {
|
||||||
|
final GroupState groupState = new GroupState(groupRebalanceConfig);
|
||||||
|
coordinator = new CoordinatorRequestManager(time,
|
||||||
|
logContext,
|
||||||
|
retryBackoffMs,
|
||||||
|
retryBackoffMaxMs,
|
||||||
|
backgroundEventHandler,
|
||||||
|
groupState.groupId);
|
||||||
|
commit = new CommitRequestManager(time, logContext, subscriptions, config, coordinator, groupState);
|
||||||
|
MembershipManager membershipManager = new MembershipManagerImpl(groupState.groupId, logContext);
|
||||||
|
heartbeatRequestManager = new HeartbeatRequestManager(
|
||||||
|
logContext,
|
||||||
|
time,
|
||||||
|
config,
|
||||||
|
coordinator,
|
||||||
|
subscriptions,
|
||||||
|
membershipManager,
|
||||||
|
backgroundEventHandler);
|
||||||
|
}
|
||||||
|
|
||||||
|
return new RequestManagers(
|
||||||
|
logContext,
|
||||||
|
listOffsets,
|
||||||
|
topic,
|
||||||
|
fetch,
|
||||||
|
Optional.ofNullable(coordinator),
|
||||||
|
Optional.ofNullable(commit),
|
||||||
|
Optional.ofNullable(heartbeatRequestManager)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -40,6 +40,8 @@ import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p>
|
||||||
* Manages the state of topic metadata requests. This manager returns a
|
* Manages the state of topic metadata requests. This manager returns a
|
||||||
|
@ -84,9 +86,7 @@ public class TopicMetadataRequestManager implements RequestManager {
|
||||||
.filter(Optional::isPresent)
|
.filter(Optional::isPresent)
|
||||||
.map(Optional::get)
|
.map(Optional::get)
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
return requests.isEmpty() ?
|
return requests.isEmpty() ? EMPTY : new NetworkClientDelegate.PollResult(0, requests);
|
||||||
new NetworkClientDelegate.PollResult(Long.MAX_VALUE, new ArrayList<>()) :
|
|
||||||
new NetworkClientDelegate.PollResult(0, Collections.unmodifiableList(requests));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -24,7 +24,7 @@ import java.util.Objects;
|
||||||
public abstract class ApplicationEvent {
|
public abstract class ApplicationEvent {
|
||||||
|
|
||||||
public enum Type {
|
public enum Type {
|
||||||
NOOP, COMMIT, POLL, FETCH_COMMITTED_OFFSET, METADATA_UPDATE, ASSIGNMENT_CHANGE,
|
COMMIT, POLL, FETCH_COMMITTED_OFFSET, METADATA_UPDATE, ASSIGNMENT_CHANGE,
|
||||||
LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA
|
LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,115 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.RequestManagers;
|
||||||
|
import org.apache.kafka.common.internals.IdempotentCloser;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
import org.apache.kafka.common.utils.Timer;
|
||||||
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An event handler that receives {@link ApplicationEvent application events} from the application thread which
|
||||||
|
* are then readable from the {@link ApplicationEventProcessor} in the {@link ConsumerNetworkThread network thread}.
|
||||||
|
*/
|
||||||
|
public class ApplicationEventHandler implements Closeable {
|
||||||
|
|
||||||
|
private final Logger log;
|
||||||
|
private final BlockingQueue<ApplicationEvent> applicationEventQueue;
|
||||||
|
private final ConsumerNetworkThread networkThread;
|
||||||
|
private final IdempotentCloser closer = new IdempotentCloser();
|
||||||
|
|
||||||
|
public ApplicationEventHandler(final LogContext logContext,
|
||||||
|
final Time time,
|
||||||
|
final BlockingQueue<ApplicationEvent> applicationEventQueue,
|
||||||
|
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier,
|
||||||
|
final Supplier<NetworkClientDelegate> networkClientDelegateSupplier,
|
||||||
|
final Supplier<RequestManagers> requestManagersSupplier) {
|
||||||
|
this.log = logContext.logger(ApplicationEventHandler.class);
|
||||||
|
this.applicationEventQueue = applicationEventQueue;
|
||||||
|
this.networkThread = new ConsumerNetworkThread(logContext,
|
||||||
|
time,
|
||||||
|
applicationEventProcessorSupplier,
|
||||||
|
networkClientDelegateSupplier,
|
||||||
|
requestManagersSupplier);
|
||||||
|
this.networkThread.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add an {@link ApplicationEvent} to the handler and then internally invoke {@link #wakeupNetworkThread}
|
||||||
|
* to alert the network I/O thread that it has something to process.
|
||||||
|
*
|
||||||
|
* @param event An {@link ApplicationEvent} created by the application thread
|
||||||
|
*/
|
||||||
|
public void add(final ApplicationEvent event) {
|
||||||
|
Objects.requireNonNull(event, "ApplicationEvent provided to add must be non-null");
|
||||||
|
log.trace("Enqueued event: {}", event);
|
||||||
|
applicationEventQueue.add(event);
|
||||||
|
wakeupNetworkThread();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wakeup the {@link ConsumerNetworkThread network I/O thread} to pull the next event(s) from the queue.
|
||||||
|
*/
|
||||||
|
public void wakeupNetworkThread() {
|
||||||
|
networkThread.wakeup();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add a {@link CompletableApplicationEvent} to the handler. The method blocks waiting for the result, and will
|
||||||
|
* return the result value upon successful completion; otherwise throws an error.
|
||||||
|
*
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details.
|
||||||
|
*
|
||||||
|
* @param event A {@link CompletableApplicationEvent} created by the polling thread
|
||||||
|
* @param timer Timer for which to wait for the event to complete
|
||||||
|
* @return Value that is the result of the event
|
||||||
|
* @param <T> Type of return value of the event
|
||||||
|
*/
|
||||||
|
public <T> T addAndGet(final CompletableApplicationEvent<T> event, final Timer timer) {
|
||||||
|
Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null");
|
||||||
|
Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null");
|
||||||
|
add(event);
|
||||||
|
return event.get(timer);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
close(Duration.ZERO);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close(final Duration timeout) {
|
||||||
|
closer.close(
|
||||||
|
() -> Utils.closeQuietly(() -> networkThread.close(timeout), "consumer network thread"),
|
||||||
|
() -> log.warn("The application event handler was already closed")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -17,146 +17,188 @@
|
||||||
package org.apache.kafka.clients.consumer.internals.events;
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.CachedSupplier;
|
||||||
import org.apache.kafka.clients.consumer.internals.CommitRequestManager;
|
import org.apache.kafka.clients.consumer.internals.CommitRequestManager;
|
||||||
import org.apache.kafka.clients.consumer.internals.ConsumerMetadata;
|
import org.apache.kafka.clients.consumer.internals.ConsumerMetadata;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread;
|
||||||
import org.apache.kafka.clients.consumer.internals.RequestManagers;
|
import org.apache.kafka.clients.consumer.internals.RequestManagers;
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.KafkaException;
|
||||||
import org.apache.kafka.common.PartitionInfo;
|
import org.apache.kafka.common.PartitionInfo;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
public class ApplicationEventProcessor {
|
/**
|
||||||
|
* An {@link EventProcessor} that is created and executes in the {@link ConsumerNetworkThread network thread}
|
||||||
private final BlockingQueue<BackgroundEvent> backgroundEventQueue;
|
* which processes {@link ApplicationEvent application events} generated by the application thread.
|
||||||
|
*/
|
||||||
|
public class ApplicationEventProcessor extends EventProcessor<ApplicationEvent> {
|
||||||
|
|
||||||
|
private final Logger log;
|
||||||
private final ConsumerMetadata metadata;
|
private final ConsumerMetadata metadata;
|
||||||
|
|
||||||
private final RequestManagers requestManagers;
|
private final RequestManagers requestManagers;
|
||||||
|
|
||||||
public ApplicationEventProcessor(final BlockingQueue<BackgroundEvent> backgroundEventQueue,
|
public ApplicationEventProcessor(final LogContext logContext,
|
||||||
|
final BlockingQueue<ApplicationEvent> applicationEventQueue,
|
||||||
final RequestManagers requestManagers,
|
final RequestManagers requestManagers,
|
||||||
final ConsumerMetadata metadata) {
|
final ConsumerMetadata metadata) {
|
||||||
this.backgroundEventQueue = backgroundEventQueue;
|
super(logContext, applicationEventQueue);
|
||||||
|
this.log = logContext.logger(ApplicationEventProcessor.class);
|
||||||
this.requestManagers = requestManagers;
|
this.requestManagers = requestManagers;
|
||||||
this.metadata = metadata;
|
this.metadata = metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean process(final ApplicationEvent event) {
|
|
||||||
Objects.requireNonNull(event);
|
|
||||||
switch (event.type()) {
|
|
||||||
case NOOP:
|
|
||||||
return process((NoopApplicationEvent) event);
|
|
||||||
case COMMIT:
|
|
||||||
return process((CommitApplicationEvent) event);
|
|
||||||
case POLL:
|
|
||||||
return process((PollApplicationEvent) event);
|
|
||||||
case FETCH_COMMITTED_OFFSET:
|
|
||||||
return process((OffsetFetchApplicationEvent) event);
|
|
||||||
case METADATA_UPDATE:
|
|
||||||
return process((NewTopicsMetadataUpdateRequestEvent) event);
|
|
||||||
case ASSIGNMENT_CHANGE:
|
|
||||||
return process((AssignmentChangeApplicationEvent) event);
|
|
||||||
case TOPIC_METADATA:
|
|
||||||
return process((TopicMetadataApplicationEvent) event);
|
|
||||||
case LIST_OFFSETS:
|
|
||||||
return process((ListOffsetsApplicationEvent) event);
|
|
||||||
case RESET_POSITIONS:
|
|
||||||
return processResetPositionsEvent();
|
|
||||||
case VALIDATE_POSITIONS:
|
|
||||||
return processValidatePositionsEvent();
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Processes {@link NoopApplicationEvent} and enqueue a
|
* Process the events—if any—that were produced by the application thread. It is possible that when processing
|
||||||
* {@link NoopBackgroundEvent}. This is intentionally left here for
|
* an event generates an error. In such cases, the processor will log an exception, but we do not want those
|
||||||
* demonstration purpose.
|
* errors to be propagated to the caller.
|
||||||
*
|
|
||||||
* @param event a {@link NoopApplicationEvent}
|
|
||||||
*/
|
*/
|
||||||
private boolean process(final NoopApplicationEvent event) {
|
@Override
|
||||||
return backgroundEventQueue.add(new NoopBackgroundEvent(event.message()));
|
public void process() {
|
||||||
|
process((event, error) -> { });
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean process(final PollApplicationEvent event) {
|
@Override
|
||||||
|
public void process(ApplicationEvent event) {
|
||||||
|
switch (event.type()) {
|
||||||
|
case COMMIT:
|
||||||
|
process((CommitApplicationEvent) event);
|
||||||
|
return;
|
||||||
|
|
||||||
|
case POLL:
|
||||||
|
process((PollApplicationEvent) event);
|
||||||
|
return;
|
||||||
|
|
||||||
|
case FETCH_COMMITTED_OFFSET:
|
||||||
|
process((OffsetFetchApplicationEvent) event);
|
||||||
|
return;
|
||||||
|
|
||||||
|
case METADATA_UPDATE:
|
||||||
|
process((NewTopicsMetadataUpdateRequestEvent) event);
|
||||||
|
return;
|
||||||
|
|
||||||
|
case ASSIGNMENT_CHANGE:
|
||||||
|
process((AssignmentChangeApplicationEvent) event);
|
||||||
|
return;
|
||||||
|
|
||||||
|
case TOPIC_METADATA:
|
||||||
|
process((TopicMetadataApplicationEvent) event);
|
||||||
|
return;
|
||||||
|
|
||||||
|
case LIST_OFFSETS:
|
||||||
|
process((ListOffsetsApplicationEvent) event);
|
||||||
|
return;
|
||||||
|
|
||||||
|
case RESET_POSITIONS:
|
||||||
|
processResetPositionsEvent();
|
||||||
|
return;
|
||||||
|
|
||||||
|
case VALIDATE_POSITIONS:
|
||||||
|
processValidatePositionsEvent();
|
||||||
|
return;
|
||||||
|
|
||||||
|
default:
|
||||||
|
log.warn("Application event type " + event.type() + " was not expected");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<ApplicationEvent> getEventClass() {
|
||||||
|
return ApplicationEvent.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void process(final PollApplicationEvent event) {
|
||||||
if (!requestManagers.commitRequestManager.isPresent()) {
|
if (!requestManagers.commitRequestManager.isPresent()) {
|
||||||
return true;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
||||||
manager.updateAutoCommitTimer(event.pollTimeMs());
|
manager.updateAutoCommitTimer(event.pollTimeMs());
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean process(final CommitApplicationEvent event) {
|
private void process(final CommitApplicationEvent event) {
|
||||||
if (!requestManagers.commitRequestManager.isPresent()) {
|
if (!requestManagers.commitRequestManager.isPresent()) {
|
||||||
// Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id
|
// Leaving this error handling here, but it is a bit strange as the commit API should enforce the group.id
|
||||||
// upfront so we should never get to this block.
|
// upfront, so we should never get to this block.
|
||||||
Exception exception = new KafkaException("Unable to commit offset. Most likely because the group.id wasn't set");
|
Exception exception = new KafkaException("Unable to commit offset. Most likely because the group.id wasn't set");
|
||||||
event.future().completeExceptionally(exception);
|
event.future().completeExceptionally(exception);
|
||||||
return false;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
||||||
event.chain(manager.addOffsetCommitRequest(event.offsets()));
|
event.chain(manager.addOffsetCommitRequest(event.offsets()));
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean process(final OffsetFetchApplicationEvent event) {
|
private void process(final OffsetFetchApplicationEvent event) {
|
||||||
if (!requestManagers.commitRequestManager.isPresent()) {
|
if (!requestManagers.commitRequestManager.isPresent()) {
|
||||||
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 CommittedRequestManager is not available. Check if group.id was set correctly"));
|
||||||
return false;
|
return;
|
||||||
}
|
}
|
||||||
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
||||||
event.chain(manager.addOffsetFetchRequest(event.partitions()));
|
event.chain(manager.addOffsetFetchRequest(event.partitions()));
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean process(final NewTopicsMetadataUpdateRequestEvent event) {
|
private void process(final NewTopicsMetadataUpdateRequestEvent ignored) {
|
||||||
metadata.requestUpdateForNewTopics();
|
metadata.requestUpdateForNewTopics();
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean process(final AssignmentChangeApplicationEvent event) {
|
private void process(final AssignmentChangeApplicationEvent event) {
|
||||||
if (!requestManagers.commitRequestManager.isPresent()) {
|
if (!requestManagers.commitRequestManager.isPresent()) {
|
||||||
return false;
|
return;
|
||||||
}
|
}
|
||||||
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
CommitRequestManager manager = requestManagers.commitRequestManager.get();
|
||||||
manager.updateAutoCommitTimer(event.currentTimeMs());
|
manager.updateAutoCommitTimer(event.currentTimeMs());
|
||||||
manager.maybeAutoCommit(event.offsets());
|
manager.maybeAutoCommit(event.offsets());
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean process(final ListOffsetsApplicationEvent event) {
|
private void process(final ListOffsetsApplicationEvent event) {
|
||||||
final CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> future =
|
final CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> future =
|
||||||
requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(),
|
requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(),
|
||||||
event.requireTimestamps());
|
event.requireTimestamps());
|
||||||
event.chain(future);
|
event.chain(future);
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean processResetPositionsEvent() {
|
private void processResetPositionsEvent() {
|
||||||
requestManagers.offsetsRequestManager.resetPositionsIfNeeded();
|
requestManagers.offsetsRequestManager.resetPositionsIfNeeded();
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean processValidatePositionsEvent() {
|
private void processValidatePositionsEvent() {
|
||||||
requestManagers.offsetsRequestManager.validatePositionsIfNeeded();
|
requestManagers.offsetsRequestManager.validatePositionsIfNeeded();
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean process(final TopicMetadataApplicationEvent event) {
|
private void process(final TopicMetadataApplicationEvent event) {
|
||||||
final CompletableFuture<Map<String, List<PartitionInfo>>> future =
|
final CompletableFuture<Map<String, List<PartitionInfo>>> future =
|
||||||
this.requestManagers.topicMetadataRequestManager.requestTopicMetadata(Optional.of(event.topic()));
|
this.requestManagers.topicMetadataRequestManager.requestTopicMetadata(Optional.of(event.topic()));
|
||||||
event.chain(future);
|
event.chain(future);
|
||||||
return true;
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a {@link Supplier} for deferred creation during invocation by
|
||||||
|
* {@link ConsumerNetworkThread}.
|
||||||
|
*/
|
||||||
|
public static Supplier<ApplicationEventProcessor> supplier(final LogContext logContext,
|
||||||
|
final ConsumerMetadata metadata,
|
||||||
|
final BlockingQueue<ApplicationEvent> applicationEventQueue,
|
||||||
|
final Supplier<RequestManagers> requestManagersSupplier) {
|
||||||
|
return new CachedSupplier<ApplicationEventProcessor>() {
|
||||||
|
@Override
|
||||||
|
protected ApplicationEventProcessor create() {
|
||||||
|
RequestManagers requestManagers = requestManagersSupplier.get();
|
||||||
|
return new ApplicationEventProcessor(
|
||||||
|
logContext,
|
||||||
|
applicationEventQueue,
|
||||||
|
requestManagers,
|
||||||
|
metadata
|
||||||
|
);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,15 +16,17 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals.events;
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread;
|
||||||
|
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is the abstract definition of the events created by the background thread.
|
* This is the abstract definition of the events created by the {@link ConsumerNetworkThread network thread}.
|
||||||
*/
|
*/
|
||||||
public abstract class BackgroundEvent {
|
public abstract class BackgroundEvent {
|
||||||
|
|
||||||
public enum Type {
|
public enum Type {
|
||||||
NOOP, ERROR,
|
ERROR,
|
||||||
}
|
}
|
||||||
|
|
||||||
protected final Type type;
|
protected final Type type;
|
||||||
|
|
|
@ -0,0 +1,52 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Queue;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An event handler that receives {@link BackgroundEvent background events} from the
|
||||||
|
* {@link ConsumerNetworkThread network thread} which are then made available to the application thread
|
||||||
|
* via the {@link BackgroundEventProcessor}.
|
||||||
|
*/
|
||||||
|
|
||||||
|
public class BackgroundEventHandler {
|
||||||
|
|
||||||
|
private final Logger log;
|
||||||
|
private final Queue<BackgroundEvent> backgroundEventQueue;
|
||||||
|
|
||||||
|
public BackgroundEventHandler(final LogContext logContext, final Queue<BackgroundEvent> backgroundEventQueue) {
|
||||||
|
this.log = logContext.logger(BackgroundEventHandler.class);
|
||||||
|
this.backgroundEventQueue = backgroundEventQueue;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add a {@link BackgroundEvent} to the handler.
|
||||||
|
*
|
||||||
|
* @param event A {@link BackgroundEvent} created by the {@link ConsumerNetworkThread network thread}
|
||||||
|
*/
|
||||||
|
public void add(BackgroundEvent event) {
|
||||||
|
Objects.requireNonNull(event, "BackgroundEvent provided to add must be non-null");
|
||||||
|
log.trace("Enqueued event: {}", event);
|
||||||
|
backgroundEventQueue.add(event);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,75 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread;
|
||||||
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An {@link EventProcessor} that is created and executes in the application thread for the purpose of processing
|
||||||
|
* {@link BackgroundEvent background events} generated by the {@link ConsumerNetworkThread network thread}.
|
||||||
|
* Those events are generally of two types:
|
||||||
|
*
|
||||||
|
* <ul>
|
||||||
|
* <li>Errors that occur in the network thread that need to be propagated to the application thread</li>
|
||||||
|
* <li>{@link ConsumerRebalanceListener} callbacks that are to be executed on the application thread</li>
|
||||||
|
* </ul>
|
||||||
|
*/
|
||||||
|
public class BackgroundEventProcessor extends EventProcessor<BackgroundEvent> {
|
||||||
|
|
||||||
|
public BackgroundEventProcessor(final LogContext logContext,
|
||||||
|
final BlockingQueue<BackgroundEvent> backgroundEventQueue) {
|
||||||
|
super(logContext, backgroundEventQueue);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}.
|
||||||
|
* It is possible that {@link ErrorBackgroundEvent an error} could occur when processing the events.
|
||||||
|
* In such cases, the processor will take a reference to the first error, continue to process the
|
||||||
|
* remaining events, and then throw the first error that occurred.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void process() {
|
||||||
|
AtomicReference<KafkaException> firstError = new AtomicReference<>();
|
||||||
|
process((event, error) -> firstError.compareAndSet(null, error));
|
||||||
|
|
||||||
|
if (firstError.get() != null)
|
||||||
|
throw firstError.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void process(final BackgroundEvent event) {
|
||||||
|
if (event.type() == BackgroundEvent.Type.ERROR)
|
||||||
|
process((ErrorBackgroundEvent) event);
|
||||||
|
else
|
||||||
|
throw new IllegalArgumentException("Background event type " + event.type() + " was not expected");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<BackgroundEvent> getEventClass() {
|
||||||
|
return BackgroundEvent.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void process(final ErrorBackgroundEvent event) {
|
||||||
|
throw event.error();
|
||||||
|
}
|
||||||
|
}
|
|
@ -27,7 +27,7 @@ import java.util.concurrent.CompletableFuture;
|
||||||
*
|
*
|
||||||
* @param <T>
|
* @param <T>
|
||||||
*/
|
*/
|
||||||
public abstract class CompletableApplicationEvent<T> extends ApplicationEvent {
|
public abstract class CompletableApplicationEvent<T> extends ApplicationEvent implements CompletableEvent<T> {
|
||||||
|
|
||||||
private final CompletableFuture<T> future;
|
private final CompletableFuture<T> future;
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,25 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
|
||||||
|
public interface CompletableEvent<T> {
|
||||||
|
|
||||||
|
CompletableFuture<T> future();
|
||||||
|
|
||||||
|
}
|
|
@ -16,16 +16,18 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals.events;
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
|
||||||
public class ErrorBackgroundEvent extends BackgroundEvent {
|
public class ErrorBackgroundEvent extends BackgroundEvent {
|
||||||
|
|
||||||
private final Throwable error;
|
private final RuntimeException error;
|
||||||
|
|
||||||
public ErrorBackgroundEvent(Throwable error) {
|
public ErrorBackgroundEvent(Throwable t) {
|
||||||
super(Type.ERROR);
|
super(Type.ERROR);
|
||||||
this.error = error;
|
this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Throwable error() {
|
public RuntimeException error() {
|
||||||
return error;
|
return error;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,65 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.kafka.clients.consumer.internals.events;
|
|
||||||
|
|
||||||
import org.apache.kafka.common.utils.Timer;
|
|
||||||
|
|
||||||
import java.io.Closeable;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.concurrent.Future;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This class interfaces with the KafkaConsumer and the background thread. It allows the caller to enqueue events via
|
|
||||||
* the {@code add()} method and to retrieve events via the {@code poll()} method.
|
|
||||||
*/
|
|
||||||
public interface EventHandler extends Closeable {
|
|
||||||
/**
|
|
||||||
* Retrieves and removes a {@link BackgroundEvent}. Returns an empty Optional instance if there is nothing.
|
|
||||||
* @return an Optional of {@link BackgroundEvent} if the value is present. Otherwise, an empty Optional.
|
|
||||||
*/
|
|
||||||
Optional<BackgroundEvent> poll();
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Check whether there are pending {@code BackgroundEvent} await to be consumed.
|
|
||||||
* @return true if there are no pending event
|
|
||||||
*/
|
|
||||||
boolean isEmpty();
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Add an {@link ApplicationEvent} to the handler. The method returns true upon successful add; otherwise returns
|
|
||||||
* false.
|
|
||||||
* @param event An {@link ApplicationEvent} created by the polling thread.
|
|
||||||
* @return true upon successful add.
|
|
||||||
*/
|
|
||||||
boolean add(ApplicationEvent event);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Add a {@link CompletableApplicationEvent} to the handler. The method blocks waiting for the result, and will
|
|
||||||
* return the result value upon successful completion; otherwise throws an error.
|
|
||||||
*
|
|
||||||
* <p/>
|
|
||||||
*
|
|
||||||
* See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details.
|
|
||||||
*
|
|
||||||
* @param event A {@link CompletableApplicationEvent} created by the polling thread.
|
|
||||||
* @param timer Timer for which to wait for the event to complete
|
|
||||||
* @return Value that is the result of the event
|
|
||||||
* @param <T> Type of return value of the event
|
|
||||||
*/
|
|
||||||
<T> T addAndGet(final CompletableApplicationEvent<T> event, final Timer timer);
|
|
||||||
}
|
|
|
@ -0,0 +1,137 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
import org.apache.kafka.common.internals.IdempotentCloser;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An {@link EventProcessor} is the means by which events <em>produced</em> by thread <em>A</em> are
|
||||||
|
* <em>processed</em> by thread <em>B</em>. By definition, threads <em>A</em> and <em>B</em> run in parallel to
|
||||||
|
* each other, so a mechanism is needed with which to receive and process the events from the other thread. That
|
||||||
|
* communication channel is formed around {@link BlockingQueue a shared queue} into which thread <em>A</em>
|
||||||
|
* enqueues events and thread <em>B</em> reads and processes those events.
|
||||||
|
*/
|
||||||
|
public abstract class EventProcessor<T> implements Closeable {
|
||||||
|
|
||||||
|
private final Logger log;
|
||||||
|
private final BlockingQueue<T> eventQueue;
|
||||||
|
private final IdempotentCloser closer;
|
||||||
|
|
||||||
|
protected EventProcessor(final LogContext logContext, final BlockingQueue<T> eventQueue) {
|
||||||
|
this.log = logContext.logger(EventProcessor.class);
|
||||||
|
this.eventQueue = eventQueue;
|
||||||
|
this.closer = new IdempotentCloser();
|
||||||
|
}
|
||||||
|
|
||||||
|
public abstract void process();
|
||||||
|
|
||||||
|
public abstract void process(T event);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
closer.close(this::closeInternal, () -> log.warn("The event processor was already closed"));
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract Class<T> getEventClass();
|
||||||
|
|
||||||
|
protected interface ProcessErrorHandler<T> {
|
||||||
|
|
||||||
|
void onError(T event, KafkaException error);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Drains all available events from the queue, and then processes them in order. If any errors are thrown while
|
||||||
|
* processing the individual events, these are submitted to the given {@link ProcessErrorHandler}.
|
||||||
|
*/
|
||||||
|
protected void process(ProcessErrorHandler<T> processErrorHandler) {
|
||||||
|
String eventClassName = getEventClass().getSimpleName();
|
||||||
|
closer.assertOpen(() -> String.format("The processor was previously closed, so no further %s processing can occur", eventClassName));
|
||||||
|
|
||||||
|
List<T> events = drain();
|
||||||
|
|
||||||
|
try {
|
||||||
|
log.debug("Starting processing of {} {}(s)", events.size(), eventClassName);
|
||||||
|
|
||||||
|
for (T event : events) {
|
||||||
|
try {
|
||||||
|
Objects.requireNonNull(event, () -> String.format("Attempted to process a null %s", eventClassName));
|
||||||
|
log.debug("Consuming {}: {}", eventClassName, event);
|
||||||
|
process(event);
|
||||||
|
} catch (Throwable t) {
|
||||||
|
log.warn("An error occurred when processing the {}: {}", eventClassName, t.getMessage(), t);
|
||||||
|
|
||||||
|
KafkaException error;
|
||||||
|
|
||||||
|
if (t instanceof KafkaException)
|
||||||
|
error = (KafkaException) t;
|
||||||
|
else
|
||||||
|
error = new KafkaException(t);
|
||||||
|
|
||||||
|
processErrorHandler.onError(event, error);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
log.debug("Completed processing of {} {}(s)", events.size(), eventClassName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* It is possible for the consumer to close before complete processing all the events in the queue. In
|
||||||
|
* this case, we need to throw an exception to notify the user the consumer is closed.
|
||||||
|
*/
|
||||||
|
private void closeInternal() {
|
||||||
|
String eventClassName = getEventClass().getSimpleName();
|
||||||
|
log.trace("Closing event processor for {}", eventClassName);
|
||||||
|
List<T> incompleteEvents = drain();
|
||||||
|
|
||||||
|
if (incompleteEvents.isEmpty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
KafkaException exception = new KafkaException("The consumer is closed");
|
||||||
|
|
||||||
|
// Check each of the events and if it has a Future that is incomplete, complete it exceptionally.
|
||||||
|
incompleteEvents
|
||||||
|
.stream()
|
||||||
|
.filter(e -> e instanceof CompletableEvent)
|
||||||
|
.map(e -> ((CompletableEvent<?>) e).future())
|
||||||
|
.filter(f -> !f.isDone())
|
||||||
|
.forEach(f -> {
|
||||||
|
log.debug("Completing {} with exception {}", f, exception.getMessage());
|
||||||
|
f.completeExceptionally(exception);
|
||||||
|
});
|
||||||
|
|
||||||
|
log.debug("Discarding {} {}s because the consumer is closing", incompleteEvents.size(), eventClassName);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Moves all the events from the queue to the returned list.
|
||||||
|
*/
|
||||||
|
private List<T> drain() {
|
||||||
|
LinkedList<T> events = new LinkedList<>();
|
||||||
|
eventQueue.drainTo(events);
|
||||||
|
return events;
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,62 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.kafka.clients.consumer.internals.events;
|
|
||||||
|
|
||||||
import java.util.Objects;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The event is a no-op, but is intentionally left here for demonstration and test purposes.
|
|
||||||
*/
|
|
||||||
public class NoopApplicationEvent extends ApplicationEvent {
|
|
||||||
|
|
||||||
private final String message;
|
|
||||||
|
|
||||||
public NoopApplicationEvent(final String message) {
|
|
||||||
super(Type.NOOP);
|
|
||||||
this.message = Objects.requireNonNull(message);
|
|
||||||
}
|
|
||||||
|
|
||||||
public String message() {
|
|
||||||
return message;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
if (this == o) return true;
|
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
|
||||||
if (!super.equals(o)) return false;
|
|
||||||
|
|
||||||
NoopApplicationEvent that = (NoopApplicationEvent) o;
|
|
||||||
|
|
||||||
return message.equals(that.message);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
int result = super.hashCode();
|
|
||||||
result = 31 * result + message.hashCode();
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "NoopApplicationEvent{" +
|
|
||||||
toStringBase() +
|
|
||||||
",message='" + message + '\'' +
|
|
||||||
'}';
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,62 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.kafka.clients.consumer.internals.events;
|
|
||||||
|
|
||||||
import java.util.Objects;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* No-op event. Intentionally left it here for demonstration purpose.
|
|
||||||
*/
|
|
||||||
public class NoopBackgroundEvent extends BackgroundEvent {
|
|
||||||
|
|
||||||
private final String message;
|
|
||||||
|
|
||||||
public NoopBackgroundEvent(final String message) {
|
|
||||||
super(Type.NOOP);
|
|
||||||
this.message = Objects.requireNonNull(message);
|
|
||||||
}
|
|
||||||
|
|
||||||
public String message() {
|
|
||||||
return message;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
if (this == o) return true;
|
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
|
||||||
if (!super.equals(o)) return false;
|
|
||||||
|
|
||||||
NoopBackgroundEvent that = (NoopBackgroundEvent) o;
|
|
||||||
|
|
||||||
return message.equals(that.message);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
int result = super.hashCode();
|
|
||||||
result = 31 * result + message.hashCode();
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "NoopBackgroundEvent{" +
|
|
||||||
toStringBase() +
|
|
||||||
", message='" + message + '\'' +
|
|
||||||
'}';
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -2674,7 +2674,7 @@ public class KafkaConsumerTest {
|
||||||
}
|
}
|
||||||
IsolationLevel isolationLevel = IsolationLevel.READ_UNCOMMITTED;
|
IsolationLevel isolationLevel = IsolationLevel.READ_UNCOMMITTED;
|
||||||
FetchMetricsManager metricsManager = new FetchMetricsManager(metrics, metricsRegistry.fetcherMetrics);
|
FetchMetricsManager metricsManager = new FetchMetricsManager(metrics, metricsRegistry.fetcherMetrics);
|
||||||
FetchConfig<String, String> fetchConfig = new FetchConfig<>(
|
FetchConfig fetchConfig = new FetchConfig(
|
||||||
minBytes,
|
minBytes,
|
||||||
maxBytes,
|
maxBytes,
|
||||||
maxWaitMs,
|
maxWaitMs,
|
||||||
|
@ -2682,7 +2682,6 @@ public class KafkaConsumerTest {
|
||||||
maxPollRecords,
|
maxPollRecords,
|
||||||
checkCrcs,
|
checkCrcs,
|
||||||
CommonClientConfigs.DEFAULT_CLIENT_RACK,
|
CommonClientConfigs.DEFAULT_CLIENT_RACK,
|
||||||
new Deserializers<>(keyDeserializer, deserializer),
|
|
||||||
isolationLevel);
|
isolationLevel);
|
||||||
Fetcher<String, String> fetcher = new Fetcher<>(
|
Fetcher<String, String> fetcher = new Fetcher<>(
|
||||||
loggerFactory,
|
loggerFactory,
|
||||||
|
@ -2690,6 +2689,7 @@ public class KafkaConsumerTest {
|
||||||
metadata,
|
metadata,
|
||||||
subscription,
|
subscription,
|
||||||
fetchConfig,
|
fetchConfig,
|
||||||
|
new Deserializers<>(keyDeserializer, deserializer),
|
||||||
metricsManager,
|
metricsManager,
|
||||||
time);
|
time);
|
||||||
OffsetFetcher offsetFetcher = new OffsetFetcher(loggerFactory,
|
OffsetFetcher offsetFetcher = new OffsetFetcher(loggerFactory,
|
||||||
|
|
|
@ -55,6 +55,7 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_COMMIT_INTER
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG;
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG;
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_ID;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
@ -76,7 +77,7 @@ public class CommitRequestManagerTest {
|
||||||
this.time = new MockTime(0);
|
this.time = new MockTime(0);
|
||||||
this.subscriptionState = mock(SubscriptionState.class);
|
this.subscriptionState = mock(SubscriptionState.class);
|
||||||
this.coordinatorRequestManager = mock(CoordinatorRequestManager.class);
|
this.coordinatorRequestManager = mock(CoordinatorRequestManager.class);
|
||||||
this.groupState = new GroupState("group-1", Optional.empty());
|
this.groupState = new GroupState(DEFAULT_GROUP_ID, Optional.empty());
|
||||||
|
|
||||||
this.props = new Properties();
|
this.props = new Properties();
|
||||||
this.props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 100);
|
this.props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 100);
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.kafka.common.record.RecordBatch;
|
||||||
import org.apache.kafka.common.record.Records;
|
import org.apache.kafka.common.record.Records;
|
||||||
import org.apache.kafka.common.record.SimpleRecord;
|
import org.apache.kafka.common.record.SimpleRecord;
|
||||||
import org.apache.kafka.common.record.TimestampType;
|
import org.apache.kafka.common.record.TimestampType;
|
||||||
import org.apache.kafka.common.serialization.Deserializer;
|
|
||||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||||
import org.apache.kafka.common.serialization.UUIDDeserializer;
|
import org.apache.kafka.common.serialization.UUIDDeserializer;
|
||||||
import org.apache.kafka.common.serialization.UUIDSerializer;
|
import org.apache.kafka.common.serialization.UUIDSerializer;
|
||||||
|
@ -67,23 +66,22 @@ public class CompletedFetchTest {
|
||||||
FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
|
FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
|
||||||
.setRecords(newRecords(startingOffset, numRecords, fetchOffset));
|
.setRecords(newRecords(startingOffset, numRecords, fetchOffset));
|
||||||
|
|
||||||
FetchConfig<String, String> fetchConfig = newFetchConfig(new StringDeserializer(),
|
Deserializers<String, String> deserializers = newStringDeserializers();
|
||||||
new StringDeserializer(),
|
FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_UNCOMMITTED, true);
|
||||||
IsolationLevel.READ_UNCOMMITTED,
|
|
||||||
true);
|
|
||||||
CompletedFetch completedFetch = newCompletedFetch(fetchOffset, partitionData);
|
CompletedFetch completedFetch = newCompletedFetch(fetchOffset, partitionData);
|
||||||
|
|
||||||
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, 10);
|
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, deserializers, 10);
|
||||||
assertEquals(10, records.size());
|
assertEquals(10, records.size());
|
||||||
ConsumerRecord<String, String> record = records.get(0);
|
ConsumerRecord<String, String> record = records.get(0);
|
||||||
assertEquals(10, record.offset());
|
assertEquals(10, record.offset());
|
||||||
|
|
||||||
records = completedFetch.fetchRecords(fetchConfig, 10);
|
records = completedFetch.fetchRecords(fetchConfig, deserializers, 10);
|
||||||
assertEquals(1, records.size());
|
assertEquals(1, records.size());
|
||||||
record = records.get(0);
|
record = records.get(0);
|
||||||
assertEquals(20, record.offset());
|
assertEquals(20, record.offset());
|
||||||
|
|
||||||
records = completedFetch.fetchRecords(fetchConfig, 10);
|
records = completedFetch.fetchRecords(fetchConfig, deserializers, 10);
|
||||||
assertEquals(0, records.size());
|
assertEquals(0, records.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -96,21 +94,15 @@ public class CompletedFetchTest {
|
||||||
.setRecords(rawRecords)
|
.setRecords(rawRecords)
|
||||||
.setAbortedTransactions(newAbortedTransactions());
|
.setAbortedTransactions(newAbortedTransactions());
|
||||||
|
|
||||||
try (final StringDeserializer deserializer = new StringDeserializer()) {
|
try (final Deserializers<String, String> deserializers = newStringDeserializers()) {
|
||||||
FetchConfig<String, String> fetchConfig = newFetchConfig(deserializer,
|
FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_COMMITTED, true);
|
||||||
deserializer,
|
|
||||||
IsolationLevel.READ_COMMITTED,
|
|
||||||
true);
|
|
||||||
CompletedFetch completedFetch = newCompletedFetch(0, partitionData);
|
CompletedFetch completedFetch = newCompletedFetch(0, partitionData);
|
||||||
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, 10);
|
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, deserializers, 10);
|
||||||
assertEquals(0, records.size());
|
assertEquals(0, records.size());
|
||||||
|
|
||||||
fetchConfig = newFetchConfig(deserializer,
|
fetchConfig = newFetchConfig(IsolationLevel.READ_UNCOMMITTED, true);
|
||||||
deserializer,
|
|
||||||
IsolationLevel.READ_UNCOMMITTED,
|
|
||||||
true);
|
|
||||||
completedFetch = newCompletedFetch(0, partitionData);
|
completedFetch = newCompletedFetch(0, partitionData);
|
||||||
records = completedFetch.fetchRecords(fetchConfig, 10);
|
records = completedFetch.fetchRecords(fetchConfig, deserializers, 10);
|
||||||
assertEquals(numRecords, records.size());
|
assertEquals(numRecords, records.size());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -122,12 +114,9 @@ public class CompletedFetchTest {
|
||||||
FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
|
FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
|
||||||
.setRecords(rawRecords);
|
.setRecords(rawRecords);
|
||||||
CompletedFetch completedFetch = newCompletedFetch(0, partitionData);
|
CompletedFetch completedFetch = newCompletedFetch(0, partitionData);
|
||||||
try (final StringDeserializer deserializer = new StringDeserializer()) {
|
try (final Deserializers<String, String> deserializers = newStringDeserializers()) {
|
||||||
FetchConfig<String, String> fetchConfig = newFetchConfig(deserializer,
|
FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_COMMITTED, true);
|
||||||
deserializer,
|
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, deserializers, 10);
|
||||||
IsolationLevel.READ_COMMITTED,
|
|
||||||
true);
|
|
||||||
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, 10);
|
|
||||||
assertEquals(10, records.size());
|
assertEquals(10, records.size());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -140,14 +129,13 @@ public class CompletedFetchTest {
|
||||||
FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
|
FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
|
||||||
.setRecords(newRecords(startingOffset, numRecords, fetchOffset));
|
.setRecords(newRecords(startingOffset, numRecords, fetchOffset));
|
||||||
|
|
||||||
CompletedFetch completedFetch = newCompletedFetch(fetchOffset, partitionData);
|
try (final Deserializers<String, String> deserializers = newStringDeserializers()) {
|
||||||
FetchConfig<String, String> fetchConfig = newFetchConfig(new StringDeserializer(),
|
CompletedFetch completedFetch = newCompletedFetch(fetchOffset, partitionData);
|
||||||
new StringDeserializer(),
|
FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_UNCOMMITTED, true);
|
||||||
IsolationLevel.READ_UNCOMMITTED,
|
|
||||||
true);
|
|
||||||
|
|
||||||
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, -10);
|
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, deserializers, -10);
|
||||||
assertEquals(0, records.size());
|
assertEquals(0, records.size());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -159,13 +147,9 @@ public class CompletedFetchTest {
|
||||||
.setLogStartOffset(0);
|
.setLogStartOffset(0);
|
||||||
|
|
||||||
CompletedFetch completedFetch = newCompletedFetch(1, partitionData);
|
CompletedFetch completedFetch = newCompletedFetch(1, partitionData);
|
||||||
try (final StringDeserializer deserializer = new StringDeserializer()) {
|
try (final Deserializers<String, String> deserializers = newStringDeserializers()) {
|
||||||
FetchConfig<String, String> fetchConfig = newFetchConfig(deserializer,
|
FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_UNCOMMITTED, true);
|
||||||
deserializer,
|
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, deserializers, 10);
|
||||||
IsolationLevel.READ_UNCOMMITTED,
|
|
||||||
true);
|
|
||||||
|
|
||||||
List<ConsumerRecord<String, String>> records = completedFetch.fetchRecords(fetchConfig, 10);
|
|
||||||
assertEquals(0, records.size());
|
assertEquals(0, records.size());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -174,8 +158,7 @@ public class CompletedFetchTest {
|
||||||
public void testCorruptedMessage() {
|
public void testCorruptedMessage() {
|
||||||
// Create one good record and then one "corrupted" record.
|
// Create one good record and then one "corrupted" record.
|
||||||
try (final MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 0);
|
try (final MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 0);
|
||||||
final UUIDSerializer serializer = new UUIDSerializer();
|
final UUIDSerializer serializer = new UUIDSerializer()) {
|
||||||
final UUIDDeserializer deserializer = new UUIDDeserializer()) {
|
|
||||||
builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID())));
|
builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID())));
|
||||||
builder.append(0L, "key".getBytes(), "value".getBytes());
|
builder.append(0L, "key".getBytes(), "value".getBytes());
|
||||||
Records records = builder.build();
|
Records records = builder.build();
|
||||||
|
@ -187,16 +170,15 @@ public class CompletedFetchTest {
|
||||||
.setLogStartOffset(0)
|
.setLogStartOffset(0)
|
||||||
.setRecords(records);
|
.setRecords(records);
|
||||||
|
|
||||||
FetchConfig<UUID, UUID> fetchConfig = newFetchConfig(deserializer,
|
try (final Deserializers<UUID, UUID> deserializers = newUuidDeserializers()) {
|
||||||
deserializer,
|
FetchConfig fetchConfig = newFetchConfig(IsolationLevel.READ_COMMITTED, false);
|
||||||
IsolationLevel.READ_COMMITTED,
|
CompletedFetch completedFetch = newCompletedFetch(0, partitionData);
|
||||||
false);
|
|
||||||
CompletedFetch completedFetch = newCompletedFetch(0, partitionData);
|
|
||||||
|
|
||||||
completedFetch.fetchRecords(fetchConfig, 10);
|
completedFetch.fetchRecords(fetchConfig, deserializers, 10);
|
||||||
|
|
||||||
assertThrows(RecordDeserializationException.class,
|
assertThrows(RecordDeserializationException.class,
|
||||||
() -> completedFetch.fetchRecords(fetchConfig, 10));
|
() -> completedFetch.fetchRecords(fetchConfig, deserializers, 10));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -219,11 +201,16 @@ public class CompletedFetchTest {
|
||||||
ApiKeys.FETCH.latestVersion());
|
ApiKeys.FETCH.latestVersion());
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <K, V> FetchConfig<K, V> newFetchConfig(Deserializer<K> keyDeserializer,
|
private static Deserializers<UUID, UUID> newUuidDeserializers() {
|
||||||
Deserializer<V> valueDeserializer,
|
return new Deserializers<>(new UUIDDeserializer(), new UUIDDeserializer());
|
||||||
IsolationLevel isolationLevel,
|
}
|
||||||
boolean checkCrcs) {
|
|
||||||
return new FetchConfig<>(
|
private static Deserializers<String, String> newStringDeserializers() {
|
||||||
|
return new Deserializers<>(new StringDeserializer(), new StringDeserializer());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static FetchConfig newFetchConfig(IsolationLevel isolationLevel, boolean checkCrcs) {
|
||||||
|
return new FetchConfig(
|
||||||
ConsumerConfig.DEFAULT_FETCH_MIN_BYTES,
|
ConsumerConfig.DEFAULT_FETCH_MIN_BYTES,
|
||||||
ConsumerConfig.DEFAULT_FETCH_MAX_BYTES,
|
ConsumerConfig.DEFAULT_FETCH_MAX_BYTES,
|
||||||
ConsumerConfig.DEFAULT_FETCH_MAX_WAIT_MS,
|
ConsumerConfig.DEFAULT_FETCH_MAX_WAIT_MS,
|
||||||
|
@ -231,7 +218,6 @@ public class CompletedFetchTest {
|
||||||
ConsumerConfig.DEFAULT_MAX_POLL_RECORDS,
|
ConsumerConfig.DEFAULT_MAX_POLL_RECORDS,
|
||||||
checkCrcs,
|
checkCrcs,
|
||||||
ConsumerConfig.DEFAULT_CLIENT_RACK,
|
ConsumerConfig.DEFAULT_CLIENT_RACK,
|
||||||
new Deserializers<>(keyDeserializer, valueDeserializer),
|
|
||||||
isolationLevel
|
isolationLevel
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -3082,7 +3082,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
|
|
||||||
subscriptions.assignFromUser(singleton(t1p));
|
subscriptions.assignFromUser(singleton(t1p));
|
||||||
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L));
|
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L));
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
||||||
|
|
||||||
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
||||||
assertTrue(subscriptions.hasAllFetchPositions());
|
assertTrue(subscriptions.hasAllFetchPositions());
|
||||||
|
@ -3103,7 +3103,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
|
|
||||||
// Load offsets from previous epoch
|
// Load offsets from previous epoch
|
||||||
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L, Optional.of(3)));
|
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L, Optional.of(3)));
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
||||||
|
|
||||||
// Offset gets loaded, but requires validation
|
// Offset gets loaded, but requires validation
|
||||||
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
||||||
|
@ -3155,7 +3155,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
subscriptions.assignFromUser(singleton(t1p));
|
subscriptions.assignFromUser(singleton(t1p));
|
||||||
client.prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap()));
|
client.prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap()));
|
||||||
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L));
|
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L));
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
||||||
|
|
||||||
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
||||||
assertTrue(subscriptions.hasAllFetchPositions());
|
assertTrue(subscriptions.hasAllFetchPositions());
|
||||||
|
@ -3170,7 +3170,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
subscriptions.assignFromUser(singleton(t1p));
|
subscriptions.assignFromUser(singleton(t1p));
|
||||||
client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, Collections.emptyMap()));
|
client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, Collections.emptyMap()));
|
||||||
try {
|
try {
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
||||||
fail("Expected group authorization error");
|
fail("Expected group authorization error");
|
||||||
} catch (GroupAuthorizationException e) {
|
} catch (GroupAuthorizationException e) {
|
||||||
assertEquals(groupId, e.groupId());
|
assertEquals(groupId, e.groupId());
|
||||||
|
@ -3186,9 +3186,9 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
client.prepareResponse(offsetFetchResponse(t1p, Errors.UNSTABLE_OFFSET_COMMIT, "", -1L));
|
client.prepareResponse(offsetFetchResponse(t1p, Errors.UNSTABLE_OFFSET_COMMIT, "", -1L));
|
||||||
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L));
|
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L));
|
||||||
assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions());
|
assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions());
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(0L));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(0L));
|
||||||
assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions());
|
assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions());
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(0L));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(0L));
|
||||||
|
|
||||||
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
||||||
assertTrue(subscriptions.hasAllFetchPositions());
|
assertTrue(subscriptions.hasAllFetchPositions());
|
||||||
|
@ -3202,7 +3202,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
|
|
||||||
subscriptions.assignFromUser(singleton(t1p));
|
subscriptions.assignFromUser(singleton(t1p));
|
||||||
client.prepareResponse(offsetFetchResponse(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION, "", 100L));
|
client.prepareResponse(offsetFetchResponse(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION, "", 100L));
|
||||||
assertThrows(KafkaException.class, () -> coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)));
|
assertThrows(KafkaException.class, () -> coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -3214,7 +3214,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap()));
|
client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap()));
|
||||||
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
|
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
|
||||||
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L));
|
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L));
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
||||||
|
|
||||||
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
||||||
assertTrue(subscriptions.hasAllFetchPositions());
|
assertTrue(subscriptions.hasAllFetchPositions());
|
||||||
|
@ -3228,7 +3228,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
|
|
||||||
subscriptions.assignFromUser(singleton(t1p));
|
subscriptions.assignFromUser(singleton(t1p));
|
||||||
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", -1L));
|
client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", -1L));
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
||||||
|
|
||||||
assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions());
|
assertEquals(Collections.singleton(t1p), subscriptions.initializingPartitions());
|
||||||
assertEquals(Collections.emptySet(), subscriptions.partitionsNeedingReset(time.milliseconds()));
|
assertEquals(Collections.emptySet(), subscriptions.partitionsNeedingReset(time.milliseconds()));
|
||||||
|
@ -3242,7 +3242,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
|
|
||||||
subscriptions.assignFromUser(singleton(t1p));
|
subscriptions.assignFromUser(singleton(t1p));
|
||||||
subscriptions.seek(t1p, 500L);
|
subscriptions.seek(t1p, 500L);
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
||||||
|
|
||||||
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
||||||
assertTrue(subscriptions.hasAllFetchPositions());
|
assertTrue(subscriptions.hasAllFetchPositions());
|
||||||
|
@ -3256,7 +3256,7 @@ public abstract class ConsumerCoordinatorTest {
|
||||||
|
|
||||||
subscriptions.assignFromUser(singleton(t1p));
|
subscriptions.assignFromUser(singleton(t1p));
|
||||||
subscriptions.requestOffsetReset(t1p, OffsetResetStrategy.EARLIEST);
|
subscriptions.requestOffsetReset(t1p, OffsetResetStrategy.EARLIEST);
|
||||||
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
|
||||||
|
|
||||||
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
assertEquals(Collections.emptySet(), subscriptions.initializingPartitions());
|
||||||
assertFalse(subscriptions.hasAllFetchPositions());
|
assertFalse(subscriptions.hasAllFetchPositions());
|
||||||
|
|
|
@ -0,0 +1,269 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.MockClient;
|
||||||
|
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.message.FindCoordinatorRequestData;
|
||||||
|
import org.apache.kafka.common.requests.FindCoordinatorRequest;
|
||||||
|
import org.apache.kafka.common.requests.MetadataResponse;
|
||||||
|
import org.apache.kafka.common.requests.RequestTestUtils;
|
||||||
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
import org.apache.kafka.test.TestCondition;
|
||||||
|
import org.apache.kafka.test.TestUtils;
|
||||||
|
import org.junit.jupiter.api.AfterEach;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REQUEST_TIMEOUT_MS;
|
||||||
|
import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
|
import static org.mockito.ArgumentMatchers.anyLong;
|
||||||
|
import static org.mockito.ArgumentMatchers.eq;
|
||||||
|
import static org.mockito.Mockito.doThrow;
|
||||||
|
import static org.mockito.Mockito.spy;
|
||||||
|
import static org.mockito.Mockito.times;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
@SuppressWarnings("ClassDataAbstractionCoupling")
|
||||||
|
public class ConsumerNetworkThreadTest {
|
||||||
|
|
||||||
|
private ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder testBuilder;
|
||||||
|
private Time time;
|
||||||
|
private ConsumerMetadata metadata;
|
||||||
|
private NetworkClientDelegate networkClient;
|
||||||
|
private BlockingQueue<ApplicationEvent> applicationEventsQueue;
|
||||||
|
private ApplicationEventProcessor applicationEventProcessor;
|
||||||
|
private OffsetsRequestManager offsetsRequestManager;
|
||||||
|
private CommitRequestManager commitManager;
|
||||||
|
private ConsumerNetworkThread consumerNetworkThread;
|
||||||
|
private MockClient client;
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setup() {
|
||||||
|
testBuilder = new ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder();
|
||||||
|
time = testBuilder.time;
|
||||||
|
metadata = testBuilder.metadata;
|
||||||
|
networkClient = testBuilder.networkClientDelegate;
|
||||||
|
client = testBuilder.client;
|
||||||
|
applicationEventsQueue = testBuilder.applicationEventQueue;
|
||||||
|
applicationEventProcessor = testBuilder.applicationEventProcessor;
|
||||||
|
commitManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new);
|
||||||
|
offsetsRequestManager = testBuilder.offsetsRequestManager;
|
||||||
|
consumerNetworkThread = testBuilder.consumerNetworkThread;
|
||||||
|
consumerNetworkThread.initializeResources();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterEach
|
||||||
|
public void tearDown() {
|
||||||
|
if (testBuilder != null)
|
||||||
|
testBuilder.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testStartupAndTearDown() throws InterruptedException {
|
||||||
|
// The consumer is closed in ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder.close()
|
||||||
|
// which is called from tearDown().
|
||||||
|
consumerNetworkThread.start();
|
||||||
|
|
||||||
|
TestCondition isStarted = () -> consumerNetworkThread.isRunning();
|
||||||
|
TestCondition isClosed = () -> !(consumerNetworkThread.isRunning() || consumerNetworkThread.isAlive());
|
||||||
|
|
||||||
|
// There's a nonzero amount of time between starting the thread and having it
|
||||||
|
// begin to execute our code. Wait for a bit before checking...
|
||||||
|
TestUtils.waitForCondition(isStarted,
|
||||||
|
"The consumer network thread did not start within " + DEFAULT_MAX_WAIT_MS + " ms");
|
||||||
|
|
||||||
|
consumerNetworkThread.close(Duration.ofMillis(DEFAULT_MAX_WAIT_MS));
|
||||||
|
|
||||||
|
TestUtils.waitForCondition(isClosed,
|
||||||
|
"The consumer network thread did not stop within " + DEFAULT_MAX_WAIT_MS + " ms");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testApplicationEvent() {
|
||||||
|
ApplicationEvent e = new CommitApplicationEvent(new HashMap<>());
|
||||||
|
applicationEventsQueue.add(e);
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(applicationEventProcessor, times(1)).process(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMetadataUpdateEvent() {
|
||||||
|
ApplicationEvent e = new NewTopicsMetadataUpdateRequestEvent();
|
||||||
|
applicationEventsQueue.add(e);
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(metadata).requestUpdateForNewTopics();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitEvent() {
|
||||||
|
ApplicationEvent e = new CommitApplicationEvent(new HashMap<>());
|
||||||
|
applicationEventsQueue.add(e);
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(applicationEventProcessor).process(any(CommitApplicationEvent.class));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testListOffsetsEventIsProcessed() {
|
||||||
|
Map<TopicPartition, Long> timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L);
|
||||||
|
ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true);
|
||||||
|
applicationEventsQueue.add(e);
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class));
|
||||||
|
assertTrue(applicationEventsQueue.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testResetPositionsEventIsProcessed() {
|
||||||
|
ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent();
|
||||||
|
applicationEventsQueue.add(e);
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class));
|
||||||
|
assertTrue(applicationEventsQueue.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testResetPositionsProcessFailureIsIgnored() {
|
||||||
|
doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded();
|
||||||
|
|
||||||
|
ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent();
|
||||||
|
applicationEventsQueue.add(event);
|
||||||
|
assertDoesNotThrow(() -> consumerNetworkThread.runOnce());
|
||||||
|
|
||||||
|
verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidatePositionsEventIsProcessed() {
|
||||||
|
ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent();
|
||||||
|
applicationEventsQueue.add(e);
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class));
|
||||||
|
assertTrue(applicationEventsQueue.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAssignmentChangeEvent() {
|
||||||
|
HashMap<TopicPartition, OffsetAndMetadata> offset = mockTopicPartitionOffset();
|
||||||
|
|
||||||
|
final long currentTimeMs = time.milliseconds();
|
||||||
|
ApplicationEvent e = new AssignmentChangeApplicationEvent(offset, currentTimeMs);
|
||||||
|
applicationEventsQueue.add(e);
|
||||||
|
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(applicationEventProcessor).process(any(AssignmentChangeApplicationEvent.class));
|
||||||
|
verify(networkClient, times(1)).poll(anyLong(), anyLong());
|
||||||
|
verify(commitManager, times(1)).updateAutoCommitTimer(currentTimeMs);
|
||||||
|
verify(commitManager, times(1)).maybeAutoCommit(offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testFetchTopicMetadata() {
|
||||||
|
applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic"));
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testPollResultTimer() {
|
||||||
|
NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest(
|
||||||
|
new FindCoordinatorRequest.Builder(
|
||||||
|
new FindCoordinatorRequestData()
|
||||||
|
.setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id())
|
||||||
|
.setKey("foobar")),
|
||||||
|
Optional.empty());
|
||||||
|
req.setTimer(time, DEFAULT_REQUEST_TIMEOUT_MS);
|
||||||
|
|
||||||
|
// purposely setting a non-MAX time to ensure it is returning Long.MAX_VALUE upon success
|
||||||
|
NetworkClientDelegate.PollResult success = new NetworkClientDelegate.PollResult(
|
||||||
|
10,
|
||||||
|
Collections.singletonList(req));
|
||||||
|
assertEquals(10, networkClient.addAll(success));
|
||||||
|
|
||||||
|
NetworkClientDelegate.PollResult failure = new NetworkClientDelegate.PollResult(
|
||||||
|
10,
|
||||||
|
new ArrayList<>());
|
||||||
|
assertEquals(10, networkClient.addAll(failure));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testRequestManagersArePolledOnce() {
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
testBuilder.requestManagers.entries().forEach(rmo -> rmo.ifPresent(rm -> verify(rm, times(1)).poll(anyLong())));
|
||||||
|
verify(networkClient, times(1)).poll(anyLong(), anyLong());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testEnsureMetadataUpdateOnPoll() {
|
||||||
|
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap());
|
||||||
|
client.prepareMetadataUpdate(metadataResponse);
|
||||||
|
metadata.requestUpdate(false);
|
||||||
|
consumerNetworkThread.runOnce();
|
||||||
|
verify(metadata, times(1)).updateWithCurrentRequestVersion(eq(metadataResponse), eq(false), anyLong());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testEnsureEventsAreCompleted() {
|
||||||
|
CompletableApplicationEvent<Void> event1 = spy(new CommitApplicationEvent(Collections.emptyMap()));
|
||||||
|
ApplicationEvent event2 = new CommitApplicationEvent(Collections.emptyMap());
|
||||||
|
CompletableFuture<Void> future = new CompletableFuture<>();
|
||||||
|
when(event1.future()).thenReturn(future);
|
||||||
|
applicationEventsQueue.add(event1);
|
||||||
|
applicationEventsQueue.add(event2);
|
||||||
|
assertFalse(future.isDone());
|
||||||
|
assertFalse(applicationEventsQueue.isEmpty());
|
||||||
|
|
||||||
|
consumerNetworkThread.cleanup();
|
||||||
|
assertTrue(future.isCompletedExceptionally());
|
||||||
|
assertTrue(applicationEventsQueue.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
private HashMap<TopicPartition, OffsetAndMetadata> mockTopicPartitionOffset() {
|
||||||
|
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));
|
||||||
|
topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L));
|
||||||
|
return topicPartitionOffsets;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,370 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.ApiVersions;
|
||||||
|
import org.apache.kafka.clients.CommonClientConfigs;
|
||||||
|
import org.apache.kafka.clients.GroupRebalanceConfig;
|
||||||
|
import org.apache.kafka.clients.MockClient;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventProcessor;
|
||||||
|
import org.apache.kafka.common.internals.ClusterResourceListeners;
|
||||||
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
|
import org.apache.kafka.common.requests.MetadataResponse;
|
||||||
|
import org.apache.kafka.common.requests.RequestTestUtils;
|
||||||
|
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||||
|
import org.apache.kafka.common.utils.LogContext;
|
||||||
|
import org.apache.kafka.common.utils.MockTime;
|
||||||
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Properties;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
|
||||||
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_INSTANCE_ID_CONFIG;
|
||||||
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
||||||
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager;
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics;
|
||||||
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState;
|
||||||
|
import static org.apache.kafka.common.utils.Utils.closeQuietly;
|
||||||
|
import static org.mockito.Mockito.spy;
|
||||||
|
|
||||||
|
@SuppressWarnings("ClassDataAbstractionCoupling")
|
||||||
|
public class ConsumerTestBuilder implements Closeable {
|
||||||
|
|
||||||
|
static final long DEFAULT_RETRY_BACKOFF_MS = 80;
|
||||||
|
static final long DEFAULT_RETRY_BACKOFF_MAX_MS = 1000;
|
||||||
|
static final int DEFAULT_REQUEST_TIMEOUT_MS = 500;
|
||||||
|
static final int DEFAULT_MAX_POLL_INTERVAL_MS = 10000;
|
||||||
|
static final String DEFAULT_GROUP_INSTANCE_ID = "group-instance-id";
|
||||||
|
static final String DEFAULT_GROUP_ID = "group-id";
|
||||||
|
static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 1000;
|
||||||
|
static final double DEFAULT_HEARTBEAT_JITTER_MS = 0.0;
|
||||||
|
|
||||||
|
final LogContext logContext = new LogContext();
|
||||||
|
final Time time = new MockTime(0);
|
||||||
|
public final BlockingQueue<ApplicationEvent> applicationEventQueue;
|
||||||
|
public final BlockingQueue<BackgroundEvent> backgroundEventQueue;
|
||||||
|
final ConsumerConfig config;
|
||||||
|
final long retryBackoffMs;
|
||||||
|
final SubscriptionState subscriptions;
|
||||||
|
final ConsumerMetadata metadata;
|
||||||
|
final FetchConfig fetchConfig;
|
||||||
|
final FetchBuffer fetchBuffer;
|
||||||
|
final Metrics metrics;
|
||||||
|
final FetchMetricsManager metricsManager;
|
||||||
|
final NetworkClientDelegate networkClientDelegate;
|
||||||
|
final OffsetsRequestManager offsetsRequestManager;
|
||||||
|
final Optional<CoordinatorRequestManager> coordinatorRequestManager;
|
||||||
|
final Optional<CommitRequestManager> commitRequestManager;
|
||||||
|
final Optional<HeartbeatRequestManager> heartbeatRequestManager;
|
||||||
|
final Optional<MembershipManager> membershipManager;
|
||||||
|
final Optional<HeartbeatRequestManager.HeartbeatRequestState> heartbeatRequestState;
|
||||||
|
final TopicMetadataRequestManager topicMetadataRequestManager;
|
||||||
|
final FetchRequestManager fetchRequestManager;
|
||||||
|
final RequestManagers requestManagers;
|
||||||
|
public final ApplicationEventProcessor applicationEventProcessor;
|
||||||
|
public final BackgroundEventProcessor backgroundEventProcessor;
|
||||||
|
public final BackgroundEventHandler backgroundEventHandler;
|
||||||
|
final MockClient client;
|
||||||
|
final Optional<GroupInformation> groupInfo;
|
||||||
|
|
||||||
|
public ConsumerTestBuilder() {
|
||||||
|
this(Optional.empty());
|
||||||
|
}
|
||||||
|
|
||||||
|
public ConsumerTestBuilder(Optional<GroupInformation> groupInfo) {
|
||||||
|
this.groupInfo = groupInfo;
|
||||||
|
this.applicationEventQueue = new LinkedBlockingQueue<>();
|
||||||
|
this.backgroundEventQueue = new LinkedBlockingQueue<>();
|
||||||
|
this.backgroundEventHandler = spy(new BackgroundEventHandler(logContext, backgroundEventQueue));
|
||||||
|
GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
|
||||||
|
100,
|
||||||
|
DEFAULT_MAX_POLL_INTERVAL_MS,
|
||||||
|
DEFAULT_HEARTBEAT_INTERVAL_MS,
|
||||||
|
groupInfo.map(gi -> gi.groupState.groupId).orElse(null),
|
||||||
|
groupInfo.flatMap(gi -> gi.groupState.groupInstanceId),
|
||||||
|
DEFAULT_RETRY_BACKOFF_MS,
|
||||||
|
DEFAULT_RETRY_BACKOFF_MAX_MS,
|
||||||
|
true);
|
||||||
|
GroupState groupState = new GroupState(groupRebalanceConfig);
|
||||||
|
ApiVersions apiVersions = new ApiVersions();
|
||||||
|
|
||||||
|
Properties properties = new Properties();
|
||||||
|
properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
||||||
|
properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
||||||
|
properties.put(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG, DEFAULT_RETRY_BACKOFF_MS);
|
||||||
|
properties.put(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG, DEFAULT_REQUEST_TIMEOUT_MS);
|
||||||
|
properties.put(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG, DEFAULT_MAX_POLL_INTERVAL_MS);
|
||||||
|
|
||||||
|
groupInfo.ifPresent(gi -> {
|
||||||
|
properties.put(GROUP_ID_CONFIG, gi.groupState.groupId);
|
||||||
|
gi.groupState.groupInstanceId.ifPresent(groupInstanceId -> properties.put(GROUP_INSTANCE_ID_CONFIG, groupInstanceId));
|
||||||
|
});
|
||||||
|
|
||||||
|
this.config = new ConsumerConfig(properties);
|
||||||
|
|
||||||
|
this.fetchConfig = new FetchConfig(config);
|
||||||
|
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
|
||||||
|
final long requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
|
||||||
|
this.metrics = createMetrics(config, time);
|
||||||
|
|
||||||
|
this.subscriptions = spy(createSubscriptionState(config, logContext));
|
||||||
|
this.metadata = spy(new ConsumerMetadata(config, subscriptions, logContext, new ClusterResourceListeners()));
|
||||||
|
this.metricsManager = createFetchMetricsManager(metrics);
|
||||||
|
|
||||||
|
this.client = new MockClient(time, metadata);
|
||||||
|
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(1, new HashMap<String, Integer>() {
|
||||||
|
{
|
||||||
|
String topic1 = "test1";
|
||||||
|
put(topic1, 1);
|
||||||
|
String topic2 = "test2";
|
||||||
|
put(topic2, 1);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
this.client.updateMetadata(metadataResponse);
|
||||||
|
|
||||||
|
this.networkClientDelegate = spy(new NetworkClientDelegate(time,
|
||||||
|
config,
|
||||||
|
logContext,
|
||||||
|
client));
|
||||||
|
this.offsetsRequestManager = spy(new OffsetsRequestManager(subscriptions,
|
||||||
|
metadata,
|
||||||
|
fetchConfig.isolationLevel,
|
||||||
|
time,
|
||||||
|
retryBackoffMs,
|
||||||
|
requestTimeoutMs,
|
||||||
|
apiVersions,
|
||||||
|
networkClientDelegate,
|
||||||
|
backgroundEventHandler,
|
||||||
|
logContext));
|
||||||
|
|
||||||
|
if (groupInfo.isPresent()) {
|
||||||
|
GroupInformation gi = groupInfo.get();
|
||||||
|
CoordinatorRequestManager coordinator = spy(new CoordinatorRequestManager(
|
||||||
|
time,
|
||||||
|
logContext,
|
||||||
|
DEFAULT_RETRY_BACKOFF_MS,
|
||||||
|
DEFAULT_RETRY_BACKOFF_MAX_MS,
|
||||||
|
backgroundEventHandler,
|
||||||
|
gi.groupState.groupId
|
||||||
|
));
|
||||||
|
CommitRequestManager commit = spy(new CommitRequestManager(time,
|
||||||
|
logContext,
|
||||||
|
subscriptions,
|
||||||
|
config,
|
||||||
|
coordinator,
|
||||||
|
groupState));
|
||||||
|
MembershipManager mm = spy(
|
||||||
|
new MembershipManagerImpl(
|
||||||
|
gi.groupState.groupId,
|
||||||
|
gi.groupState.groupInstanceId.orElse(null),
|
||||||
|
null,
|
||||||
|
logContext
|
||||||
|
)
|
||||||
|
);
|
||||||
|
HeartbeatRequestManager.HeartbeatRequestState state = spy(new HeartbeatRequestManager.HeartbeatRequestState(logContext,
|
||||||
|
time,
|
||||||
|
gi.heartbeatIntervalMs,
|
||||||
|
retryBackoffMs,
|
||||||
|
DEFAULT_RETRY_BACKOFF_MAX_MS,
|
||||||
|
gi.heartbeatJitterMs));
|
||||||
|
HeartbeatRequestManager heartbeat = spy(new HeartbeatRequestManager(
|
||||||
|
logContext,
|
||||||
|
time,
|
||||||
|
config,
|
||||||
|
coordinator,
|
||||||
|
subscriptions,
|
||||||
|
mm,
|
||||||
|
state,
|
||||||
|
backgroundEventHandler));
|
||||||
|
|
||||||
|
this.coordinatorRequestManager = Optional.of(coordinator);
|
||||||
|
this.commitRequestManager = Optional.of(commit);
|
||||||
|
this.heartbeatRequestManager = Optional.of(heartbeat);
|
||||||
|
this.heartbeatRequestState = Optional.of(state);
|
||||||
|
this.membershipManager = Optional.of(mm);
|
||||||
|
} else {
|
||||||
|
this.coordinatorRequestManager = Optional.empty();
|
||||||
|
this.commitRequestManager = Optional.empty();
|
||||||
|
this.heartbeatRequestManager = Optional.empty();
|
||||||
|
this.heartbeatRequestState = Optional.empty();
|
||||||
|
this.membershipManager = Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
this.fetchBuffer = new FetchBuffer(logContext);
|
||||||
|
this.fetchRequestManager = spy(new FetchRequestManager(logContext,
|
||||||
|
time,
|
||||||
|
metadata,
|
||||||
|
subscriptions,
|
||||||
|
fetchConfig,
|
||||||
|
fetchBuffer,
|
||||||
|
metricsManager,
|
||||||
|
networkClientDelegate));
|
||||||
|
this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext,
|
||||||
|
config));
|
||||||
|
this.requestManagers = new RequestManagers(logContext,
|
||||||
|
offsetsRequestManager,
|
||||||
|
topicMetadataRequestManager,
|
||||||
|
fetchRequestManager,
|
||||||
|
coordinatorRequestManager,
|
||||||
|
commitRequestManager,
|
||||||
|
heartbeatRequestManager);
|
||||||
|
this.applicationEventProcessor = spy(new ApplicationEventProcessor(
|
||||||
|
logContext,
|
||||||
|
applicationEventQueue,
|
||||||
|
requestManagers,
|
||||||
|
metadata)
|
||||||
|
);
|
||||||
|
this.backgroundEventProcessor = spy(new BackgroundEventProcessor(logContext, backgroundEventQueue));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
closeQuietly(requestManagers, RequestManagers.class.getSimpleName());
|
||||||
|
closeQuietly(applicationEventProcessor, ApplicationEventProcessor.class.getSimpleName());
|
||||||
|
closeQuietly(backgroundEventProcessor, BackgroundEventProcessor.class.getSimpleName());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class ConsumerNetworkThreadTestBuilder extends ConsumerTestBuilder {
|
||||||
|
|
||||||
|
final ConsumerNetworkThread consumerNetworkThread;
|
||||||
|
|
||||||
|
public ConsumerNetworkThreadTestBuilder() {
|
||||||
|
this(createDefaultGroupInformation());
|
||||||
|
}
|
||||||
|
|
||||||
|
public ConsumerNetworkThreadTestBuilder(Optional<GroupInformation> groupInfo) {
|
||||||
|
super(groupInfo);
|
||||||
|
this.consumerNetworkThread = new ConsumerNetworkThread(
|
||||||
|
logContext,
|
||||||
|
time,
|
||||||
|
() -> applicationEventProcessor,
|
||||||
|
() -> networkClientDelegate,
|
||||||
|
() -> requestManagers
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
closeQuietly(consumerNetworkThread, ConsumerNetworkThread.class.getSimpleName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class ApplicationEventHandlerTestBuilder extends ConsumerTestBuilder {
|
||||||
|
|
||||||
|
public final ApplicationEventHandler applicationEventHandler;
|
||||||
|
|
||||||
|
public ApplicationEventHandlerTestBuilder() {
|
||||||
|
this(createDefaultGroupInformation());
|
||||||
|
}
|
||||||
|
|
||||||
|
public ApplicationEventHandlerTestBuilder(Optional<GroupInformation> groupInfo) {
|
||||||
|
super(groupInfo);
|
||||||
|
this.applicationEventHandler = spy(new ApplicationEventHandler(
|
||||||
|
logContext,
|
||||||
|
time,
|
||||||
|
applicationEventQueue,
|
||||||
|
() -> applicationEventProcessor,
|
||||||
|
() -> networkClientDelegate,
|
||||||
|
() -> requestManagers));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
closeQuietly(applicationEventHandler, ApplicationEventHandler.class.getSimpleName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class PrototypeAsyncConsumerTestBuilder extends ApplicationEventHandlerTestBuilder {
|
||||||
|
|
||||||
|
final PrototypeAsyncConsumer<String, String> consumer;
|
||||||
|
|
||||||
|
public PrototypeAsyncConsumerTestBuilder(Optional<GroupInformation> groupInfo) {
|
||||||
|
super(groupInfo);
|
||||||
|
String clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG);
|
||||||
|
List<ConsumerPartitionAssignor> assignors = ConsumerPartitionAssignor.getAssignorInstances(
|
||||||
|
config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
|
||||||
|
config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId))
|
||||||
|
);
|
||||||
|
Deserializers<String, String> deserializers = new Deserializers<>(new StringDeserializer(), new StringDeserializer());
|
||||||
|
FetchCollector<String, String> fetchCollector = new FetchCollector<>(logContext,
|
||||||
|
metadata,
|
||||||
|
subscriptions,
|
||||||
|
fetchConfig,
|
||||||
|
deserializers,
|
||||||
|
metricsManager,
|
||||||
|
time);
|
||||||
|
this.consumer = spy(new PrototypeAsyncConsumer<>(
|
||||||
|
logContext,
|
||||||
|
clientId,
|
||||||
|
deserializers,
|
||||||
|
new FetchBuffer(logContext),
|
||||||
|
fetchCollector,
|
||||||
|
new ConsumerInterceptors<>(Collections.emptyList()),
|
||||||
|
time,
|
||||||
|
applicationEventHandler,
|
||||||
|
backgroundEventQueue,
|
||||||
|
metrics,
|
||||||
|
subscriptions,
|
||||||
|
metadata,
|
||||||
|
retryBackoffMs,
|
||||||
|
60000,
|
||||||
|
assignors,
|
||||||
|
groupInfo.map(groupInformation -> groupInformation.groupState.groupId).orElse(null)));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
consumer.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class GroupInformation {
|
||||||
|
|
||||||
|
final GroupState groupState;
|
||||||
|
final int heartbeatIntervalMs;
|
||||||
|
final double heartbeatJitterMs;
|
||||||
|
|
||||||
|
public GroupInformation(GroupState groupState) {
|
||||||
|
this(groupState, DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_HEARTBEAT_JITTER_MS);
|
||||||
|
}
|
||||||
|
|
||||||
|
public GroupInformation(GroupState groupState, int heartbeatIntervalMs, double heartbeatJitterMs) {
|
||||||
|
this.groupState = groupState;
|
||||||
|
this.heartbeatIntervalMs = heartbeatIntervalMs;
|
||||||
|
this.heartbeatJitterMs = heartbeatJitterMs;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static Optional<GroupInformation> createDefaultGroupInformation() {
|
||||||
|
return Optional.of(new GroupInformation(new GroupState(DEFAULT_GROUP_ID, Optional.empty())));
|
||||||
|
}
|
||||||
|
}
|
|
@ -17,6 +17,8 @@
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.clients.ClientResponse;
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
||||||
import org.apache.kafka.common.errors.TimeoutException;
|
import org.apache.kafka.common.errors.TimeoutException;
|
||||||
|
@ -46,14 +48,14 @@ public class CoordinatorRequestManagerTest {
|
||||||
private static final int RETRY_BACKOFF_MS = 500;
|
private static final int RETRY_BACKOFF_MS = 500;
|
||||||
private static final String GROUP_ID = "group-1";
|
private static final String GROUP_ID = "group-1";
|
||||||
private MockTime time;
|
private MockTime time;
|
||||||
private ErrorEventHandler errorEventHandler;
|
private BackgroundEventHandler backgroundEventHandler;
|
||||||
private Node node;
|
private Node node;
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
public void setup() {
|
public void setup() {
|
||||||
this.time = new MockTime(0);
|
this.time = new MockTime(0);
|
||||||
this.node = new Node(1, "localhost", 9092);
|
this.node = new Node(1, "localhost", 9092);
|
||||||
this.errorEventHandler = mock(ErrorEventHandler.class);
|
this.backgroundEventHandler = mock(BackgroundEventHandler.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -97,7 +99,7 @@ public class CoordinatorRequestManagerTest {
|
||||||
public void testBackoffAfterRetriableFailure() {
|
public void testBackoffAfterRetriableFailure() {
|
||||||
CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID);
|
CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID);
|
||||||
expectFindCoordinatorRequest(coordinatorManager, Errors.COORDINATOR_LOAD_IN_PROGRESS);
|
expectFindCoordinatorRequest(coordinatorManager, Errors.COORDINATOR_LOAD_IN_PROGRESS);
|
||||||
verifyNoInteractions(errorEventHandler);
|
verifyNoInteractions(backgroundEventHandler);
|
||||||
|
|
||||||
time.sleep(RETRY_BACKOFF_MS - 1);
|
time.sleep(RETRY_BACKOFF_MS - 1);
|
||||||
assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests);
|
assertEquals(Collections.emptyList(), coordinatorManager.poll(time.milliseconds()).unsentRequests);
|
||||||
|
@ -111,10 +113,15 @@ public class CoordinatorRequestManagerTest {
|
||||||
CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID);
|
CoordinatorRequestManager coordinatorManager = setupCoordinatorManager(GROUP_ID);
|
||||||
expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED);
|
expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED);
|
||||||
|
|
||||||
verify(errorEventHandler).handle(argThat(exception -> {
|
verify(backgroundEventHandler).add(argThat(backgroundEvent -> {
|
||||||
if (!(exception instanceof GroupAuthorizationException)) {
|
if (!(backgroundEvent instanceof ErrorBackgroundEvent))
|
||||||
return false;
|
return false;
|
||||||
}
|
|
||||||
|
RuntimeException exception = ((ErrorBackgroundEvent) backgroundEvent).error();
|
||||||
|
|
||||||
|
if (!(exception instanceof GroupAuthorizationException))
|
||||||
|
return false;
|
||||||
|
|
||||||
GroupAuthorizationException groupAuthException = (GroupAuthorizationException) exception;
|
GroupAuthorizationException groupAuthException = (GroupAuthorizationException) exception;
|
||||||
return groupAuthException.groupId().equals(GROUP_ID);
|
return groupAuthException.groupId().equals(GROUP_ID);
|
||||||
}));
|
}));
|
||||||
|
@ -185,7 +192,7 @@ public class CoordinatorRequestManagerTest {
|
||||||
new LogContext(),
|
new LogContext(),
|
||||||
RETRY_BACKOFF_MS,
|
RETRY_BACKOFF_MS,
|
||||||
RETRY_BACKOFF_MS,
|
RETRY_BACKOFF_MS,
|
||||||
this.errorEventHandler,
|
this.backgroundEventHandler,
|
||||||
groupId
|
groupId
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,436 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
|
||||||
|
|
||||||
import org.apache.kafka.clients.GroupRebalanceConfig;
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
|
||||||
import org.apache.kafka.clients.consumer.LogTruncationException;
|
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.NoopApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent;
|
|
||||||
import org.apache.kafka.common.TopicPartition;
|
|
||||||
import org.apache.kafka.common.errors.TopicAuthorizationException;
|
|
||||||
import org.apache.kafka.common.message.FindCoordinatorRequestData;
|
|
||||||
import org.apache.kafka.common.requests.FindCoordinatorRequest;
|
|
||||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
|
||||||
import org.apache.kafka.common.utils.Time;
|
|
||||||
import org.apache.kafka.test.TestUtils;
|
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
|
||||||
import org.junit.jupiter.api.Test;
|
|
||||||
import org.mockito.Mockito;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Properties;
|
|
||||||
import java.util.concurrent.BlockingQueue;
|
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
|
||||||
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.RETRY_BACKOFF_MS_CONFIG;
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
|
||||||
import static org.mockito.ArgumentMatchers.anyLong;
|
|
||||||
import static org.mockito.Mockito.doThrow;
|
|
||||||
import static org.mockito.Mockito.mock;
|
|
||||||
import static org.mockito.Mockito.spy;
|
|
||||||
import static org.mockito.Mockito.times;
|
|
||||||
import static org.mockito.Mockito.verify;
|
|
||||||
import static org.mockito.Mockito.when;
|
|
||||||
|
|
||||||
@SuppressWarnings("ClassDataAbstractionCoupling")
|
|
||||||
public class DefaultBackgroundThreadTest {
|
|
||||||
private static final long RETRY_BACKOFF_MS = 100;
|
|
||||||
private final Properties properties = new Properties();
|
|
||||||
private MockTime time;
|
|
||||||
private ConsumerMetadata metadata;
|
|
||||||
private NetworkClientDelegate networkClient;
|
|
||||||
private BlockingQueue<BackgroundEvent> backgroundEventsQueue;
|
|
||||||
private BlockingQueue<ApplicationEvent> applicationEventsQueue;
|
|
||||||
private ApplicationEventProcessor applicationEventProcessor;
|
|
||||||
private CoordinatorRequestManager coordinatorManager;
|
|
||||||
private OffsetsRequestManager offsetsRequestManager;
|
|
||||||
private ErrorEventHandler errorEventHandler;
|
|
||||||
private final int requestTimeoutMs = 500;
|
|
||||||
private GroupState groupState;
|
|
||||||
private CommitRequestManager commitManager;
|
|
||||||
private TopicMetadataRequestManager topicMetadataRequestManager;
|
|
||||||
private HeartbeatRequestManager heartbeatRequestManager;
|
|
||||||
|
|
||||||
@BeforeEach
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
public void setup() {
|
|
||||||
this.time = new MockTime(0);
|
|
||||||
this.metadata = mock(ConsumerMetadata.class);
|
|
||||||
this.networkClient = mock(NetworkClientDelegate.class);
|
|
||||||
this.applicationEventsQueue = (BlockingQueue<ApplicationEvent>) mock(BlockingQueue.class);
|
|
||||||
this.backgroundEventsQueue = (BlockingQueue<BackgroundEvent>) mock(BlockingQueue.class);
|
|
||||||
this.applicationEventProcessor = mock(ApplicationEventProcessor.class);
|
|
||||||
this.coordinatorManager = mock(CoordinatorRequestManager.class);
|
|
||||||
this.offsetsRequestManager = mock(OffsetsRequestManager.class);
|
|
||||||
this.heartbeatRequestManager = mock(HeartbeatRequestManager.class);
|
|
||||||
this.errorEventHandler = mock(ErrorEventHandler.class);
|
|
||||||
GroupRebalanceConfig rebalanceConfig = new GroupRebalanceConfig(
|
|
||||||
100,
|
|
||||||
100,
|
|
||||||
100,
|
|
||||||
"group_id",
|
|
||||||
Optional.empty(),
|
|
||||||
100,
|
|
||||||
1000,
|
|
||||||
true);
|
|
||||||
this.groupState = new GroupState(rebalanceConfig);
|
|
||||||
this.commitManager = mock(CommitRequestManager.class);
|
|
||||||
this.topicMetadataRequestManager = mock(TopicMetadataRequestManager.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testStartupAndTearDown() throws InterruptedException {
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
backgroundThread.start();
|
|
||||||
TestUtils.waitForCondition(backgroundThread::isRunning, "Failed awaiting for the background thread to be running");
|
|
||||||
backgroundThread.close();
|
|
||||||
assertFalse(backgroundThread.isRunning());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testApplicationEvent() {
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
ApplicationEvent e = new NoopApplicationEvent("noop event");
|
|
||||||
this.applicationEventsQueue.add(e);
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
verify(applicationEventProcessor, times(1)).process(e);
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testMetadataUpdateEvent() {
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.applicationEventProcessor = new ApplicationEventProcessor(
|
|
||||||
this.backgroundEventsQueue,
|
|
||||||
mockRequestManagers(),
|
|
||||||
metadata);
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
ApplicationEvent e = new NewTopicsMetadataUpdateRequestEvent();
|
|
||||||
this.applicationEventsQueue.add(e);
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
verify(metadata).requestUpdateForNewTopics();
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCommitEvent() {
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
ApplicationEvent e = new CommitApplicationEvent(new HashMap<>());
|
|
||||||
this.applicationEventsQueue.add(e);
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
verify(applicationEventProcessor).process(any(CommitApplicationEvent.class));
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testListOffsetsEventIsProcessed() {
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
Map<TopicPartition, Long> timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L);
|
|
||||||
ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true);
|
|
||||||
this.applicationEventsQueue.add(e);
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class));
|
|
||||||
assertTrue(applicationEventsQueue.isEmpty());
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testResetPositionsEventIsProcessed() {
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent();
|
|
||||||
this.applicationEventsQueue.add(e);
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class));
|
|
||||||
assertTrue(applicationEventsQueue.isEmpty());
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testResetPositionsProcessFailure() {
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
applicationEventProcessor = spy(new ApplicationEventProcessor(
|
|
||||||
this.backgroundEventsQueue,
|
|
||||||
mockRequestManagers(),
|
|
||||||
metadata));
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
|
|
||||||
TopicAuthorizationException authException = new TopicAuthorizationException("Topic authorization failed");
|
|
||||||
doThrow(authException).when(offsetsRequestManager).resetPositionsIfNeeded();
|
|
||||||
|
|
||||||
ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent();
|
|
||||||
this.applicationEventsQueue.add(event);
|
|
||||||
assertThrows(TopicAuthorizationException.class, backgroundThread::runOnce);
|
|
||||||
|
|
||||||
verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class));
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testValidatePositionsEventIsProcessed() {
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent();
|
|
||||||
this.applicationEventsQueue.add(e);
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class));
|
|
||||||
assertTrue(applicationEventsQueue.isEmpty());
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testValidatePositionsProcessFailure() {
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
applicationEventProcessor = spy(new ApplicationEventProcessor(
|
|
||||||
this.backgroundEventsQueue,
|
|
||||||
mockRequestManagers(),
|
|
||||||
metadata));
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
|
|
||||||
LogTruncationException logTruncationException = new LogTruncationException(Collections.emptyMap(), Collections.emptyMap());
|
|
||||||
doThrow(logTruncationException).when(offsetsRequestManager).validatePositionsIfNeeded();
|
|
||||||
|
|
||||||
ValidatePositionsApplicationEvent event = new ValidatePositionsApplicationEvent();
|
|
||||||
this.applicationEventsQueue.add(event);
|
|
||||||
assertThrows(LogTruncationException.class, backgroundThread::runOnce);
|
|
||||||
|
|
||||||
verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class));
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testAssignmentChangeEvent() {
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.backgroundEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
this.applicationEventProcessor = spy(new ApplicationEventProcessor(
|
|
||||||
this.backgroundEventsQueue,
|
|
||||||
mockRequestManagers(),
|
|
||||||
metadata));
|
|
||||||
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
HashMap<TopicPartition, OffsetAndMetadata> offset = mockTopicPartitionOffset();
|
|
||||||
|
|
||||||
final long currentTimeMs = time.milliseconds();
|
|
||||||
ApplicationEvent e = new AssignmentChangeApplicationEvent(offset, currentTimeMs);
|
|
||||||
this.applicationEventsQueue.add(e);
|
|
||||||
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
verify(applicationEventProcessor).process(any(AssignmentChangeApplicationEvent.class));
|
|
||||||
verify(networkClient, times(1)).poll(anyLong(), anyLong());
|
|
||||||
verify(commitManager, times(1)).updateAutoCommitTimer(currentTimeMs);
|
|
||||||
verify(commitManager, times(1)).maybeAutoCommit(offset);
|
|
||||||
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void testFindCoordinator() {
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
Mockito.verify(coordinatorManager, times(1)).poll(anyLong());
|
|
||||||
Mockito.verify(networkClient, times(1)).poll(anyLong(), anyLong());
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void testFetchTopicMetadata() {
|
|
||||||
this.applicationEventsQueue = new LinkedBlockingQueue<>();
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
when(coordinatorManager.poll(anyLong())).thenReturn(mockPollCoordinatorResult());
|
|
||||||
when(commitManager.poll(anyLong())).thenReturn(mockPollCommitResult());
|
|
||||||
when(offsetsRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(topicMetadataRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
when(heartbeatRequestManager.poll(anyLong())).thenReturn(emptyPollResults());
|
|
||||||
this.applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic"));
|
|
||||||
backgroundThread.runOnce();
|
|
||||||
verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class));
|
|
||||||
backgroundThread.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
void testPollResultTimer() {
|
|
||||||
DefaultBackgroundThread backgroundThread = mockBackgroundThread();
|
|
||||||
// purposely setting a non MAX time to ensure it is returning Long.MAX_VALUE upon success
|
|
||||||
NetworkClientDelegate.PollResult success = new NetworkClientDelegate.PollResult(
|
|
||||||
10,
|
|
||||||
Collections.singletonList(findCoordinatorUnsentRequest(time, requestTimeoutMs)));
|
|
||||||
assertEquals(10, backgroundThread.handlePollResult(success));
|
|
||||||
|
|
||||||
NetworkClientDelegate.PollResult failure = new NetworkClientDelegate.PollResult(
|
|
||||||
10,
|
|
||||||
new ArrayList<>());
|
|
||||||
assertEquals(10, backgroundThread.handlePollResult(failure));
|
|
||||||
}
|
|
||||||
|
|
||||||
private HashMap<TopicPartition, OffsetAndMetadata> mockTopicPartitionOffset() {
|
|
||||||
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));
|
|
||||||
topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L));
|
|
||||||
return topicPartitionOffsets;
|
|
||||||
}
|
|
||||||
|
|
||||||
private RequestManagers mockRequestManagers() {
|
|
||||||
return new RequestManagers(
|
|
||||||
offsetsRequestManager,
|
|
||||||
topicMetadataRequestManager,
|
|
||||||
Optional.of(coordinatorManager),
|
|
||||||
Optional.of(commitManager),
|
|
||||||
Optional.of(heartbeatRequestManager));
|
|
||||||
}
|
|
||||||
|
|
||||||
private static NetworkClientDelegate.UnsentRequest findCoordinatorUnsentRequest(
|
|
||||||
final Time time,
|
|
||||||
final long timeout
|
|
||||||
) {
|
|
||||||
NetworkClientDelegate.UnsentRequest req = new NetworkClientDelegate.UnsentRequest(
|
|
||||||
new FindCoordinatorRequest.Builder(
|
|
||||||
new FindCoordinatorRequestData()
|
|
||||||
.setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id())
|
|
||||||
.setKey("foobar")),
|
|
||||||
Optional.empty());
|
|
||||||
req.setTimer(time, timeout);
|
|
||||||
return req;
|
|
||||||
}
|
|
||||||
|
|
||||||
private DefaultBackgroundThread mockBackgroundThread() {
|
|
||||||
properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
|
||||||
properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
|
||||||
properties.put(RETRY_BACKOFF_MS_CONFIG, RETRY_BACKOFF_MS);
|
|
||||||
|
|
||||||
return new DefaultBackgroundThread(
|
|
||||||
this.time,
|
|
||||||
new ConsumerConfig(properties),
|
|
||||||
new LogContext(),
|
|
||||||
applicationEventsQueue,
|
|
||||||
backgroundEventsQueue,
|
|
||||||
this.errorEventHandler,
|
|
||||||
applicationEventProcessor,
|
|
||||||
this.metadata,
|
|
||||||
this.networkClient,
|
|
||||||
this.groupState,
|
|
||||||
this.coordinatorManager,
|
|
||||||
this.commitManager,
|
|
||||||
this.offsetsRequestManager,
|
|
||||||
this.topicMetadataRequestManager,
|
|
||||||
this.heartbeatRequestManager);
|
|
||||||
}
|
|
||||||
|
|
||||||
private NetworkClientDelegate.PollResult mockPollCoordinatorResult() {
|
|
||||||
return new NetworkClientDelegate.PollResult(
|
|
||||||
RETRY_BACKOFF_MS,
|
|
||||||
Collections.singletonList(findCoordinatorUnsentRequest(time, requestTimeoutMs)));
|
|
||||||
}
|
|
||||||
|
|
||||||
private NetworkClientDelegate.PollResult mockPollCommitResult() {
|
|
||||||
return new NetworkClientDelegate.PollResult(
|
|
||||||
RETRY_BACKOFF_MS,
|
|
||||||
Collections.singletonList(findCoordinatorUnsentRequest(time, requestTimeoutMs)));
|
|
||||||
}
|
|
||||||
|
|
||||||
private NetworkClientDelegate.PollResult emptyPollResults() {
|
|
||||||
return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, Collections.emptyList());
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,81 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
|
||||||
|
|
||||||
import org.apache.kafka.clients.GroupRebalanceConfig;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.NoopApplicationEvent;
|
|
||||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
|
||||||
import org.junit.jupiter.api.Test;
|
|
||||||
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Properties;
|
|
||||||
import java.util.concurrent.BlockingQueue;
|
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
|
||||||
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.RETRY_BACKOFF_MS_CONFIG;
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
|
||||||
import static org.mockito.Mockito.mock;
|
|
||||||
import static org.mockito.Mockito.times;
|
|
||||||
import static org.mockito.Mockito.verify;
|
|
||||||
|
|
||||||
public class DefaultEventHandlerTest {
|
|
||||||
private int sessionTimeoutMs = 1000;
|
|
||||||
private int rebalanceTimeoutMs = 1000;
|
|
||||||
private int heartbeatIntervalMs = 1000;
|
|
||||||
private String groupId = "g-1";
|
|
||||||
private Optional<String> groupInstanceId = Optional.of("g-1");
|
|
||||||
private long retryBackoffMs = 1000;
|
|
||||||
private final Properties properties = new Properties();
|
|
||||||
private GroupRebalanceConfig rebalanceConfig;
|
|
||||||
|
|
||||||
@BeforeEach
|
|
||||||
public void setup() {
|
|
||||||
properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
|
||||||
properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
|
||||||
properties.put(RETRY_BACKOFF_MS_CONFIG, "100");
|
|
||||||
|
|
||||||
this.rebalanceConfig = new GroupRebalanceConfig(sessionTimeoutMs,
|
|
||||||
rebalanceTimeoutMs,
|
|
||||||
heartbeatIntervalMs,
|
|
||||||
groupId,
|
|
||||||
groupInstanceId,
|
|
||||||
retryBackoffMs,
|
|
||||||
retryBackoffMs,
|
|
||||||
true);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testBasicHandlerOps() {
|
|
||||||
final DefaultBackgroundThread bt = mock(DefaultBackgroundThread.class);
|
|
||||||
final BlockingQueue<ApplicationEvent> aq = new LinkedBlockingQueue<>();
|
|
||||||
final BlockingQueue<BackgroundEvent> bq = new LinkedBlockingQueue<>();
|
|
||||||
final DefaultEventHandler handler = new DefaultEventHandler(bt, aq, bq);
|
|
||||||
assertTrue(handler.isEmpty());
|
|
||||||
assertFalse(handler.poll().isPresent());
|
|
||||||
handler.add(new NoopApplicationEvent("test"));
|
|
||||||
assertEquals(1, aq.size());
|
|
||||||
handler.close();
|
|
||||||
verify(bt, times(1)).close();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -51,7 +51,6 @@ import java.util.Properties;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchConfig;
|
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics;
|
||||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState;
|
||||||
|
@ -79,10 +78,11 @@ public class FetchCollectorTest {
|
||||||
private LogContext logContext;
|
private LogContext logContext;
|
||||||
|
|
||||||
private SubscriptionState subscriptions;
|
private SubscriptionState subscriptions;
|
||||||
private FetchConfig<String, String> fetchConfig;
|
private FetchConfig fetchConfig;
|
||||||
private FetchMetricsManager metricsManager;
|
private FetchMetricsManager metricsManager;
|
||||||
private ConsumerMetadata metadata;
|
private ConsumerMetadata metadata;
|
||||||
private FetchBuffer fetchBuffer;
|
private FetchBuffer fetchBuffer;
|
||||||
|
private Deserializers<String, String> deserializers;
|
||||||
private FetchCollector<String, String> fetchCollector;
|
private FetchCollector<String, String> fetchCollector;
|
||||||
private CompletedFetchBuilder completedFetchBuilder;
|
private CompletedFetchBuilder completedFetchBuilder;
|
||||||
|
|
||||||
|
@ -193,6 +193,7 @@ public class FetchCollectorTest {
|
||||||
metadata,
|
metadata,
|
||||||
subscriptions,
|
subscriptions,
|
||||||
fetchConfig,
|
fetchConfig,
|
||||||
|
deserializers,
|
||||||
metricsManager,
|
metricsManager,
|
||||||
time) {
|
time) {
|
||||||
|
|
||||||
|
@ -427,10 +428,10 @@ public class FetchCollectorTest {
|
||||||
|
|
||||||
ConsumerConfig config = new ConsumerConfig(p);
|
ConsumerConfig config = new ConsumerConfig(p);
|
||||||
|
|
||||||
Deserializers<String, String> deserializers = new Deserializers<>(new StringDeserializer(), new StringDeserializer());
|
deserializers = new Deserializers<>(new StringDeserializer(), new StringDeserializer());
|
||||||
|
|
||||||
subscriptions = createSubscriptionState(config, logContext);
|
subscriptions = createSubscriptionState(config, logContext);
|
||||||
fetchConfig = createFetchConfig(config, deserializers);
|
fetchConfig = new FetchConfig(config);
|
||||||
|
|
||||||
Metrics metrics = createMetrics(config, time);
|
Metrics metrics = createMetrics(config, time);
|
||||||
metricsManager = createFetchMetricsManager(metrics);
|
metricsManager = createFetchMetricsManager(metrics);
|
||||||
|
@ -448,6 +449,7 @@ public class FetchCollectorTest {
|
||||||
metadata,
|
metadata,
|
||||||
subscriptions,
|
subscriptions,
|
||||||
fetchConfig,
|
fetchConfig,
|
||||||
|
deserializers,
|
||||||
metricsManager,
|
metricsManager,
|
||||||
time);
|
time);
|
||||||
fetchBuffer = new FetchBuffer(logContext);
|
fetchBuffer = new FetchBuffer(logContext);
|
||||||
|
|
|
@ -19,14 +19,11 @@ package org.apache.kafka.clients.consumer.internals;
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||||
import org.apache.kafka.common.IsolationLevel;
|
import org.apache.kafka.common.IsolationLevel;
|
||||||
import org.apache.kafka.common.serialization.Deserializer;
|
import org.apache.kafka.common.serialization.Deserializer;
|
||||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
|
||||||
import org.apache.kafka.common.serialization.StringSerializer;
|
import org.apache.kafka.common.serialization.StringSerializer;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
|
||||||
|
|
||||||
public class FetchConfigTest {
|
public class FetchConfigTest {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -35,60 +32,27 @@ public class FetchConfigTest {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testBasicFromConsumerConfig() {
|
public void testBasicFromConsumerConfig() {
|
||||||
try (StringDeserializer keyDeserializer = new StringDeserializer(); StringDeserializer valueDeserializer = new StringDeserializer()) {
|
newFetchConfigFromConsumerConfig();
|
||||||
newFetchConfigFromConsumerConfig(keyDeserializer, valueDeserializer);
|
newFetchConfigFromValues();
|
||||||
newFetchConfigFromValues(keyDeserializer, valueDeserializer);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
private void newFetchConfigFromConsumerConfig() {
|
||||||
* Verify an exception is thrown if the key {@link Deserializer deserializer} provided to the
|
|
||||||
* {@link FetchConfig} constructors is {@code null}.
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
public void testPreventNullKeyDeserializer() {
|
|
||||||
try (StringDeserializer valueDeserializer = new StringDeserializer()) {
|
|
||||||
assertThrows(NullPointerException.class, () -> newFetchConfigFromConsumerConfig(null, valueDeserializer));
|
|
||||||
assertThrows(NullPointerException.class, () -> newFetchConfigFromValues(null, valueDeserializer));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Verify an exception is thrown if the value {@link Deserializer deserializer} provided to the
|
|
||||||
* {@link FetchConfig} constructors is {@code null}.
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
@SuppressWarnings("resources")
|
|
||||||
public void testPreventNullValueDeserializer() {
|
|
||||||
try (StringDeserializer keyDeserializer = new StringDeserializer()) {
|
|
||||||
assertThrows(NullPointerException.class, () -> newFetchConfigFromConsumerConfig(keyDeserializer, null));
|
|
||||||
assertThrows(NullPointerException.class, () -> newFetchConfigFromValues(keyDeserializer, null));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void newFetchConfigFromConsumerConfig(Deserializer<String> keyDeserializer,
|
|
||||||
Deserializer<String> valueDeserializer) {
|
|
||||||
Properties p = new Properties();
|
Properties p = new Properties();
|
||||||
p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
||||||
p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
|
p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
|
||||||
p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
|
p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
|
||||||
ConsumerConfig config = new ConsumerConfig(p);
|
ConsumerConfig config = new ConsumerConfig(p);
|
||||||
new FetchConfig<>(
|
new FetchConfig(config);
|
||||||
config,
|
|
||||||
new Deserializers<>(keyDeserializer, valueDeserializer),
|
|
||||||
IsolationLevel.READ_UNCOMMITTED);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void newFetchConfigFromValues(Deserializer<String> keyDeserializer,
|
private void newFetchConfigFromValues() {
|
||||||
Deserializer<String> valueDeserializer) {
|
new FetchConfig(ConsumerConfig.DEFAULT_FETCH_MIN_BYTES,
|
||||||
new FetchConfig<>(ConsumerConfig.DEFAULT_FETCH_MIN_BYTES,
|
|
||||||
ConsumerConfig.DEFAULT_FETCH_MAX_BYTES,
|
ConsumerConfig.DEFAULT_FETCH_MAX_BYTES,
|
||||||
ConsumerConfig.DEFAULT_FETCH_MAX_WAIT_MS,
|
ConsumerConfig.DEFAULT_FETCH_MAX_WAIT_MS,
|
||||||
ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES,
|
ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES,
|
||||||
ConsumerConfig.DEFAULT_MAX_POLL_RECORDS,
|
ConsumerConfig.DEFAULT_MAX_POLL_RECORDS,
|
||||||
true,
|
true,
|
||||||
ConsumerConfig.DEFAULT_CLIENT_RACK,
|
ConsumerConfig.DEFAULT_CLIENT_RACK,
|
||||||
new Deserializers<>(keyDeserializer, valueDeserializer),
|
|
||||||
IsolationLevel.READ_UNCOMMITTED);
|
IsolationLevel.READ_UNCOMMITTED);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -17,8 +17,7 @@
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.clients.ClientResponse;
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
|
||||||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.KafkaException;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.errors.TimeoutException;
|
import org.apache.kafka.common.errors.TimeoutException;
|
||||||
|
@ -28,11 +27,9 @@ import org.apache.kafka.common.protocol.Errors;
|
||||||
import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest;
|
import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest;
|
||||||
import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse;
|
import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse;
|
||||||
import org.apache.kafka.common.requests.RequestHeader;
|
import org.apache.kafka.common.requests.RequestHeader;
|
||||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
|
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
|
||||||
|
import org.junit.jupiter.api.AfterEach;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
|
@ -46,80 +43,77 @@ import java.util.Collections;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Properties;
|
|
||||||
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_ID;
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_INSTANCE_ID;
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_HEARTBEAT_INTERVAL_MS;
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.RETRY_BACKOFF_MS_CONFIG;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_MAX_POLL_INTERVAL_MS;
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_RETRY_BACKOFF_MS;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
import static org.mockito.ArgumentMatchers.anyLong;
|
import static org.mockito.ArgumentMatchers.anyLong;
|
||||||
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.times;
|
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;
|
||||||
|
|
||||||
public class HeartbeatRequestManagerTest {
|
public class HeartbeatRequestManagerTest {
|
||||||
|
|
||||||
private static final int HEARTBEAT_INTERVAL_MS = 1000;
|
private ConsumerTestBuilder testBuilder;
|
||||||
private static final long RETRY_BACKOFF_MAX_MS = 3000;
|
|
||||||
private static final long RETRY_BACKOFF_MS = 100;
|
|
||||||
private static final String GROUP_INSTANCE_ID = "group-instance-id";
|
|
||||||
private static final String GROUP_ID = "group-id";
|
|
||||||
|
|
||||||
private Time time;
|
private Time time;
|
||||||
private LogContext logContext;
|
|
||||||
private CoordinatorRequestManager coordinatorRequestManager;
|
private CoordinatorRequestManager coordinatorRequestManager;
|
||||||
private SubscriptionState subscriptionState;
|
private SubscriptionState subscriptions;
|
||||||
private HeartbeatRequestManager heartbeatRequestManager;
|
private HeartbeatRequestManager heartbeatRequestManager;
|
||||||
private MembershipManager membershipManager;
|
private MembershipManager membershipManager;
|
||||||
private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState;
|
private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState;
|
||||||
private ConsumerConfig config;
|
private final String memberId = "member-id";
|
||||||
|
private final int memberEpoch = 1;
|
||||||
private String memberId = "member-id";
|
private BackgroundEventHandler backgroundEventHandler;
|
||||||
private int memberEpoch = 1;
|
|
||||||
private ErrorEventHandler errorEventHandler;
|
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
time = new MockTime();
|
setUp(ConsumerTestBuilder.createDefaultGroupInformation());
|
||||||
logContext = new LogContext();
|
|
||||||
config = new ConsumerConfig(createConsumerConfig());
|
|
||||||
coordinatorRequestManager = mock(CoordinatorRequestManager.class);
|
|
||||||
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999)));
|
|
||||||
subscriptionState = mock(SubscriptionState.class);
|
|
||||||
membershipManager = spy(new MembershipManagerImpl(GROUP_ID, logContext));
|
|
||||||
heartbeatRequestState = mock(HeartbeatRequestManager.HeartbeatRequestState.class);
|
|
||||||
errorEventHandler = mock(ErrorEventHandler.class);
|
|
||||||
heartbeatRequestManager = createManager();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private Properties createConsumerConfig() {
|
private void setUp(Optional<ConsumerTestBuilder.GroupInformation> groupInfo) {
|
||||||
Properties properties = new Properties();
|
testBuilder = new ConsumerTestBuilder(groupInfo);
|
||||||
properties.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
|
time = testBuilder.time;
|
||||||
properties.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
coordinatorRequestManager = testBuilder.coordinatorRequestManager.orElseThrow(IllegalStateException::new);
|
||||||
properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
heartbeatRequestManager = testBuilder.heartbeatRequestManager.orElseThrow(IllegalStateException::new);
|
||||||
properties.put(RETRY_BACKOFF_MS_CONFIG, "100");
|
heartbeatRequestState = testBuilder.heartbeatRequestState.orElseThrow(IllegalStateException::new);
|
||||||
return properties;
|
backgroundEventHandler = testBuilder.backgroundEventHandler;
|
||||||
|
subscriptions = testBuilder.subscriptions;
|
||||||
|
membershipManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new);
|
||||||
|
|
||||||
|
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999)));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void resetWithZeroHeartbeatInterval(Optional<String> groupInstanceId) {
|
||||||
|
cleanup();
|
||||||
|
|
||||||
|
ConsumerTestBuilder.GroupInformation gi = new ConsumerTestBuilder.GroupInformation(
|
||||||
|
new GroupState(DEFAULT_GROUP_ID, groupInstanceId),
|
||||||
|
0,
|
||||||
|
0.0
|
||||||
|
);
|
||||||
|
|
||||||
|
setUp(Optional.of(gi));
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterEach
|
||||||
|
public void cleanup() {
|
||||||
|
if (testBuilder != null) {
|
||||||
|
testBuilder.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testHeartbeatOnStartup() {
|
public void testHeartbeatOnStartup() {
|
||||||
// The initial heartbeatInterval is set to 0
|
// The initial heartbeatInterval is set to 0
|
||||||
heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState(
|
resetWithZeroHeartbeatInterval(Optional.empty());
|
||||||
logContext,
|
|
||||||
time,
|
|
||||||
0,
|
|
||||||
RETRY_BACKOFF_MS,
|
|
||||||
RETRY_BACKOFF_MAX_MS,
|
|
||||||
0);
|
|
||||||
heartbeatRequestManager = createManager();
|
|
||||||
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(1, result.unsentRequests.size());
|
assertEquals(1, result.unsentRequests.size());
|
||||||
|
|
||||||
|
@ -131,6 +125,9 @@ public class HeartbeatRequestManagerTest {
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@ValueSource(booleans = {true, false})
|
@ValueSource(booleans = {true, false})
|
||||||
public void testSendHeartbeatOnMemberState(final boolean shouldSendHeartbeat) {
|
public void testSendHeartbeatOnMemberState(final boolean shouldSendHeartbeat) {
|
||||||
|
// The initial heartbeatInterval is set to 0
|
||||||
|
resetWithZeroHeartbeatInterval(Optional.empty());
|
||||||
|
|
||||||
// Mocking notInGroup
|
// Mocking notInGroup
|
||||||
when(membershipManager.shouldSendHeartbeat()).thenReturn(shouldSendHeartbeat);
|
when(membershipManager.shouldSendHeartbeat()).thenReturn(shouldSendHeartbeat);
|
||||||
when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true);
|
when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true);
|
||||||
|
@ -150,21 +147,13 @@ public class HeartbeatRequestManagerTest {
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("stateProvider")
|
@MethodSource("stateProvider")
|
||||||
public void testTimerNotDue(final MemberState state) {
|
public void testTimerNotDue(final MemberState state) {
|
||||||
heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState(
|
|
||||||
logContext,
|
|
||||||
time,
|
|
||||||
HEARTBEAT_INTERVAL_MS,
|
|
||||||
RETRY_BACKOFF_MS,
|
|
||||||
RETRY_BACKOFF_MAX_MS);
|
|
||||||
heartbeatRequestManager = createManager();
|
|
||||||
|
|
||||||
when(membershipManager.state()).thenReturn(state);
|
when(membershipManager.state()).thenReturn(state);
|
||||||
time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent
|
time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent
|
||||||
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(0, result.unsentRequests.size());
|
assertEquals(0, result.unsentRequests.size());
|
||||||
|
|
||||||
if (membershipManager.shouldSendHeartbeat()) {
|
if (membershipManager.shouldSendHeartbeat()) {
|
||||||
assertEquals(HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs);
|
assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs);
|
||||||
} else {
|
} else {
|
||||||
assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs);
|
assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs);
|
||||||
}
|
}
|
||||||
|
@ -172,14 +161,8 @@ public class HeartbeatRequestManagerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNetworkTimeout() {
|
public void testNetworkTimeout() {
|
||||||
heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState(
|
// The initial heartbeatInterval is set to 0
|
||||||
logContext,
|
resetWithZeroHeartbeatInterval(Optional.empty());
|
||||||
time,
|
|
||||||
0,
|
|
||||||
RETRY_BACKOFF_MS,
|
|
||||||
RETRY_BACKOFF_MAX_MS,
|
|
||||||
0);
|
|
||||||
heartbeatRequestManager = createManager();
|
|
||||||
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999)));
|
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999)));
|
||||||
when(membershipManager.shouldSendHeartbeat()).thenReturn(true);
|
when(membershipManager.shouldSendHeartbeat()).thenReturn(true);
|
||||||
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
||||||
|
@ -188,7 +171,7 @@ public class HeartbeatRequestManagerTest {
|
||||||
result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException("timeout"));
|
result.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException("timeout"));
|
||||||
|
|
||||||
// Assure the manager will backoff on timeout
|
// Assure the manager will backoff on timeout
|
||||||
time.sleep(RETRY_BACKOFF_MS - 1);
|
time.sleep(DEFAULT_RETRY_BACKOFF_MS - 1);
|
||||||
result = heartbeatRequestManager.poll(time.milliseconds());
|
result = heartbeatRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(0, result.unsentRequests.size());
|
assertEquals(0, result.unsentRequests.size());
|
||||||
|
|
||||||
|
@ -199,21 +182,16 @@ public class HeartbeatRequestManagerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFailureOnFatalException() {
|
public void testFailureOnFatalException() {
|
||||||
heartbeatRequestState = spy(new HeartbeatRequestManager.HeartbeatRequestState(
|
// The initial heartbeatInterval is set to 0
|
||||||
logContext,
|
resetWithZeroHeartbeatInterval(Optional.empty());
|
||||||
time,
|
|
||||||
0,
|
|
||||||
RETRY_BACKOFF_MS,
|
|
||||||
RETRY_BACKOFF_MAX_MS,
|
|
||||||
0));
|
|
||||||
heartbeatRequestManager = createManager();
|
|
||||||
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999)));
|
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999)));
|
||||||
when(membershipManager.shouldSendHeartbeat()).thenReturn(true);
|
when(membershipManager.shouldSendHeartbeat()).thenReturn(true);
|
||||||
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(1, result.unsentRequests.size());
|
assertEquals(1, result.unsentRequests.size());
|
||||||
result.unsentRequests.get(0).future().completeExceptionally(new KafkaException("fatal"));
|
result.unsentRequests.get(0).future().completeExceptionally(new KafkaException("fatal"));
|
||||||
verify(membershipManager).transitionToFailed();
|
verify(membershipManager).transitionToFailed();
|
||||||
verify(errorEventHandler).handle(any());
|
verify(backgroundEventHandler).add(any());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -228,22 +206,12 @@ public class HeartbeatRequestManagerTest {
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT)
|
@ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT)
|
||||||
public void testValidateConsumerGroupHeartbeatRequest(final short version) {
|
public void testValidateConsumerGroupHeartbeatRequest(final short version) {
|
||||||
List<String> subscribedTopics = Collections.singletonList("topic");
|
// The initial heartbeatInterval is set to 0, but we're testing
|
||||||
subscriptionState = new SubscriptionState(logContext, OffsetResetStrategy.NONE);
|
resetWithZeroHeartbeatInterval(Optional.of(DEFAULT_GROUP_INSTANCE_ID));
|
||||||
subscriptionState.subscribe(new HashSet<>(subscribedTopics), new NoOpConsumerRebalanceListener());
|
|
||||||
|
List<String> subscribedTopics = Collections.singletonList("topic");
|
||||||
|
subscriptions.subscribe(new HashSet<>(subscribedTopics), new NoOpConsumerRebalanceListener());
|
||||||
|
|
||||||
Properties prop = createConsumerConfig();
|
|
||||||
prop.setProperty(MAX_POLL_INTERVAL_MS_CONFIG, "10000");
|
|
||||||
config = new ConsumerConfig(prop);
|
|
||||||
membershipManager = new MembershipManagerImpl(GROUP_ID, GROUP_INSTANCE_ID, null, logContext);
|
|
||||||
heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState(
|
|
||||||
logContext,
|
|
||||||
time,
|
|
||||||
0,
|
|
||||||
RETRY_BACKOFF_MS,
|
|
||||||
RETRY_BACKOFF_MAX_MS,
|
|
||||||
0);
|
|
||||||
heartbeatRequestManager = createManager();
|
|
||||||
// Update membershipManager's memberId and memberEpoch
|
// Update membershipManager's memberId and memberEpoch
|
||||||
ConsumerGroupHeartbeatResponse result =
|
ConsumerGroupHeartbeatResponse result =
|
||||||
new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData()
|
new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData()
|
||||||
|
@ -259,12 +227,12 @@ public class HeartbeatRequestManagerTest {
|
||||||
|
|
||||||
ConsumerGroupHeartbeatRequest heartbeatRequest =
|
ConsumerGroupHeartbeatRequest heartbeatRequest =
|
||||||
(ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version);
|
(ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version);
|
||||||
assertEquals(GROUP_ID, heartbeatRequest.data().groupId());
|
assertEquals(DEFAULT_GROUP_ID, heartbeatRequest.data().groupId());
|
||||||
assertEquals(memberId, heartbeatRequest.data().memberId());
|
assertEquals(memberId, heartbeatRequest.data().memberId());
|
||||||
assertEquals(memberEpoch, heartbeatRequest.data().memberEpoch());
|
assertEquals(memberEpoch, heartbeatRequest.data().memberEpoch());
|
||||||
assertEquals(10000, heartbeatRequest.data().rebalanceTimeoutMs());
|
assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, heartbeatRequest.data().rebalanceTimeoutMs());
|
||||||
assertEquals(subscribedTopics, heartbeatRequest.data().subscribedTopicNames());
|
assertEquals(subscribedTopics, heartbeatRequest.data().subscribedTopicNames());
|
||||||
assertEquals(GROUP_INSTANCE_ID, heartbeatRequest.data().instanceId());
|
assertEquals(DEFAULT_GROUP_INSTANCE_ID, heartbeatRequest.data().instanceId());
|
||||||
// TODO: Test pattern subscription and user provided assignor selection.
|
// TODO: Test pattern subscription and user provided assignor selection.
|
||||||
assertNull(heartbeatRequest.data().serverAssignor());
|
assertNull(heartbeatRequest.data().serverAssignor());
|
||||||
assertNull(heartbeatRequest.data().subscribedTopicRegex());
|
assertNull(heartbeatRequest.data().subscribedTopicRegex());
|
||||||
|
@ -273,25 +241,16 @@ public class HeartbeatRequestManagerTest {
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("errorProvider")
|
@MethodSource("errorProvider")
|
||||||
public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) {
|
public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) {
|
||||||
heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState(
|
|
||||||
logContext,
|
|
||||||
time,
|
|
||||||
HEARTBEAT_INTERVAL_MS,
|
|
||||||
RETRY_BACKOFF_MS,
|
|
||||||
RETRY_BACKOFF_MAX_MS,
|
|
||||||
0);
|
|
||||||
heartbeatRequestManager = createManager();
|
|
||||||
|
|
||||||
// Sending first heartbeat w/o assignment to set the state to STABLE
|
// Sending first heartbeat w/o assignment to set the state to STABLE
|
||||||
ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData()
|
ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData()
|
||||||
.setHeartbeatIntervalMs(HEARTBEAT_INTERVAL_MS)
|
.setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS)
|
||||||
.setMemberId(memberId)
|
.setMemberId(memberId)
|
||||||
.setMemberEpoch(memberEpoch));
|
.setMemberEpoch(memberEpoch));
|
||||||
membershipManager.updateState(rs1.data());
|
membershipManager.updateState(rs1.data());
|
||||||
assertEquals(MemberState.STABLE, membershipManager.state());
|
assertEquals(MemberState.STABLE, membershipManager.state());
|
||||||
|
|
||||||
// Handling errors on the second heartbeat
|
// Handling errors on the second heartbeat
|
||||||
time.sleep(HEARTBEAT_INTERVAL_MS);
|
time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS);
|
||||||
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(1, result.unsentRequests.size());
|
assertEquals(1, result.unsentRequests.size());
|
||||||
|
|
||||||
|
@ -304,19 +263,19 @@ public class HeartbeatRequestManagerTest {
|
||||||
|
|
||||||
switch (error) {
|
switch (error) {
|
||||||
case NONE:
|
case NONE:
|
||||||
verify(errorEventHandler, never()).handle(any());
|
verify(backgroundEventHandler, never()).add(any());
|
||||||
verify(membershipManager, times(2)).updateState(mockResponse.data());
|
verify(membershipManager, times(2)).updateState(mockResponse.data());
|
||||||
assertEquals(HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
|
assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case COORDINATOR_LOAD_IN_PROGRESS:
|
case COORDINATOR_LOAD_IN_PROGRESS:
|
||||||
verify(errorEventHandler, never()).handle(any());
|
verify(backgroundEventHandler, never()).add(any());
|
||||||
assertEquals(RETRY_BACKOFF_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
|
assertEquals(DEFAULT_RETRY_BACKOFF_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case COORDINATOR_NOT_AVAILABLE:
|
case COORDINATOR_NOT_AVAILABLE:
|
||||||
case NOT_COORDINATOR:
|
case NOT_COORDINATOR:
|
||||||
verify(errorEventHandler, never()).handle(any());
|
verify(backgroundEventHandler, never()).add(any());
|
||||||
verify(coordinatorRequestManager).markCoordinatorUnknown(any(), anyLong());
|
verify(coordinatorRequestManager).markCoordinatorUnknown(any(), anyLong());
|
||||||
assertEquals(0, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
|
assertEquals(0, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
|
||||||
break;
|
break;
|
||||||
|
@ -326,7 +285,7 @@ public class HeartbeatRequestManagerTest {
|
||||||
// The memberStateManager should have stopped heartbeat at this point
|
// The memberStateManager should have stopped heartbeat at this point
|
||||||
ensureFatalError();
|
ensureFatalError();
|
||||||
} else {
|
} else {
|
||||||
verify(errorEventHandler, never()).handle(any());
|
verify(backgroundEventHandler, never()).add(any());
|
||||||
assertEquals(0, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
|
assertEquals(0, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
|
@ -335,12 +294,12 @@ public class HeartbeatRequestManagerTest {
|
||||||
|
|
||||||
private void ensureFatalError() {
|
private void ensureFatalError() {
|
||||||
verify(membershipManager).transitionToFailed();
|
verify(membershipManager).transitionToFailed();
|
||||||
verify(errorEventHandler).handle(any());
|
verify(backgroundEventHandler).add(any());
|
||||||
ensureHeartbeatStopped();
|
ensureHeartbeatStopped();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void ensureHeartbeatStopped() {
|
private void ensureHeartbeatStopped() {
|
||||||
time.sleep(HEARTBEAT_INTERVAL_MS);
|
time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS);
|
||||||
assertEquals(MemberState.FAILED, membershipManager.state());
|
assertEquals(MemberState.FAILED, membershipManager.state());
|
||||||
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
|
||||||
assertEquals(0, result.unsentRequests.size());
|
assertEquals(0, result.unsentRequests.size());
|
||||||
|
@ -377,7 +336,7 @@ public class HeartbeatRequestManagerTest {
|
||||||
final Errors error) {
|
final Errors error) {
|
||||||
ConsumerGroupHeartbeatResponseData data = new ConsumerGroupHeartbeatResponseData()
|
ConsumerGroupHeartbeatResponseData data = new ConsumerGroupHeartbeatResponseData()
|
||||||
.setErrorCode(error.code())
|
.setErrorCode(error.code())
|
||||||
.setHeartbeatIntervalMs(HEARTBEAT_INTERVAL_MS)
|
.setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS)
|
||||||
.setMemberId(memberId)
|
.setMemberId(memberId)
|
||||||
.setMemberEpoch(memberEpoch);
|
.setMemberEpoch(memberEpoch);
|
||||||
if (error != Errors.NONE) {
|
if (error != Errors.NONE) {
|
||||||
|
@ -395,16 +354,4 @@ public class HeartbeatRequestManagerTest {
|
||||||
null,
|
null,
|
||||||
response);
|
response);
|
||||||
}
|
}
|
||||||
|
|
||||||
private HeartbeatRequestManager createManager() {
|
|
||||||
return new HeartbeatRequestManager(
|
|
||||||
logContext,
|
|
||||||
time,
|
|
||||||
config,
|
|
||||||
coordinatorRequestManager,
|
|
||||||
subscriptionState,
|
|
||||||
membershipManager,
|
|
||||||
heartbeatRequestState,
|
|
||||||
errorEventHandler);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -564,7 +564,7 @@ public class OffsetFetcherTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRestOffsetsAuthorizationFailure() {
|
public void testResetOffsetsAuthorizationFailure() {
|
||||||
buildFetcher();
|
buildFetcher();
|
||||||
assignFromUser(singleton(tp0));
|
assignFromUser(singleton(tp0));
|
||||||
subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST);
|
subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST);
|
||||||
|
@ -1246,7 +1246,7 @@ public class OffsetFetcherTest {
|
||||||
buildFetcher(metricConfig, isolationLevel, metadataExpireMs, subscriptionState, logContext);
|
buildFetcher(metricConfig, isolationLevel, metadataExpireMs, subscriptionState, logContext);
|
||||||
|
|
||||||
FetchMetricsRegistry metricsRegistry = new FetchMetricsRegistry(metricConfig.tags().keySet(), "consumertest-group");
|
FetchMetricsRegistry metricsRegistry = new FetchMetricsRegistry(metricConfig.tags().keySet(), "consumertest-group");
|
||||||
FetchConfig<byte[], byte[]> fetchConfig = new FetchConfig<>(
|
FetchConfig fetchConfig = new FetchConfig(
|
||||||
minBytes,
|
minBytes,
|
||||||
maxBytes,
|
maxBytes,
|
||||||
maxWaitMs,
|
maxWaitMs,
|
||||||
|
@ -1254,7 +1254,6 @@ public class OffsetFetcherTest {
|
||||||
maxPollRecords,
|
maxPollRecords,
|
||||||
true, // check crc
|
true, // check crc
|
||||||
CommonClientConfigs.DEFAULT_CLIENT_RACK,
|
CommonClientConfigs.DEFAULT_CLIENT_RACK,
|
||||||
new Deserializers<>(new ByteArrayDeserializer(), new ByteArrayDeserializer()),
|
|
||||||
isolationLevel);
|
isolationLevel);
|
||||||
Fetcher<byte[], byte[]> fetcher = new Fetcher<>(
|
Fetcher<byte[], byte[]> fetcher = new Fetcher<>(
|
||||||
logContext,
|
logContext,
|
||||||
|
@ -1262,6 +1261,7 @@ public class OffsetFetcherTest {
|
||||||
metadata,
|
metadata,
|
||||||
subscriptions,
|
subscriptions,
|
||||||
fetchConfig,
|
fetchConfig,
|
||||||
|
new Deserializers<>(new ByteArrayDeserializer(), new ByteArrayDeserializer()),
|
||||||
new FetchMetricsManager(metrics, metricsRegistry),
|
new FetchMetricsManager(metrics, metricsRegistry),
|
||||||
time);
|
time);
|
||||||
|
|
||||||
|
|
|
@ -22,6 +22,9 @@ import org.apache.kafka.clients.Metadata;
|
||||||
import org.apache.kafka.clients.NodeApiVersions;
|
import org.apache.kafka.clients.NodeApiVersions;
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
||||||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent;
|
||||||
import org.apache.kafka.common.Cluster;
|
import org.apache.kafka.common.Cluster;
|
||||||
import org.apache.kafka.common.ClusterResource;
|
import org.apache.kafka.common.ClusterResource;
|
||||||
import org.apache.kafka.common.IsolationLevel;
|
import org.apache.kafka.common.IsolationLevel;
|
||||||
|
@ -57,15 +60,20 @@ import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
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.assertFalse;
|
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
|
@ -84,6 +92,7 @@ public class OffsetsRequestManagerTest {
|
||||||
private SubscriptionState subscriptionState;
|
private SubscriptionState subscriptionState;
|
||||||
private MockTime time;
|
private MockTime time;
|
||||||
private ApiVersions apiVersions;
|
private ApiVersions apiVersions;
|
||||||
|
private BlockingQueue<BackgroundEvent> backgroundEventQueue;
|
||||||
private static final String TEST_TOPIC = "t1";
|
private static final String TEST_TOPIC = "t1";
|
||||||
private static final TopicPartition TEST_PARTITION_1 = new TopicPartition(TEST_TOPIC, 1);
|
private static final TopicPartition TEST_PARTITION_1 = new TopicPartition(TEST_TOPIC, 1);
|
||||||
private static final TopicPartition TEST_PARTITION_2 = new TopicPartition(TEST_TOPIC, 2);
|
private static final TopicPartition TEST_PARTITION_2 = new TopicPartition(TEST_TOPIC, 2);
|
||||||
|
@ -95,13 +104,25 @@ public class OffsetsRequestManagerTest {
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
public void setup() {
|
public void setup() {
|
||||||
|
LogContext logContext = new LogContext();
|
||||||
|
backgroundEventQueue = new LinkedBlockingQueue<>();
|
||||||
|
BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(logContext, backgroundEventQueue);
|
||||||
metadata = mock(ConsumerMetadata.class);
|
metadata = mock(ConsumerMetadata.class);
|
||||||
subscriptionState = mock(SubscriptionState.class);
|
subscriptionState = mock(SubscriptionState.class);
|
||||||
this.time = new MockTime(0);
|
time = new MockTime(0);
|
||||||
apiVersions = mock(ApiVersions.class);
|
apiVersions = mock(ApiVersions.class);
|
||||||
requestManager = new OffsetsRequestManager(subscriptionState, metadata,
|
requestManager = new OffsetsRequestManager(
|
||||||
DEFAULT_ISOLATION_LEVEL, time, RETRY_BACKOFF_MS, REQUEST_TIMEOUT_MS,
|
subscriptionState,
|
||||||
apiVersions, mock(NetworkClientDelegate.class), new LogContext());
|
metadata,
|
||||||
|
DEFAULT_ISOLATION_LEVEL,
|
||||||
|
time,
|
||||||
|
RETRY_BACKOFF_MS,
|
||||||
|
REQUEST_TIMEOUT_MS,
|
||||||
|
apiVersions,
|
||||||
|
mock(NetworkClientDelegate.class),
|
||||||
|
backgroundEventHandler,
|
||||||
|
logContext
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -500,7 +521,7 @@ public class OffsetsRequestManagerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testResetPositionsThrowsPreviousException() {
|
public void testResetOffsetsAuthorizationFailure() {
|
||||||
when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1));
|
when(subscriptionState.partitionsNeedingReset(time.milliseconds())).thenReturn(Collections.singleton(TEST_PARTITION_1));
|
||||||
when(subscriptionState.resetStrategy(any())).thenReturn(OffsetResetStrategy.EARLIEST);
|
when(subscriptionState.resetStrategy(any())).thenReturn(OffsetResetStrategy.EARLIEST);
|
||||||
mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1));
|
mockSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1));
|
||||||
|
@ -510,8 +531,9 @@ public class OffsetsRequestManagerTest {
|
||||||
// Reset positions response with TopicAuthorizationException
|
// Reset positions response with TopicAuthorizationException
|
||||||
NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds());
|
NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds());
|
||||||
NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0);
|
NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0);
|
||||||
|
Errors topicAuthorizationFailedError = Errors.TOPIC_AUTHORIZATION_FAILED;
|
||||||
ClientResponse clientResponse = buildClientResponseWithErrors(
|
ClientResponse clientResponse = buildClientResponseWithErrors(
|
||||||
unsentRequest, Collections.singletonMap(TEST_PARTITION_1, Errors.TOPIC_AUTHORIZATION_FAILED));
|
unsentRequest, Collections.singletonMap(TEST_PARTITION_1, topicAuthorizationFailedError));
|
||||||
clientResponse.onComplete();
|
clientResponse.onComplete();
|
||||||
|
|
||||||
assertTrue(unsentRequest.future().isDone());
|
assertTrue(unsentRequest.future().isDone());
|
||||||
|
@ -520,11 +542,23 @@ public class OffsetsRequestManagerTest {
|
||||||
verify(subscriptionState).requestFailed(any(), anyLong());
|
verify(subscriptionState).requestFailed(any(), anyLong());
|
||||||
verify(metadata).requestUpdate(false);
|
verify(metadata).requestUpdate(false);
|
||||||
|
|
||||||
// Following resetPositions should raise the previous exception without performing any
|
// Following resetPositions should enqueue the previous exception in the background event queue
|
||||||
// request
|
// without performing any request
|
||||||
assertThrows(TopicAuthorizationException.class,
|
assertDoesNotThrow(() -> requestManager.resetPositionsIfNeeded());
|
||||||
() -> requestManager.resetPositionsIfNeeded());
|
|
||||||
assertEquals(0, requestManager.requestsToSend());
|
assertEquals(0, requestManager.requestsToSend());
|
||||||
|
|
||||||
|
// Check that the event was enqueued during resetPositionsIfNeeded
|
||||||
|
assertEquals(1, backgroundEventQueue.size());
|
||||||
|
BackgroundEvent event = backgroundEventQueue.poll();
|
||||||
|
assertNotNull(event);
|
||||||
|
|
||||||
|
// Check that the event itself is of the expected type
|
||||||
|
assertInstanceOf(ErrorBackgroundEvent.class, event);
|
||||||
|
ErrorBackgroundEvent errorEvent = (ErrorBackgroundEvent) event;
|
||||||
|
assertNotNull(errorEvent.error());
|
||||||
|
|
||||||
|
// Check that the error held in the event is of the expected type
|
||||||
|
assertInstanceOf(topicAuthorizationFailedError.exception().getClass(), errorEvent.error());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -16,15 +16,12 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
||||||
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
|
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
|
||||||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler;
|
||||||
import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent;
|
import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent;
|
||||||
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.EventHandler;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent;
|
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent;
|
||||||
import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent;
|
import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent;
|
||||||
import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent;
|
import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent;
|
||||||
|
@ -35,13 +32,7 @@ import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.errors.InvalidGroupIdException;
|
import org.apache.kafka.common.errors.InvalidGroupIdException;
|
||||||
import org.apache.kafka.common.errors.TimeoutException;
|
import org.apache.kafka.common.errors.TimeoutException;
|
||||||
import org.apache.kafka.common.errors.WakeupException;
|
import org.apache.kafka.common.errors.WakeupException;
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
|
||||||
import org.apache.kafka.common.requests.ListOffsetsRequest;
|
import org.apache.kafka.common.requests.ListOffsetsRequest;
|
||||||
import org.apache.kafka.common.serialization.Deserializer;
|
|
||||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
|
||||||
import org.apache.kafka.common.utils.LogContext;
|
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
|
||||||
import org.apache.kafka.common.utils.Time;
|
|
||||||
import org.apache.kafka.common.utils.Timer;
|
import org.apache.kafka.common.utils.Timer;
|
||||||
import org.apache.kafka.test.TestUtils;
|
import org.apache.kafka.test.TestUtils;
|
||||||
import org.junit.jupiter.api.AfterEach;
|
import org.junit.jupiter.api.AfterEach;
|
||||||
|
@ -58,20 +49,15 @@ import java.util.HashSet;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.BlockingQueue;
|
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static java.util.Collections.singleton;
|
import static java.util.Collections.singleton;
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG;
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
|
||||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
|
||||||
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.assertFalse;
|
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
|
@ -80,56 +66,49 @@ import static org.mockito.Mockito.doThrow;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.mockConstruction;
|
import static org.mockito.Mockito.mockConstruction;
|
||||||
import static org.mockito.Mockito.never;
|
import static org.mockito.Mockito.never;
|
||||||
import static org.mockito.Mockito.spy;
|
|
||||||
import static org.mockito.Mockito.verify;
|
import static org.mockito.Mockito.verify;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
public class PrototypeAsyncConsumerTest {
|
public class PrototypeAsyncConsumerTest {
|
||||||
|
|
||||||
private PrototypeAsyncConsumer<?, ?> consumer;
|
private PrototypeAsyncConsumer<?, ?> consumer;
|
||||||
private final Map<String, Object> consumerProps = new HashMap<>();
|
private ConsumerTestBuilder.PrototypeAsyncConsumerTestBuilder testBuilder;
|
||||||
|
private ApplicationEventHandler applicationEventHandler;
|
||||||
private final Time time = new MockTime();
|
|
||||||
private LogContext logContext;
|
|
||||||
private SubscriptionState subscriptions;
|
|
||||||
private ConsumerMetadata metadata;
|
|
||||||
private EventHandler eventHandler;
|
|
||||||
private Metrics metrics;
|
|
||||||
|
|
||||||
private String groupId = "group.id";
|
|
||||||
private ConsumerConfig config;
|
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
public void setup() {
|
public void setup() {
|
||||||
injectConsumerConfigs();
|
// By default, the consumer is part of a group.
|
||||||
this.config = new ConsumerConfig(consumerProps);
|
setup(ConsumerTestBuilder.createDefaultGroupInformation());
|
||||||
this.logContext = new LogContext();
|
}
|
||||||
this.subscriptions = mock(SubscriptionState.class);
|
|
||||||
this.metadata = mock(ConsumerMetadata.class);
|
private void setup(Optional<ConsumerTestBuilder.GroupInformation> groupInfo) {
|
||||||
final DefaultBackgroundThread bt = mock(DefaultBackgroundThread.class);
|
testBuilder = new ConsumerTestBuilder.PrototypeAsyncConsumerTestBuilder(groupInfo);
|
||||||
final BlockingQueue<ApplicationEvent> aq = new LinkedBlockingQueue<>();
|
applicationEventHandler = testBuilder.applicationEventHandler;
|
||||||
final BlockingQueue<BackgroundEvent> bq = new LinkedBlockingQueue<>();
|
consumer = testBuilder.consumer;
|
||||||
this.eventHandler = spy(new DefaultEventHandler(bt, aq, bq));
|
|
||||||
this.metrics = new Metrics(time);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterEach
|
@AfterEach
|
||||||
public void cleanup() {
|
public void cleanup() {
|
||||||
if (consumer != null) {
|
if (testBuilder != null) {
|
||||||
consumer.close(Duration.ZERO);
|
testBuilder.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void resetWithEmptyGroupId() {
|
||||||
|
// Create a consumer that is not configured as part of a group.
|
||||||
|
cleanup();
|
||||||
|
setup(Optional.empty());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSuccessfulStartupShutdown() {
|
public void testSuccessfulStartupShutdown() {
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertDoesNotThrow(() -> consumer.close());
|
assertDoesNotThrow(() -> consumer.close());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testInvalidGroupId() {
|
public void testInvalidGroupId() {
|
||||||
this.groupId = null;
|
// Create consumer without group id
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
resetWithEmptyGroupId();
|
||||||
assertThrows(InvalidGroupIdException.class, () -> consumer.committed(new HashSet<>()));
|
assertThrows(InvalidGroupIdException.class, () -> consumer.committed(new HashSet<>()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -140,11 +119,10 @@ public class PrototypeAsyncConsumerTest {
|
||||||
offsets.put(new TopicPartition("my-topic", 0), new OffsetAndMetadata(100L));
|
offsets.put(new TopicPartition("my-topic", 0), new OffsetAndMetadata(100L));
|
||||||
offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L));
|
offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L));
|
||||||
|
|
||||||
PrototypeAsyncConsumer<?, ?> mockedConsumer = spy(newConsumer(time, new StringDeserializer(), new StringDeserializer()));
|
doReturn(future).when(consumer).commit(offsets, false);
|
||||||
doReturn(future).when(mockedConsumer).commit(offsets, false);
|
consumer.commitAsync(offsets, null);
|
||||||
mockedConsumer.commitAsync(offsets, null);
|
|
||||||
future.complete(null);
|
future.complete(null);
|
||||||
TestUtils.waitForCondition(() -> future.isDone(),
|
TestUtils.waitForCondition(future::isDone,
|
||||||
2000,
|
2000,
|
||||||
"commit future should complete");
|
"commit future should complete");
|
||||||
|
|
||||||
|
@ -159,11 +137,9 @@ public class PrototypeAsyncConsumerTest {
|
||||||
offsets.put(new TopicPartition("my-topic", 0), new OffsetAndMetadata(100L));
|
offsets.put(new TopicPartition("my-topic", 0), new OffsetAndMetadata(100L));
|
||||||
offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L));
|
offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L));
|
||||||
|
|
||||||
PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
doReturn(future).when(consumer).commit(offsets, false);
|
||||||
PrototypeAsyncConsumer<?, ?> mockedConsumer = spy(consumer);
|
|
||||||
doReturn(future).when(mockedConsumer).commit(offsets, false);
|
|
||||||
OffsetCommitCallback customCallback = mock(OffsetCommitCallback.class);
|
OffsetCommitCallback customCallback = mock(OffsetCommitCallback.class);
|
||||||
mockedConsumer.commitAsync(offsets, customCallback);
|
consumer.commitAsync(offsets, customCallback);
|
||||||
future.complete(null);
|
future.complete(null);
|
||||||
verify(customCallback).onComplete(offsets, null);
|
verify(customCallback).onComplete(offsets, null);
|
||||||
}
|
}
|
||||||
|
@ -175,9 +151,8 @@ public class PrototypeAsyncConsumerTest {
|
||||||
committedFuture.complete(offsets);
|
committedFuture.complete(offsets);
|
||||||
|
|
||||||
try (MockedConstruction<OffsetFetchApplicationEvent> ignored = offsetFetchEventMocker(committedFuture)) {
|
try (MockedConstruction<OffsetFetchApplicationEvent> ignored = offsetFetchEventMocker(committedFuture)) {
|
||||||
this.consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertDoesNotThrow(() -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000)));
|
assertDoesNotThrow(() -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000)));
|
||||||
verify(eventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
verify(applicationEventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -188,9 +163,8 @@ public class PrototypeAsyncConsumerTest {
|
||||||
committedFuture.completeExceptionally(new KafkaException("Test exception"));
|
committedFuture.completeExceptionally(new KafkaException("Test exception"));
|
||||||
|
|
||||||
try (MockedConstruction<OffsetFetchApplicationEvent> ignored = offsetFetchEventMocker(committedFuture)) {
|
try (MockedConstruction<OffsetFetchApplicationEvent> ignored = offsetFetchEventMocker(committedFuture)) {
|
||||||
this.consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertThrows(KafkaException.class, () -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000)));
|
assertThrows(KafkaException.class, () -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000)));
|
||||||
verify(eventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
verify(applicationEventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -230,25 +204,21 @@ public class PrototypeAsyncConsumerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAssign() {
|
public void testAssign() {
|
||||||
this.subscriptions = new SubscriptionState(logContext, OffsetResetStrategy.EARLIEST);
|
|
||||||
this.consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
final TopicPartition tp = new TopicPartition("foo", 3);
|
final TopicPartition tp = new TopicPartition("foo", 3);
|
||||||
consumer.assign(singleton(tp));
|
consumer.assign(singleton(tp));
|
||||||
assertTrue(consumer.subscription().isEmpty());
|
assertTrue(consumer.subscription().isEmpty());
|
||||||
assertTrue(consumer.assignment().contains(tp));
|
assertTrue(consumer.assignment().contains(tp));
|
||||||
verify(eventHandler).add(any(AssignmentChangeApplicationEvent.class));
|
verify(applicationEventHandler).add(any(AssignmentChangeApplicationEvent.class));
|
||||||
verify(eventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class));
|
verify(applicationEventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAssignOnNullTopicPartition() {
|
public void testAssignOnNullTopicPartition() {
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertThrows(IllegalArgumentException.class, () -> consumer.assign(null));
|
assertThrows(IllegalArgumentException.class, () -> consumer.assign(null));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAssignOnEmptyTopicPartition() {
|
public void testAssignOnEmptyTopicPartition() {
|
||||||
consumer = spy(newConsumer(time, new StringDeserializer(), new StringDeserializer()));
|
|
||||||
consumer.assign(Collections.emptyList());
|
consumer.assign(Collections.emptyList());
|
||||||
assertTrue(consumer.subscription().isEmpty());
|
assertTrue(consumer.subscription().isEmpty());
|
||||||
assertTrue(consumer.assignment().isEmpty());
|
assertTrue(consumer.assignment().isEmpty());
|
||||||
|
@ -256,76 +226,68 @@ public class PrototypeAsyncConsumerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAssignOnNullTopicInPartition() {
|
public void testAssignOnNullTopicInPartition() {
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0))));
|
assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0))));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAssignOnEmptyTopicInPartition() {
|
public void testAssignOnEmptyTopicInPartition() {
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0))));
|
assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0))));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testBeginningOffsetsFailsIfNullPartitions() {
|
public void testBeginningOffsetsFailsIfNullPartitions() {
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null,
|
assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null,
|
||||||
Duration.ofMillis(1)));
|
Duration.ofMillis(1)));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testBeginningOffsets() {
|
public void testBeginningOffsets() {
|
||||||
PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
Map<TopicPartition, OffsetAndTimestamp> expectedOffsetsAndTimestamp =
|
Map<TopicPartition, OffsetAndTimestamp> expectedOffsetsAndTimestamp =
|
||||||
mockOffsetAndTimestamp();
|
mockOffsetAndTimestamp();
|
||||||
Set<TopicPartition> partitions = expectedOffsetsAndTimestamp.keySet();
|
Set<TopicPartition> partitions = expectedOffsetsAndTimestamp.keySet();
|
||||||
doReturn(expectedOffsetsAndTimestamp).when(eventHandler).addAndGet(any(), any());
|
doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any());
|
||||||
Map<TopicPartition, Long> result =
|
Map<TopicPartition, Long> result =
|
||||||
assertDoesNotThrow(() -> consumer.beginningOffsets(partitions,
|
assertDoesNotThrow(() -> consumer.beginningOffsets(partitions,
|
||||||
Duration.ofMillis(1)));
|
Duration.ofMillis(1)));
|
||||||
Map<TopicPartition, Long> expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream()
|
Map<TopicPartition, Long> expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream()
|
||||||
.collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue().offset()));
|
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset()));
|
||||||
assertEquals(expectedOffsets, result);
|
assertEquals(expectedOffsets, result);
|
||||||
verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class),
|
verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class),
|
||||||
ArgumentMatchers.isA(Timer.class));
|
ArgumentMatchers.isA(Timer.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() {
|
public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() {
|
||||||
PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
Set<TopicPartition> partitions = mockTopicPartitionOffset().keySet();
|
Set<TopicPartition> partitions = mockTopicPartitionOffset().keySet();
|
||||||
Throwable eventProcessingFailure = new KafkaException("Unexpected failure " +
|
Throwable eventProcessingFailure = new KafkaException("Unexpected failure " +
|
||||||
"processing List Offsets event");
|
"processing List Offsets event");
|
||||||
doThrow(eventProcessingFailure).when(eventHandler).addAndGet(any(), any());
|
doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet(any(), any());
|
||||||
Throwable consumerError = assertThrows(KafkaException.class,
|
Throwable consumerError = assertThrows(KafkaException.class,
|
||||||
() -> consumer.beginningOffsets(partitions,
|
() -> consumer.beginningOffsets(partitions,
|
||||||
Duration.ofMillis(1)));
|
Duration.ofMillis(1)));
|
||||||
assertEquals(eventProcessingFailure, consumerError);
|
assertEquals(eventProcessingFailure, consumerError);
|
||||||
verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class));
|
verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() {
|
public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() {
|
||||||
PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any());
|
||||||
doThrow(new TimeoutException()).when(eventHandler).addAndGet(any(), any());
|
|
||||||
assertThrows(TimeoutException.class,
|
assertThrows(TimeoutException.class,
|
||||||
() -> consumer.beginningOffsets(
|
() -> consumer.beginningOffsets(
|
||||||
Collections.singletonList(new TopicPartition("t1", 0)),
|
Collections.singletonList(new TopicPartition("t1", 0)),
|
||||||
Duration.ofMillis(1)));
|
Duration.ofMillis(1)));
|
||||||
verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class),
|
verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class),
|
||||||
ArgumentMatchers.isA(Timer.class));
|
ArgumentMatchers.isA(Timer.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testOffsetsForTimesOnNullPartitions() {
|
public void testOffsetsForTimesOnNullPartitions() {
|
||||||
PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null,
|
assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null,
|
||||||
Duration.ofMillis(1)));
|
Duration.ofMillis(1)));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testOffsetsForTimesFailsOnNegativeTargetTimes() {
|
public void testOffsetsForTimesFailsOnNegativeTargetTimes() {
|
||||||
PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
assertThrows(IllegalArgumentException.class,
|
assertThrows(IllegalArgumentException.class,
|
||||||
() -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition(
|
() -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition(
|
||||||
"topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP),
|
"topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP),
|
||||||
|
@ -344,15 +306,14 @@ public class PrototypeAsyncConsumerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testOffsetsForTimes() {
|
public void testOffsetsForTimes() {
|
||||||
PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
Map<TopicPartition, OffsetAndTimestamp> expectedResult = mockOffsetAndTimestamp();
|
Map<TopicPartition, OffsetAndTimestamp> expectedResult = mockOffsetAndTimestamp();
|
||||||
Map<TopicPartition, Long> timestampToSearch = mockTimestampToSearch();
|
Map<TopicPartition, Long> timestampToSearch = mockTimestampToSearch();
|
||||||
|
|
||||||
doReturn(expectedResult).when(eventHandler).addAndGet(any(), any());
|
doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any());
|
||||||
Map<TopicPartition, OffsetAndTimestamp> result =
|
Map<TopicPartition, OffsetAndTimestamp> result =
|
||||||
assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1)));
|
assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1)));
|
||||||
assertEquals(expectedResult, result);
|
assertEquals(expectedResult, result);
|
||||||
verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class),
|
verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class),
|
||||||
ArgumentMatchers.isA(Timer.class));
|
ArgumentMatchers.isA(Timer.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -361,7 +322,6 @@ public class PrototypeAsyncConsumerTest {
|
||||||
// OffsetAndTimestamp as value.
|
// OffsetAndTimestamp as value.
|
||||||
@Test
|
@Test
|
||||||
public void testOffsetsForTimesWithZeroTimeout() {
|
public void testOffsetsForTimesWithZeroTimeout() {
|
||||||
PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
TopicPartition tp = new TopicPartition("topic1", 0);
|
TopicPartition tp = new TopicPartition("topic1", 0);
|
||||||
Map<TopicPartition, OffsetAndTimestamp> expectedResult =
|
Map<TopicPartition, OffsetAndTimestamp> expectedResult =
|
||||||
Collections.singletonMap(tp, null);
|
Collections.singletonMap(tp, null);
|
||||||
|
@ -371,14 +331,12 @@ public class PrototypeAsyncConsumerTest {
|
||||||
assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch,
|
assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch,
|
||||||
Duration.ofMillis(0)));
|
Duration.ofMillis(0)));
|
||||||
assertEquals(expectedResult, result);
|
assertEquals(expectedResult, result);
|
||||||
verify(eventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class),
|
verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class),
|
||||||
ArgumentMatchers.isA(Timer.class));
|
ArgumentMatchers.isA(Timer.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWakeup_committed() {
|
public void testWakeup_committed() {
|
||||||
consumer = newConsumer(time, new StringDeserializer(),
|
|
||||||
new StringDeserializer());
|
|
||||||
consumer.wakeup();
|
consumer.wakeup();
|
||||||
assertThrows(WakeupException.class, () -> consumer.committed(mockTopicPartitionOffset().keySet()));
|
assertThrows(WakeupException.class, () -> consumer.committed(mockTopicPartitionOffset().keySet()));
|
||||||
assertNoPendingWakeup(consumer.wakeupTrigger());
|
assertNoPendingWakeup(consumer.wakeupTrigger());
|
||||||
|
@ -386,87 +344,80 @@ public class PrototypeAsyncConsumerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRefreshCommittedOffsetsSuccess() {
|
public void testRefreshCommittedOffsetsSuccess() {
|
||||||
Map<TopicPartition, OffsetAndMetadata> committedOffsets =
|
TopicPartition partition = new TopicPartition("t1", 1);
|
||||||
Collections.singletonMap(new TopicPartition("t1", 1), new OffsetAndMetadata(10L));
|
Set<TopicPartition> partitions = Collections.singleton(partition);
|
||||||
testRefreshCommittedOffsetsSuccess(committedOffsets);
|
Map<TopicPartition, OffsetAndMetadata> committedOffsets = Collections.singletonMap(partition, new OffsetAndMetadata(10L));
|
||||||
|
testRefreshCommittedOffsetsSuccess(partitions, committedOffsets);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRefreshCommittedOffsetsSuccessButNoCommittedOffsetsFound() {
|
public void testRefreshCommittedOffsetsSuccessButNoCommittedOffsetsFound() {
|
||||||
testRefreshCommittedOffsetsSuccess(Collections.emptyMap());
|
TopicPartition partition = new TopicPartition("t1", 1);
|
||||||
|
Set<TopicPartition> partitions = Collections.singleton(partition);
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> committedOffsets = Collections.emptyMap();
|
||||||
|
testRefreshCommittedOffsetsSuccess(partitions, committedOffsets);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() {
|
public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() {
|
||||||
// Create consumer with group id to enable committed offset usage
|
|
||||||
this.groupId = "consumer-group-1";
|
|
||||||
|
|
||||||
testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(true);
|
testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() {
|
public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() {
|
||||||
// Create consumer without group id so committed offsets are not used for updating positions
|
// Create consumer without group id so committed offsets are not used for updating positions
|
||||||
this.groupId = null;
|
resetWithEmptyGroupId();
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
|
|
||||||
testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false);
|
testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) {
|
private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) {
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
|
|
||||||
// Uncompleted future that will time out if used
|
// Uncompleted future that will time out if used
|
||||||
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> committedFuture = new CompletableFuture<>();
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> committedFuture = new CompletableFuture<>();
|
||||||
|
|
||||||
when(subscriptions.initializingPartitions()).thenReturn(Collections.singleton(new TopicPartition("t1", 1)));
|
|
||||||
|
consumer.assign(singleton(new TopicPartition("t1", 1)));
|
||||||
|
|
||||||
try (MockedConstruction<OffsetFetchApplicationEvent> ignored = offsetFetchEventMocker(committedFuture)) {
|
try (MockedConstruction<OffsetFetchApplicationEvent> ignored = offsetFetchEventMocker(committedFuture)) {
|
||||||
|
|
||||||
// Poll with 0 timeout to run a single iteration of the poll loop
|
// Poll with 0 timeout to run a single iteration of the poll loop
|
||||||
consumer.poll(Duration.ofMillis(0));
|
consumer.poll(Duration.ofMillis(0));
|
||||||
|
|
||||||
verify(eventHandler).add(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class));
|
verify(applicationEventHandler).add(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class));
|
||||||
|
|
||||||
if (committedOffsetsEnabled) {
|
if (committedOffsetsEnabled) {
|
||||||
// Verify there was an OffsetFetch event and no ResetPositions event
|
// Verify there was an OffsetFetch event and no ResetPositions event
|
||||||
verify(eventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
verify(applicationEventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
||||||
verify(eventHandler,
|
verify(applicationEventHandler,
|
||||||
never()).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class));
|
never()).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class));
|
||||||
} else {
|
} else {
|
||||||
// Verify there was not any OffsetFetch event but there should be a ResetPositions
|
// Verify there was not any OffsetFetch event but there should be a ResetPositions
|
||||||
verify(eventHandler,
|
verify(applicationEventHandler,
|
||||||
never()).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
never()).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
||||||
verify(eventHandler).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class));
|
verify(applicationEventHandler).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void testRefreshCommittedOffsetsSuccess(Map<TopicPartition, OffsetAndMetadata> committedOffsets) {
|
private void testRefreshCommittedOffsetsSuccess(Set<TopicPartition> partitions,
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> committedOffsets) {
|
||||||
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> committedFuture = new CompletableFuture<>();
|
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> committedFuture = new CompletableFuture<>();
|
||||||
committedFuture.complete(committedOffsets);
|
committedFuture.complete(committedOffsets);
|
||||||
|
consumer.assign(partitions);
|
||||||
// Create consumer with group id to enable committed offset usage
|
|
||||||
this.groupId = "consumer-group-1";
|
|
||||||
consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer());
|
|
||||||
|
|
||||||
try (MockedConstruction<OffsetFetchApplicationEvent> ignored = offsetFetchEventMocker(committedFuture)) {
|
try (MockedConstruction<OffsetFetchApplicationEvent> ignored = offsetFetchEventMocker(committedFuture)) {
|
||||||
when(subscriptions.initializingPartitions()).thenReturn(committedOffsets.keySet());
|
|
||||||
|
|
||||||
// Poll with 0 timeout to run a single iteration of the poll loop
|
// Poll with 0 timeout to run a single iteration of the poll loop
|
||||||
consumer.poll(Duration.ofMillis(0));
|
consumer.poll(Duration.ofMillis(0));
|
||||||
|
|
||||||
verify(eventHandler).add(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class));
|
verify(applicationEventHandler).add(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class));
|
||||||
verify(eventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
verify(applicationEventHandler).add(ArgumentMatchers.isA(OffsetFetchApplicationEvent.class));
|
||||||
verify(eventHandler).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class));
|
verify(applicationEventHandler).add(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertNoPendingWakeup(final WakeupTrigger wakeupTrigger) {
|
private void assertNoPendingWakeup(final WakeupTrigger wakeupTrigger) {
|
||||||
assertTrue(wakeupTrigger.getPendingTask() == null);
|
assertNull(wakeupTrigger.getPendingTask());
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<TopicPartition, OffsetAndMetadata> mockTopicPartitionOffset() {
|
private HashMap<TopicPartition, OffsetAndMetadata> mockTopicPartitionOffset() {
|
||||||
final TopicPartition t0 = new TopicPartition("t0", 2);
|
final TopicPartition t0 = new TopicPartition("t0", 2);
|
||||||
final TopicPartition t1 = new TopicPartition("t0", 3);
|
final TopicPartition t1 = new TopicPartition("t0", 3);
|
||||||
HashMap<TopicPartition, OffsetAndMetadata> topicPartitionOffsets = new HashMap<>();
|
HashMap<TopicPartition, OffsetAndMetadata> topicPartitionOffsets = new HashMap<>();
|
||||||
|
@ -475,7 +426,7 @@ public class PrototypeAsyncConsumerTest {
|
||||||
return topicPartitionOffsets;
|
return topicPartitionOffsets;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<TopicPartition, OffsetAndTimestamp> mockOffsetAndTimestamp() {
|
private HashMap<TopicPartition, OffsetAndTimestamp> mockOffsetAndTimestamp() {
|
||||||
final TopicPartition t0 = new TopicPartition("t0", 2);
|
final TopicPartition t0 = new TopicPartition("t0", 2);
|
||||||
final TopicPartition t1 = new TopicPartition("t0", 3);
|
final TopicPartition t1 = new TopicPartition("t0", 3);
|
||||||
HashMap<TopicPartition, OffsetAndTimestamp> offsetAndTimestamp = new HashMap<>();
|
HashMap<TopicPartition, OffsetAndTimestamp> offsetAndTimestamp = new HashMap<>();
|
||||||
|
@ -484,7 +435,7 @@ public class PrototypeAsyncConsumerTest {
|
||||||
return offsetAndTimestamp;
|
return offsetAndTimestamp;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<TopicPartition, Long> mockTimestampToSearch() {
|
private HashMap<TopicPartition, Long> mockTimestampToSearch() {
|
||||||
final TopicPartition t0 = new TopicPartition("t0", 2);
|
final TopicPartition t0 = new TopicPartition("t0", 2);
|
||||||
final TopicPartition t1 = new TopicPartition("t0", 3);
|
final TopicPartition t1 = new TopicPartition("t0", 3);
|
||||||
HashMap<TopicPartition, Long> timestampToSearch = new HashMap<>();
|
HashMap<TopicPartition, Long> timestampToSearch = new HashMap<>();
|
||||||
|
@ -492,30 +443,5 @@ public class PrototypeAsyncConsumerTest {
|
||||||
timestampToSearch.put(t1, 2L);
|
timestampToSearch.put(t1, 2L);
|
||||||
return timestampToSearch;
|
return timestampToSearch;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void injectConsumerConfigs() {
|
|
||||||
consumerProps.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
|
|
||||||
consumerProps.put(DEFAULT_API_TIMEOUT_MS_CONFIG, "60000");
|
|
||||||
consumerProps.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
|
||||||
consumerProps.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
private PrototypeAsyncConsumer<?, ?> newConsumer(final Time time,
|
|
||||||
final Deserializer<?> keyDeserializer,
|
|
||||||
final Deserializer<?> valueDeserializer) {
|
|
||||||
consumerProps.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass());
|
|
||||||
consumerProps.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass());
|
|
||||||
|
|
||||||
return new PrototypeAsyncConsumer<>(
|
|
||||||
time,
|
|
||||||
logContext,
|
|
||||||
config,
|
|
||||||
subscriptions,
|
|
||||||
metadata,
|
|
||||||
eventHandler,
|
|
||||||
metrics,
|
|
||||||
Optional.ofNullable(this.groupId),
|
|
||||||
config.getInt(DEFAULT_API_TIMEOUT_MS_CONFIG));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,141 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder;
|
||||||
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
import org.junit.jupiter.api.AfterEach;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
import static org.junit.jupiter.api.Assertions.fail;
|
||||||
|
|
||||||
|
public class BackgroundEventHandlerTest {
|
||||||
|
|
||||||
|
private ConsumerTestBuilder testBuilder;
|
||||||
|
private BlockingQueue<BackgroundEvent> backgroundEventQueue;
|
||||||
|
private BackgroundEventHandler backgroundEventHandler;
|
||||||
|
private BackgroundEventProcessor backgroundEventProcessor;
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setup() {
|
||||||
|
testBuilder = new ConsumerTestBuilder();
|
||||||
|
backgroundEventQueue = testBuilder.backgroundEventQueue;
|
||||||
|
backgroundEventHandler = testBuilder.backgroundEventHandler;
|
||||||
|
backgroundEventProcessor = testBuilder.backgroundEventProcessor;
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterEach
|
||||||
|
public void tearDown() {
|
||||||
|
if (testBuilder != null)
|
||||||
|
testBuilder.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNoEvents() {
|
||||||
|
assertTrue(backgroundEventQueue.isEmpty());
|
||||||
|
backgroundEventProcessor.process((event, error) -> { });
|
||||||
|
assertTrue(backgroundEventQueue.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSingleEvent() {
|
||||||
|
BackgroundEvent event = new ErrorBackgroundEvent(new RuntimeException("A"));
|
||||||
|
backgroundEventQueue.add(event);
|
||||||
|
assertPeeked(event);
|
||||||
|
backgroundEventProcessor.process((e, error) -> { });
|
||||||
|
assertTrue(backgroundEventQueue.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSingleErrorEvent() {
|
||||||
|
KafkaException error = new KafkaException("error");
|
||||||
|
BackgroundEvent event = new ErrorBackgroundEvent(error);
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(error));
|
||||||
|
assertPeeked(event);
|
||||||
|
assertProcessThrows(error);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMultipleEvents() {
|
||||||
|
BackgroundEvent event1 = new ErrorBackgroundEvent(new RuntimeException("A"));
|
||||||
|
backgroundEventQueue.add(event1);
|
||||||
|
backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("B")));
|
||||||
|
backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("C")));
|
||||||
|
|
||||||
|
assertPeeked(event1);
|
||||||
|
backgroundEventProcessor.process((event, error) -> { });
|
||||||
|
assertTrue(backgroundEventQueue.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMultipleErrorEvents() {
|
||||||
|
Throwable error1 = new Throwable("error1");
|
||||||
|
KafkaException error2 = new KafkaException("error2");
|
||||||
|
KafkaException error3 = new KafkaException("error3");
|
||||||
|
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(error1));
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(error2));
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(error3));
|
||||||
|
|
||||||
|
assertProcessThrows(new KafkaException(error1));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMixedEventsWithErrorEvents() {
|
||||||
|
Throwable error1 = new Throwable("error1");
|
||||||
|
KafkaException error2 = new KafkaException("error2");
|
||||||
|
KafkaException error3 = new KafkaException("error3");
|
||||||
|
|
||||||
|
RuntimeException errorToCheck = new RuntimeException("A");
|
||||||
|
backgroundEventQueue.add(new ErrorBackgroundEvent(errorToCheck));
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(error1));
|
||||||
|
backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("B")));
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(error2));
|
||||||
|
backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("C")));
|
||||||
|
backgroundEventHandler.add(new ErrorBackgroundEvent(error3));
|
||||||
|
backgroundEventQueue.add(new ErrorBackgroundEvent(new RuntimeException("D")));
|
||||||
|
|
||||||
|
assertProcessThrows(new KafkaException(errorToCheck));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertPeeked(BackgroundEvent event) {
|
||||||
|
BackgroundEvent peekEvent = backgroundEventQueue.peek();
|
||||||
|
assertNotNull(peekEvent);
|
||||||
|
assertEquals(event, peekEvent);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertProcessThrows(Throwable error) {
|
||||||
|
assertFalse(backgroundEventQueue.isEmpty());
|
||||||
|
|
||||||
|
try {
|
||||||
|
backgroundEventProcessor.process();
|
||||||
|
fail("Should have thrown error: " + error);
|
||||||
|
} catch (Throwable t) {
|
||||||
|
assertEquals(error.getClass(), t.getClass());
|
||||||
|
assertEquals(error.getMessage(), t.getMessage());
|
||||||
|
}
|
||||||
|
|
||||||
|
assertTrue(backgroundEventQueue.isEmpty());
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue