From c498b9c6acfe795ee0ec51dddc5fc50a21699640 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 6 Jul 2015 09:20:26 -0700 Subject: [PATCH 1/2] KAFKA-2123; add commit callback with consumer network layer refactor --- .../apache/kafka/clients/consumer/Consumer.java | 10 + .../clients/consumer/ConsumerCommitCallback.java | 28 + .../kafka/clients/consumer/ConsumerRecords.java | 4 + .../kafka/clients/consumer/KafkaConsumer.java | 427 ++++----------- .../kafka/clients/consumer/MockConsumer.java | 19 +- .../consumer/internals/ConsumerNetworkClient.java | 214 ++++++++ .../clients/consumer/internals/Coordinator.java | 593 +++++++++++++-------- .../clients/consumer/internals/DelayedTask.java | 25 + .../consumer/internals/DelayedTaskQueue.java | 82 +++ .../kafka/clients/consumer/internals/Fetcher.java | 183 +++++-- .../clients/consumer/internals/Heartbeat.java | 18 +- .../internals/NoAvailableBrokersException.java | 23 + .../clients/consumer/internals/RequestFuture.java | 207 ++++--- .../consumer/internals/RequestFutureAdapter.java | 43 ++ .../consumer/internals/RequestFutureListener.java | 23 + .../consumer/internals/SendFailedException.java | 27 + .../consumer/internals/StaleMetadataException.java | 22 + .../ConsumerCoordinatorNotAvailableException.java | 40 ++ .../kafka/common/errors/DisconnectException.java | 39 ++ .../errors/NotCoordinatorForConsumerException.java | 40 ++ .../errors/OffsetLoadInProgressException.java | 40 ++ .../org/apache/kafka/common/protocol/Errors.java | 6 +- .../internals/ConsumerNetworkClientTest.java | 108 ++++ .../consumer/internals/CoordinatorTest.java | 337 +++++++----- .../consumer/internals/DelayedTaskQueueTest.java | 68 +++ .../clients/consumer/internals/FetcherTest.java | 37 +- .../clients/consumer/internals/HeartbeatTest.java | 15 + .../consumer/internals/RequestFutureTest.java | 57 ++ .../scala/integration/kafka/api/ConsumerTest.scala | 79 ++- 29 files changed, 1914 insertions(+), 900 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index fd98740..252b759 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -64,11 +64,21 @@ public interface Consumer extends Closeable { public void commit(CommitType commitType); /** + * @see KafkaConsumer#commit(CommitType, ConsumerCommitCallback) + */ + public void commit(CommitType commitType, ConsumerCommitCallback callback); + + /** * @see KafkaConsumer#commit(Map, CommitType) */ public void commit(Map offsets, CommitType commitType); /** + * @see KafkaConsumer#commit(Map, CommitType, ConsumerCommitCallback) + */ + public void commit(Map offsets, CommitType commitType, ConsumerCommitCallback callback); + + /** * @see KafkaConsumer#seek(TopicPartition, long) */ public void seek(TopicPartition partition, long offset); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java new file mode 100644 index 0000000..49a14d8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java @@ -0,0 +1,28 @@ +/** + * 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; + +/** + * A callback interface that the user can implement to trigger custom actions when a commit request completes. The callback + * may be executed in any thread calling {@link Consumer#poll(long) poll()}. + */ +public interface ConsumerCommitCallback { + + /** + * A callback method the user can implement to provide asynchronous handling of commit request completion. + * This method will be called when the commit request sent to the server has been acknowledged. + * + * @param exception The exception thrown during processing of the request, or null if the commit completed successfully + */ + void onComplete(Exception exception); +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index eb75d2e..16a8357 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -105,6 +105,10 @@ public class ConsumerRecords implements Iterable> { } } + public boolean isEmpty() { + return records.isEmpty(); + } + @SuppressWarnings("unchecked") public static ConsumerRecords empty() { return (ConsumerRecords) EMPTY; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 7aa0760..b6b7e98 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -15,9 +15,10 @@ package org.apache.kafka.clients.consumer; import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient; import org.apache.kafka.clients.consumer.internals.Coordinator; +import org.apache.kafka.clients.consumer.internals.DelayedTask; import org.apache.kafka.clients.consumer.internals.Fetcher; -import org.apache.kafka.clients.consumer.internals.RequestFuture; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; @@ -49,7 +50,6 @@ import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -393,8 +393,6 @@ import static org.apache.kafka.common.utils.Utils.min; public class KafkaConsumer implements Consumer { private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); - private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; - private static final long LATEST_OFFSET_TIMESTAMP = -1L; private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); private final Coordinator coordinator; @@ -403,7 +401,7 @@ public class KafkaConsumer implements Consumer { private final Fetcher fetcher; private final Time time; - private final NetworkClient client; + private final ConsumerNetworkClient client; private final Metrics metrics; private final SubscriptionState subscriptions; private final Metadata metadata; @@ -411,9 +409,7 @@ public class KafkaConsumer implements Consumer { private final boolean autoCommit; private final long autoCommitIntervalMs; private final ConsumerRebalanceCallback rebalanceCallback; - private long lastCommitAttemptMs; private boolean closed = false; - private final AtomicBoolean wakeup = new AtomicBoolean(false); // currentThread holds the threadId of the current thread accessing KafkaConsumer // and is used to prevent multi-threaded access @@ -512,7 +508,6 @@ public class KafkaConsumer implements Consumer { this.time = new SystemTime(); this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); - this.lastCommitAttemptMs = time.milliseconds(); MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), @@ -533,7 +528,7 @@ public class KafkaConsumer implements Consumer { String metricGrpPrefix = "consumer"; Map metricsTags = new LinkedHashMap(); metricsTags.put("client-id", clientId); - this.client = new NetworkClient( + NetworkClient netClient = new NetworkClient( new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags), this.metadata, clientId, @@ -541,17 +536,22 @@ public class KafkaConsumer implements Consumer { config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG), config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG), config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG)); + this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs); OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase()); this.subscriptions = new SubscriptionState(offsetResetStrategy); + int sessionTimeoutMs = config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG); this.coordinator = new Coordinator(this.client, config.getString(ConsumerConfig.GROUP_ID_CONFIG), - config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), + sessionTimeoutMs, config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), this.subscriptions, metrics, metricGrpPrefix, metricsTags, - this.time); + this.time, + requestTimeoutMs, + retryBackoffMs); + coordinator.initHeartbeatTask(); if (keyDeserializer == null) { this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, @@ -579,10 +579,14 @@ public class KafkaConsumer implements Consumer { metrics, metricGrpPrefix, metricsTags, - this.time); + this.time, + this.retryBackoffMs); config.logUnused(); + if (autoCommit) + scheduleAutoCommitTask(autoCommitIntervalMs); + log.debug("Kafka consumer created"); } catch (Throwable t) { // call close methods if internal objects are already constructed @@ -721,17 +725,15 @@ public class KafkaConsumer implements Consumer { long remaining = timeout; while (remaining >= 0) { long start = time.milliseconds(); - long pollTimeout = min(remaining, timeToNextCommit(start), coordinator.timeToNextHeartbeat(start)); - - Map>> records = pollOnce(pollTimeout, start); + Map>> records = pollOnce(remaining, start); long end = time.milliseconds(); if (!records.isEmpty()) { // If data is available, then return it, but first send off the // next round of fetches to enable pipelining while the user is // handling the fetched records. - fetcher.initFetches(metadata.fetch(), end); - pollClient(0, end); + fetcher.initFetches(metadata.fetch()); + client.poll(0, end); return new ConsumerRecords(records); } @@ -750,7 +752,6 @@ public class KafkaConsumer implements Consumer { } } - /** * Do one round of polling. In addition to checking for new data, this does any needed * heart-beating, auto-commits, and offset updates. @@ -759,37 +760,35 @@ public class KafkaConsumer implements Consumer { * @return The fetched records (may be empty) */ private Map>> pollOnce(long timeout, long now) { - Cluster cluster = this.metadata.fetch(); + coordinator.ensureCoordinatorKnown(); // TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) - - if (subscriptions.partitionsAutoAssigned()) { - if (subscriptions.partitionAssignmentNeeded()) { - // rebalance to get partition assignment - reassignPartitions(now); - } else { - // try to heartbeat with the coordinator if needed - coordinator.maybeHeartbeat(now); - } - } + if (subscriptions.partitionAssignmentNeeded()) + // rebalance to get partition assignment + reassignPartitions(); // fetch positions if we have partitions we're subscribed to that we // don't know the offset for if (!subscriptions.hasAllFetchPositions()) updateFetchPositions(this.subscriptions.missingFetchPositions()); - // maybe autocommit position - if (shouldAutoCommit(now)) - commit(CommitType.ASYNC); - // Init any new fetches (won't resend pending fetches) - fetcher.initFetches(cluster, now); - - pollClient(timeout, now); - + Cluster cluster = this.metadata.fetch(); + fetcher.initFetches(cluster); + client.poll(timeout, now); return fetcher.fetchedRecords(); } + private void scheduleAutoCommitTask(final long interval) { + DelayedTask task = new DelayedTask() { + public void run(long now) { + commit(CommitType.ASYNC); + client.schedule(this, now + interval); + } + }; + client.schedule(task, time.milliseconds() + interval); + } + /** * Commits the specified offsets for the specified list of topics and partitions to Kafka. *

@@ -806,16 +805,31 @@ public class KafkaConsumer implements Consumer { */ @Override public void commit(final Map offsets, CommitType commitType) { + commit(offsets, commitType, null); + } + + /** + * Commits the specified offsets for the specified list of topics and partitions to Kafka. + *

+ * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every + * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + *

+ * A non-blocking commit will attempt to commit offsets asynchronously. No error will be thrown if the commit fails. + * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until + * the commit succeeds. In either case, you can retrieve the result of the commit request using the callback + * parameter. + * + * @param offsets The list of offsets per partition that should be committed to Kafka. + * @param commitType Control whether the commit is blocking + * @param callback Callback to invoke when the commit completes + */ + @Override + public void commit(final Map offsets, CommitType commitType, ConsumerCommitCallback callback) { acquire(); try { log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); - - this.lastCommitAttemptMs = time.milliseconds(); - - // commit the offsets with the coordinator - if (commitType == CommitType.ASYNC) - this.subscriptions.needRefreshCommits(); - commitOffsets(offsets, commitType); + coordinator.commitOffsets(offsets, commitType, callback); } finally { release(); } @@ -827,22 +841,37 @@ public class KafkaConsumer implements Consumer { * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API * should not be used. - * + * * @param commitType Whether or not the commit should block until it is acknowledged. + * @param callback Callback to invoke when the commit completes */ @Override - public void commit(CommitType commitType) { + public void commit(CommitType commitType, ConsumerCommitCallback callback) { acquire(); try { // Need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance) Map allConsumed = new HashMap(this.subscriptions.allConsumed()); - commit(allConsumed, commitType); + commit(allConsumed, commitType, callback); } finally { release(); } } /** + * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. + *

+ * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after + * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + * + * @param commitType Whether or not the commit should block until it is acknowledged. + */ + @Override + public void commit(CommitType commitType) { + commit(commitType, null); + } + + /** * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets @@ -866,8 +895,10 @@ public class KafkaConsumer implements Consumer { try { Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() : Arrays.asList(partitions); - for (TopicPartition tp : parts) + for (TopicPartition tp : parts) { + log.debug("Seeking to beginning of partition {}", tp); subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); + } } finally { release(); } @@ -881,8 +912,10 @@ public class KafkaConsumer implements Consumer { try { Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() : Arrays.asList(partitions); - for (TopicPartition tp : parts) + for (TopicPartition tp : parts) { + log.debug("Seeking to end of partition {}", tp); subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + } } finally { release(); } @@ -938,7 +971,7 @@ public class KafkaConsumer implements Consumer { } else { partitionsToFetch = Collections.singleton(partition); } - refreshCommittedOffsets(partitionsToFetch); + coordinator.refreshCommittedOffsets(partitionsToFetch); Long committed = this.subscriptions.committed(partition); if (committed == null) throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); @@ -971,7 +1004,7 @@ public class KafkaConsumer implements Consumer { List parts = cluster.partitionsForTopic(topic); if (parts == null) { metadata.add(topic); - awaitMetadataUpdate(); + client.awaitMetadataUpdate(); parts = metadata.fetch().partitionsForTopic(topic); } return parts; @@ -997,7 +1030,6 @@ public class KafkaConsumer implements Consumer { */ @Override public void wakeup() { - this.wakeup.set(true); this.client.wakeup(); } @@ -1015,35 +1047,10 @@ public class KafkaConsumer implements Consumer { } } - - private boolean shouldAutoCommit(long now) { - return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs; - } - - private long timeToNextCommit(long now) { - if (!this.autoCommit) - return Long.MAX_VALUE; - long timeSinceLastCommit = now - this.lastCommitAttemptMs; - if (timeSinceLastCommit > this.autoCommitIntervalMs) - return 0; - return this.autoCommitIntervalMs - timeSinceLastCommit; - } - - /** - * Request a metadata update and wait until it has occurred - */ - private void awaitMetadataUpdate() { - int version = this.metadata.requestUpdate(); - do { - long now = time.milliseconds(); - this.pollClient(this.retryBackoffMs, now); - } while (this.metadata.version() == version); - } - /** * Get partition assignment */ - private void reassignPartitions(long now) { + private void reassignPartitions() { // execute the user's callback before rebalance log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); try { @@ -1054,7 +1061,7 @@ public class KafkaConsumer implements Consumer { } // get new assigned partitions from the coordinator - assignPartitions(); + coordinator.assignPartitions(); // execute the user's callback after rebalance log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); @@ -1076,266 +1083,10 @@ public class KafkaConsumer implements Consumer { */ private void updateFetchPositions(Set partitions) { // first refresh the committed positions in case they are not up-to-date - refreshCommittedOffsets(partitions); - - // reset the fetch position to the committed position - for (TopicPartition tp : partitions) { - // Skip if we already have a fetch position - if (subscriptions.fetched(tp) != null) - continue; - - // TODO: If there are several offsets to reset, we could submit offset requests in parallel - if (subscriptions.isOffsetResetNeeded(tp)) { - resetOffset(tp); - } else if (subscriptions.committed(tp) == null) { - // There's no committed position, so we need to reset with the default strategy - subscriptions.needOffsetReset(tp); - resetOffset(tp); - } else { - log.debug("Resetting offset for partition {} to the committed offset {}", - tp, subscriptions.committed(tp)); - subscriptions.seek(tp, subscriptions.committed(tp)); - } - } - } - - /** - * Reset offsets for the given partition using the offset reset strategy. - * - * @param partition The given partition that needs reset offset - * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined - */ - private void resetOffset(TopicPartition partition) { - OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); - final long timestamp; - if (strategy == OffsetResetStrategy.EARLIEST) - timestamp = EARLIEST_OFFSET_TIMESTAMP; - else if (strategy == OffsetResetStrategy.LATEST) - timestamp = LATEST_OFFSET_TIMESTAMP; - else - throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); - - log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase()); - long offset = listOffset(partition, timestamp); - this.subscriptions.seek(partition, offset); - } - - /** - * Fetch a single offset before the given timestamp for the partition. - * - * @param partition The partition that needs fetching offset. - * @param timestamp The timestamp for fetching offset. - * @return The offset of the message that is published before the given timestamp - */ - private long listOffset(TopicPartition partition, long timestamp) { - while (true) { - RequestFuture future = fetcher.listOffset(partition, timestamp); - - if (!future.isDone()) - pollFuture(future, requestTimeoutMs); - - if (future.isDone()) { - if (future.succeeded()) - return future.value(); - handleRequestFailure(future); - } - } - } - - /** - * Refresh the committed offsets for given set of partitions and update the cache - */ - private void refreshCommittedOffsets(Set partitions) { - // we only need to fetch latest committed offset from coordinator if there - // is some commit process in progress, otherwise our current - // committed cache is up-to-date - if (subscriptions.refreshCommitsNeeded()) { - // contact coordinator to fetch committed offsets - Map offsets = fetchCommittedOffsets(partitions); - - // update the position with the offsets - for (Map.Entry entry : offsets.entrySet()) { - TopicPartition tp = entry.getKey(); - this.subscriptions.committed(tp, entry.getValue()); - } - } - } - - /** - * Block until we have received a partition assignment from the coordinator. - */ - private void assignPartitions() { - // Ensure that there are no pending requests to the coordinator. This is important - // in particular to avoid resending a pending JoinGroup request. - awaitCoordinatorInFlightRequests(); - - while (subscriptions.partitionAssignmentNeeded()) { - RequestFuture future = coordinator.assignPartitions(time.milliseconds()); - - // Block indefinitely for the join group request (which can take as long as a session timeout) - if (!future.isDone()) - pollFuture(future); - - if (future.failed()) - handleRequestFailure(future); - } - } - - /** - * Block until the coordinator for this group is known. - */ - private void ensureCoordinatorKnown() { - while (coordinator.coordinatorUnknown()) { - RequestFuture future = coordinator.discoverConsumerCoordinator(); + coordinator.refreshCommittedOffsets(partitions); - if (!future.isDone()) - pollFuture(future, requestTimeoutMs); - - if (future.failed()) - handleRequestFailure(future); - } - } - - /** - * Block until any pending requests to the coordinator have been handled. - */ - public void awaitCoordinatorInFlightRequests() { - while (coordinator.hasInFlightRequests()) { - long now = time.milliseconds(); - pollClient(-1, now); - } - } - - /** - * Lookup the committed offsets for a set of partitions. This will block until the coordinator has - * responded to the offset fetch request. - * @param partitions List of partitions to get offsets for - * @return Map from partition to its respective offset - */ - private Map fetchCommittedOffsets(Set partitions) { - while (true) { - long now = time.milliseconds(); - RequestFuture> future = coordinator.fetchOffsets(partitions, now); - - if (!future.isDone()) - pollFuture(future, requestTimeoutMs); - - if (future.isDone()) { - if (future.succeeded()) - return future.value(); - handleRequestFailure(future); - } - } - } - - /** - * Commit offsets. This call blocks (regardless of commitType) until the coordinator - * can receive the commit request. Once the request has been made, however, only the - * synchronous commits will wait for a successful response from the coordinator. - * @param offsets Offsets to commit. - * @param commitType Commit policy - */ - private void commitOffsets(Map offsets, CommitType commitType) { - if (commitType == CommitType.ASYNC) { - commitOffsetsAsync(offsets); - } else { - commitOffsetsSync(offsets); - } - } - - private void commitOffsetsAsync(Map offsets) { - while (true) { - long now = time.milliseconds(); - RequestFuture future = coordinator.commitOffsets(offsets, now); - - if (!future.isDone() || future.succeeded()) - return; - - handleRequestFailure(future); - } - } - - private void commitOffsetsSync(Map offsets) { - while (true) { - long now = time.milliseconds(); - RequestFuture future = coordinator.commitOffsets(offsets, now); - - if (!future.isDone()) - pollFuture(future, requestTimeoutMs); - - if (future.isDone()) { - if (future.succeeded()) - return; - else - handleRequestFailure(future); - } - } - } - - private void handleRequestFailure(RequestFuture future) { - if (future.hasException()) - throw future.exception(); - - switch (future.retryAction()) { - case BACKOFF: - Utils.sleep(retryBackoffMs); - break; - case POLL: - pollClient(retryBackoffMs, time.milliseconds()); - break; - case FIND_COORDINATOR: - ensureCoordinatorKnown(); - break; - case REFRESH_METADATA: - awaitMetadataUpdate(); - break; - case NOOP: - // Do nothing (retry now) - } - } - - /** - * Poll until a result is ready or timeout expires - * @param future The future to poll for - * @param timeout The time in milliseconds to wait for the result - */ - private void pollFuture(RequestFuture future, long timeout) { - // TODO: Update this code for KAFKA-2120, which adds request timeout to NetworkClient - // In particular, we must ensure that "timed out" requests will not have their callbacks - // invoked at a later time. - long remaining = timeout; - while (!future.isDone() && remaining >= 0) { - long start = time.milliseconds(); - pollClient(remaining, start); - if (future.isDone()) return; - remaining -= time.milliseconds() - start; - } - } - - /** - * Poll indefinitely until the result is ready. - * @param future The future to poll for. - */ - private void pollFuture(RequestFuture future) { - while (!future.isDone()) { - long now = time.milliseconds(); - pollClient(-1, now); - } - } - - /** - * Poll for IO. - * @param timeout The maximum time to wait for IO to become available - * @param now The current time in milliseconds - * @throws ConsumerWakeupException if {@link #wakeup()} is invoked while the poll is active - */ - private void pollClient(long timeout, long now) { - this.client.poll(timeout, now); - - if (wakeup.get()) { - wakeup.set(false); - throw new ConsumerWakeupException(); - } + // then do any lookups in case some positions are not known + fetcher.updateFetchPositions(partitions); } /* diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 46e26a6..49d4f35 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -106,16 +106,29 @@ public class MockConsumer implements Consumer { } @Override - public synchronized void commit(Map offsets, CommitType commitType) { + public synchronized void commit(Map offsets, CommitType commitType, ConsumerCommitCallback callback) { ensureNotClosed(); for (Entry entry : offsets.entrySet()) subscriptions.committed(entry.getKey(), entry.getValue()); + if (callback != null) { + callback.onComplete(null); + } } @Override - public synchronized void commit(CommitType commitType) { + public synchronized void commit(Map offsets, CommitType commitType) { + commit(offsets, commitType, null); + } + + @Override + public synchronized void commit(CommitType commitType, ConsumerCommitCallback callback) { ensureNotClosed(); - commit(this.subscriptions.allConsumed(), commitType); + commit(this.subscriptions.allConsumed(), commitType, callback); + } + + @Override + public synchronized void commit(CommitType commitType) { + commit(commitType, null); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java new file mode 100644 index 0000000..71f6769 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -0,0 +1,214 @@ +/** + * 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.ClientRequest; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.ConsumerWakeupException; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.utils.Time; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Higher level consumer access to the network layer with basic support for futures and + * task scheduling. NOT thread-safe! + */ +public class ConsumerNetworkClient implements Closeable { + private final KafkaClient client; + private final AtomicBoolean wakeup = new AtomicBoolean(false); + private final DelayedTaskQueue delayedTasks = new DelayedTaskQueue(); + private final Map> unsent = new HashMap>(); + private final Metadata metadata; + private final Time time; + private final long retryBackoffMs; + + public ConsumerNetworkClient(KafkaClient client, + Metadata metadata, + Time time, + long retryBackoffMs) { + this.client = client; + this.metadata = metadata; + this.time = time; + this.retryBackoffMs = retryBackoffMs; + } + + public void schedule(DelayedTask task, long at) { + delayedTasks.add(task, at); + } + + public RequestFuture send(Node node, + ApiKeys api, + AbstractRequest request) { + long now = time.milliseconds(); + RequestFutureCompletionHandler future = new RequestFutureCompletionHandler(); + RequestHeader header = client.nextRequestHeader(api); + RequestSend send = new RequestSend(node.idString(), header, request.toStruct()); + put(node, new ClientRequest(now, true, send, future)); + return future; + } + + private void put(Node node, ClientRequest request) { + List nodeUnsent = unsent.get(node); + if (nodeUnsent == null) { + nodeUnsent = new ArrayList(); + unsent.put(node, nodeUnsent); + } + nodeUnsent.add(request); + } + + public Node leastLoadedNode() { + return client.leastLoadedNode(time.milliseconds()); + } + + public void awaitMetadataUpdate() { + int version = this.metadata.requestUpdate(); + do { + poll(retryBackoffMs); + } while (this.metadata.version() == version); + } + + public void wakeup() { + this.wakeup.set(true); + this.client.wakeup(); + } + + public void poll(RequestFuture future) { + while (!future.isDone()) + poll(Long.MAX_VALUE); + } + + public boolean poll(RequestFuture future, long timeout) { + long now = time.milliseconds(); + long deadline = now + timeout; + while (!future.isDone() && now < deadline) { + poll(deadline - now, now); + now = time.milliseconds(); + } + return future.isDone(); + } + + public void poll(long timeout) { + poll(timeout, time.milliseconds()); + } + + public void poll(long timeout, long now) { + // All unsent requests will either be successfully transmitted + // to the networking layer or fail in this call. + delayedTasks.poll(now); + pollUnsentRequests(now); + + timeout = Math.min(timeout, delayedTasks.nextTimeout(now)); + client.poll(timeout, now); + clearUnsentRequests(now); + + if (wakeup.get()) { + wakeup.set(false); + throw new ConsumerWakeupException(); + } + } + + public void awaitPendingRequests(Node node) { + while (pendingRequestCount(node) > 0) + poll(retryBackoffMs); + } + + public int pendingRequestCount(Node node) { + List pending = unsent.get(node); + int unsentCount = pending == null ? 0 : pending.size(); + return unsentCount + client.inFlightRequestCount(node.idString()); + } + + public int pendingRequestCount() { + int total = 0; + for (List requests: unsent.values()) + total += requests.size(); + return total + client.inFlightRequestCount(); + } + + private void pollUnsentRequests(long now) { + while (trySend(now)) + client.poll(0, now); + } + + private void clearUnsentRequests(long now) { + // Try again to send the unsent requests (pending requests may have been + // pushed in the poll call) + pollUnsentRequests(now); + + // Fail whatever is left + for (Map.Entry> requestEntry: unsent.entrySet()) { + Iterator iterator = requestEntry.getValue().iterator(); + while (iterator.hasNext()) { + ClientRequest request = iterator.next(); + RequestFutureCompletionHandler handler = + (RequestFutureCompletionHandler) request.callback(); + handler.raise(SendFailedException.INSTANCE); + iterator.remove(); + } + } + unsent.clear(); + } + + private boolean trySend(long now) { + // Send any requests that can be sent + boolean requestsSent = false; + for (Map.Entry> requestEntry: unsent.entrySet()) { + Node node = requestEntry.getKey(); + Iterator iterator = requestEntry.getValue().iterator(); + while (iterator.hasNext()) { + ClientRequest request = iterator.next(); + if (client.ready(node, now)) { + client.send(request); + iterator.remove(); + requestsSent = true; + } else if (client.connectionFailed(node)) { + RequestFutureCompletionHandler handler = + (RequestFutureCompletionHandler) request.callback(); + handler.onComplete(new ClientResponse(request, now, true, null)); + iterator.remove(); + } + } + } + return requestsSent; + } + + @Override + public void close() throws IOException { + client.close(); + } + + public static class RequestFutureCompletionHandler + extends RequestFuture + implements RequestCompletionHandler { + + @Override + public void onComplete(ClientResponse response) { + complete(response); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index c1c8172..b46528f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -12,14 +12,18 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.KafkaClient; -import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.ConsumerCommitCallback; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ConsumerCoordinatorNotAvailableException; +import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.InvalidMetadataException; +import org.apache.kafka.common.errors.NotCoordinatorForConsumerException; +import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -30,7 +34,7 @@ import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.ConsumerMetadataRequest; import org.apache.kafka.common.requests.ConsumerMetadataResponse; import org.apache.kafka.common.requests.HeartbeatRequest; @@ -41,8 +45,6 @@ import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; -import org.apache.kafka.common.requests.RequestHeader; -import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -62,8 +64,7 @@ public final class Coordinator { private static final Logger log = LoggerFactory.getLogger(Coordinator.class); - private final KafkaClient client; - + private final ConsumerNetworkClient client; private final Time time; private final String groupId; private final Heartbeat heartbeat; @@ -71,6 +72,8 @@ public final class Coordinator { private final String assignmentStrategy; private final SubscriptionState subscriptions; private final CoordinatorMetrics sensors; + private final long requestTimeoutMs; + private final long retryBackoffMs; private Node consumerCoordinator; private String consumerId; private int generation; @@ -78,7 +81,7 @@ public final class Coordinator { /** * Initialize the coordination manager. */ - public Coordinator(KafkaClient client, + public Coordinator(ConsumerNetworkClient client, String groupId, int sessionTimeoutMs, String assignmentStrategy, @@ -86,10 +89,12 @@ public final class Coordinator { Metrics metrics, String metricGrpPrefix, Map metricTags, - Time time) { + Time time, + long requestTimeoutMs, + long retryBackoffMs) { - this.time = time; this.client = client; + this.time = time; this.generation = -1; this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; this.groupId = groupId; @@ -99,18 +104,130 @@ public final class Coordinator { this.assignmentStrategy = assignmentStrategy; this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags); + this.requestTimeoutMs = requestTimeoutMs; + this.retryBackoffMs = retryBackoffMs; + } + + /** + * Refresh the committed offsets for given set of partitions and update the cache + */ + public void refreshCommittedOffsets(Set partitions) { + // we only need to fetch latest committed offset from coordinator if there + // is some commit process in progress, otherwise our current + // committed cache is up-to-date + if (subscriptions.refreshCommitsNeeded()) { + // contact coordinator to fetch committed offsets + Map offsets = fetchCommittedOffsets(partitions); + + // update the position with the offsets + for (Map.Entry entry : offsets.entrySet()) { + TopicPartition tp = entry.getKey(); + this.subscriptions.committed(tp, entry.getValue()); + } + } + } + + /** + * Block until we have received a partition assignment from the coordinator. + */ + public void assignPartitions() { + ensureCoordinatorKnown(); + + // Ensure that there are no pending requests to the coordinator. This is important + // in particular to avoid resending a pending JoinGroup request. + client.awaitPendingRequests(this.consumerCoordinator); + + while (subscriptions.partitionAssignmentNeeded()) { + RequestFuture future = sendJoinGroupRequest(); + client.poll(future); + + if (future.failed()) + handleRequestFailure(future); + } + } + + /** + * Block until the coordinator for this group is known. + */ + public void ensureCoordinatorKnown() { + while (coordinatorUnknown()) { + RequestFuture future = sendConsumerMetadataRequest(); + client.poll(future, requestTimeoutMs); + + if (future.failed()) + handleRequestFailure(future); + } + } + + /** + * Lookup the committed offsets for a set of partitions. This will block until the coordinator has + * responded to the offset fetch request. + * @param partitions List of partitions to get offsets for + * @return Map from partition to its respective offset + */ + public Map fetchCommittedOffsets(Set partitions) { + while (true) { + RequestFuture> future = sendOffsetFetchRequest(partitions); + client.poll(future); + + if (future.succeeded()) + return future.value(); + + handleRequestFailure(future); + } + } + + /** + * Commit offsets. This call blocks (regardless of commitType) until the coordinator + * can receive the commit request. Once the request has been made, however, only the + * synchronous commits will wait for a successful response from the coordinator. + * @param offsets Offsets to commit. + * @param commitType Commit policy + */ + public void commitOffsets(Map offsets, CommitType commitType, ConsumerCommitCallback callback) { + if (commitType == CommitType.ASYNC) + commitOffsetsAsync(offsets, callback); + else + commitOffsetsSync(offsets, callback); + } + + public void initHeartbeatTask() { + final long interval = heartbeat.interval(); + DelayedTask heartbeatTask = new DelayedTask() { + @Override + public void run(long now) { + if (!subscriptions.partitionsAutoAssigned() || coordinatorUnknown()) { + client.schedule(this, now + interval); + } else if (heartbeat.sessionTimeoutExpired(now)) { + // We haven't sent a successful heartbeat in one session interval + coordinatorDead(); + client.schedule(this, now + interval); + } else if (!heartbeat.shouldHeartbeat(now)) { + client.schedule(this, now + heartbeat.timeToNextHeartbeat(now)); + } else { + RequestFuture future = sendHeartbeatRequest(); + client.poll(future, 0); + if (future.failed()) { + client.schedule(this, retryBackoffMs); + } else { + heartbeat.sentHeartbeat(now); + client.schedule(this, now + interval); + } + } + } + }; + client.schedule(heartbeatTask, time.milliseconds() + interval); } /** * Send a request to get a new partition assignment. This is a non-blocking call which sends * a JoinGroup request to the coordinator (if it is available). The returned future must * be polled to see if the request completed successfully. - * @param now The current time in milliseconds * @return A request future whose completion indicates the result of the JoinGroup request. */ - public RequestFuture assignPartitions(final long now) { - final RequestFuture future = newCoordinatorRequestFuture(now); - if (future.isDone()) return future; + private RequestFuture sendJoinGroupRequest() { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); // send a join group request to the coordinator List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); @@ -124,25 +241,19 @@ public final class Coordinator { // create the request for the coordinator log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); + return sendCoordinator(ApiKeys.JOIN_GROUP, request).compose(new JoinGroupResponseHandler()); + } - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - handleJoinResponse(resp, future); - } - }; + private class JoinGroupResponseHandler extends CoordinatorResponseHandler { - sendCoordinator(ApiKeys.JOIN_GROUP, request.toStruct(), completionHandler, now); - return future; - } + @Override + public JoinGroupResponse parse(ClientResponse response) { + return new JoinGroupResponse(response.responseBody()); + } - private void handleJoinResponse(ClientResponse response, RequestFuture future) { - if (response.wasDisconnected()) { - handleCoordinatorDisconnect(response); - future.retryWithNewCoordinator(); - } else { + @Override + public void handle(JoinGroupResponse joinResponse, RequestFuture future) { // process the response - JoinGroupResponse joinResponse = new JoinGroupResponse(response.responseBody()); short errorCode = joinResponse.errorCode(); if (errorCode == Errors.NONE.code()) { @@ -152,10 +263,10 @@ public final class Coordinator { // set the flag to refresh last committed offsets subscriptions.needRefreshCommits(); - log.debug("Joined group: {}", response); + log.debug("Joined group: {}", joinResponse.toStruct()); // record re-assignment time - this.sensors.partitionReassignments.record(response.requestLatencyMs()); + sensors.partitionReassignments.record(response.requestLatencyMs()); // update partition assignment subscriptions.changePartitionAssignment(joinResponse.assignedPartitions()); @@ -165,23 +276,22 @@ public final class Coordinator { Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", groupId); - - future.retryNow(); + future.raise(Errors.UNKNOWN_CONSUMER_ID); } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { // re-discover the coordinator and retry with backoff coordinatorDead(); log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", groupId); - future.retryWithNewCoordinator(); + future.raise(Errors.forCode(errorCode)); } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code() || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code() || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) { // log the error and re-throw the exception - KafkaException e = Errors.forCode(errorCode).exception(); + Errors error = Errors.forCode(errorCode); log.error("Attempt to join group {} failed due to: {}", - groupId, e.getMessage()); - future.raise(e); + groupId, error.exception().getMessage()); + future.raise(error); } else { // unexpected error, throw the exception future.raise(new KafkaException("Unexpected error in join group response: " @@ -190,55 +300,94 @@ public final class Coordinator { } } + private void commitOffsetsAsync(Map offsets, final ConsumerCommitCallback callback) { + this.subscriptions.needRefreshCommits(); + RequestFuture future = sendOffsetCommitRequest(offsets); + if (callback != null) { + future.addListener(new RequestFutureListener() { + @Override + public void onSuccess(Void value) { + callback.onComplete(null); + } + + @Override + public void onFailure(RuntimeException e) { + callback.onComplete(e); + } + }); + } + } + + private void commitOffsetsSync(Map offsets, ConsumerCommitCallback callback) { + while (true) { + RequestFuture future = sendOffsetCommitRequest(offsets); + client.poll(future); + + if (future.succeeded()) { + if (callback != null) + callback.onComplete(null); + return; + } + + if (!future.isRetriable()) { + if (callback == null) + throw future.exception(); + else + callback.onComplete(future.exception()); + return; + } + + handleRetriableException((RetriableException) future.exception()); + } + } + + private void handleRequestFailure(RequestFuture future) { + if (!future.isRetriable()) + throw future.exception(); + + handleRetriableException((RetriableException) future.exception()); + } + + private void handleRetriableException(RetriableException e) { + if (e instanceof ConsumerCoordinatorNotAvailableException || + e instanceof NotCoordinatorForConsumerException || + e instanceof DisconnectException) { + ensureCoordinatorKnown(); + } else if (e instanceof InvalidMetadataException) { + client.awaitMetadataUpdate(); + } else { + Utils.sleep(retryBackoffMs); + } + } + /** * Commit offsets for the specified list of topics and partitions. This is a non-blocking call * which returns a request future that can be polled in the case of a synchronous commit or ignored in the * asynchronous case. * * @param offsets The list of offsets per partition that should be committed. - * @param now The current time * @return A request future whose value indicates whether the commit was successful or not */ - public RequestFuture commitOffsets(final Map offsets, long now) { - final RequestFuture future = newCoordinatorRequestFuture(now); - if (future.isDone()) return future; + private RequestFuture sendOffsetCommitRequest(final Map offsets) { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); - if (offsets.isEmpty()) { - future.complete(null); - } else { - // create the offset commit request - Map offsetData; - offsetData = new HashMap(offsets.size()); - for (Map.Entry entry : offsets.entrySet()) - offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), "")); - OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, + if (offsets.isEmpty()) + return RequestFuture.voidSuccess(); + + // create the offset commit request + Map offsetData; + offsetData = new HashMap(offsets.size()); + for (Map.Entry entry : offsets.entrySet()) + offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), "")); + OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, this.generation, this.consumerId, OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - RequestCompletionHandler handler = new OffsetCommitCompletionHandler(offsets, future); - sendCoordinator(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); - } - - return future; - } - - private RequestFuture newCoordinatorRequestFuture(long now) { - if (coordinatorUnknown()) - return RequestFuture.newCoordinatorNeeded(); - - if (client.ready(this.consumerCoordinator, now)) - // We have an open connection and we're ready to send - return new RequestFuture(); - - if (this.client.connectionFailed(this.consumerCoordinator)) { - coordinatorDead(); - return RequestFuture.newCoordinatorNeeded(); - } - - // The connection has been initiated, so we need to poll to finish it - return RequestFuture.pollNeeded(); + return sendCoordinator(ApiKeys.OFFSET_COMMIT, req) + .compose(new OffsetCommitResponseHandler(offsets)); } /** @@ -246,35 +395,30 @@ public final class Coordinator { * returned future can be polled to get the actual offsets returned from the broker. * * @param partitions The set of partitions to get offsets for. - * @param now The current time in milliseconds * @return A request future containing the committed offsets. */ - public RequestFuture> fetchOffsets(Set partitions, long now) { - final RequestFuture> future = newCoordinatorRequestFuture(now); - if (future.isDone()) return future; + private RequestFuture> sendOffsetFetchRequest(Set partitions) { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); - log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); + log.debug("Fetching committed offsets for partitions: {}", Utils.join(partitions, ", ")); // construct the request OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); // send the request with a callback - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - handleOffsetFetchResponse(resp, future); - } - }; - sendCoordinator(ApiKeys.OFFSET_FETCH, request.toStruct(), completionHandler, now); - return future; + return sendCoordinator(ApiKeys.OFFSET_FETCH, request) + .compose(new OffsetFetchResponseHandler()); } - private void handleOffsetFetchResponse(ClientResponse resp, RequestFuture> future) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); - future.retryWithNewCoordinator(); - } else { - // parse the response to get the offsets - OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); + private class OffsetFetchResponseHandler extends CoordinatorResponseHandler> { + + @Override + public OffsetFetchResponse parse(ClientResponse response) { + return new OffsetFetchResponse(response.responseBody()); + } + + @Override + public void handle(OffsetFetchResponse response, RequestFuture> future) { Map offsets = new HashMap(response.responseData().size()); for (Map.Entry entry : response.responseData().entrySet()) { TopicPartition tp = entry.getKey(); @@ -285,19 +429,21 @@ public final class Coordinator { .getMessage()); if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { // just retry - future.retryAfterBackoff(); + future.raise(Errors.OFFSET_LOAD_IN_PROGRESS); } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { // re-discover the coordinator and retry coordinatorDead(); - future.retryWithNewCoordinator(); + future.raise(Errors.NOT_COORDINATOR_FOR_CONSUMER); } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code() || data.errorCode == Errors.ILLEGAL_GENERATION.code()) { // need to re-join group subscriptions.needReassignment(); + future.raise(Errors.forCode(data.errorCode)); } else { future.raise(new KafkaException("Unexpected error in fetch offset response: " + Errors.forCode(data.errorCode).exception().getMessage())); } + return; } else if (data.offset >= 0) { // record the position with the offset (-1 indicates no committed offset to fetch) offsets.put(tp, data.offset); @@ -306,82 +452,47 @@ public final class Coordinator { } } - if (!future.isDone()) - future.complete(offsets); + future.complete(offsets); } } /** - * Attempt to heartbeat the consumer coordinator if necessary, and check if the coordinator is still alive. - * - * @param now The current time - */ - public void maybeHeartbeat(long now) { - if (heartbeat.shouldHeartbeat(now) && coordinatorReady(now)) { - HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); - sendCoordinator(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now); - this.heartbeat.sentHeartbeat(now); - } - } - - /** - * Get the time until the next heartbeat is needed. - * @param now The current time - * @return The duration in milliseconds before the next heartbeat will be needed. - */ - public long timeToNextHeartbeat(long now) { - return heartbeat.timeToNextHeartbeat(now); - } - - /** - * Check whether the coordinator has any in-flight requests. - * @return true if the coordinator has pending requests. + * Send a heartbeat request now (visible only for testing). */ - public boolean hasInFlightRequests() { - return !coordinatorUnknown() && client.inFlightRequestCount(consumerCoordinator.idString()) > 0; + public RequestFuture sendHeartbeatRequest() { + HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); + return sendCoordinator(ApiKeys.HEARTBEAT, req) + .compose(new HeartbeatCompletionHandler()); } public boolean coordinatorUnknown() { return this.consumerCoordinator == null; } - private boolean coordinatorReady(long now) { - return !coordinatorUnknown() && this.client.ready(this.consumerCoordinator, now); - } - /** * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to * one of the brokers. The returned future should be polled to get the result of the request. * @return A request future which indicates the completion of the metadata request */ - public RequestFuture discoverConsumerCoordinator() { + private RequestFuture sendConsumerMetadataRequest() { // initiate the consumer metadata request // find a node to ask about the coordinator - long now = time.milliseconds(); - Node node = this.client.leastLoadedNode(now); - + Node node = this.client.leastLoadedNode(); if (node == null) { - return RequestFuture.metadataRefreshNeeded(); - } else if (!this.client.ready(node, now)) { - if (this.client.connectionFailed(node)) { - return RequestFuture.metadataRefreshNeeded(); - } else { - return RequestFuture.pollNeeded(); - } + // TODO: If there are no brokers left, we should use the bootstrap set + // from configuration? + return RequestFuture.noBrokersAvailable(); } else { - final RequestFuture future = new RequestFuture(); - // create a consumer metadata request log.debug("Issuing consumer metadata request to broker {}", node.id()); ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId); - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - handleConsumerMetadataResponse(resp, future); - } - }; - send(node, ApiKeys.CONSUMER_METADATA, metadataRequest.toStruct(), completionHandler, now); - return future; + return client.send(node, ApiKeys.CONSUMER_METADATA, metadataRequest) + .compose(new RequestFutureAdapter() { + @Override + public void onSuccess(ClientResponse response, RequestFuture future) { + handleConsumerMetadataResponse(response, future); + } + }); } } @@ -391,7 +502,10 @@ public final class Coordinator { // parse the response to get the coordinator info if it is not disconnected, // otherwise we need to request metadata update if (resp.wasDisconnected()) { - future.retryAfterMetadataRefresh(); + future.raise(new DisconnectException()); + } else if (!coordinatorUnknown()) { + // We already found the coordinator, so ignore the request + future.complete(null); } else { ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody()); // use MAX_VALUE - node.id as the coordinator id to mimic separate connections @@ -401,9 +515,10 @@ public final class Coordinator { this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), consumerMetadataResponse.node().host(), consumerMetadataResponse.node().port()); + heartbeat.resetSession(time.milliseconds()); future.complete(null); } else { - future.retryAfterBackoff(); + future.raise(Errors.forCode(consumerMetadataResponse.errorCode())); } } } @@ -432,98 +547,130 @@ public final class Coordinator { coordinatorDead(); } - - private void sendCoordinator(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { - send(this.consumerCoordinator, api, request, handler, now); + private RequestFuture sendCoordinator(ApiKeys api, AbstractRequest request) { + return client.send(this.consumerCoordinator, api, request); } - private void send(Node node, ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { - RequestHeader header = this.client.nextRequestHeader(api); - RequestSend send = new RequestSend(node.idString(), header, request); - this.client.send(new ClientRequest(now, true, send, handler)); - } - private class HeartbeatCompletionHandler implements RequestCompletionHandler { + + private class HeartbeatCompletionHandler extends Coordinator.CoordinatorResponseHandler { + @Override + public HeartbeatResponse parse(ClientResponse response) { + return new HeartbeatResponse(response.responseBody()); + } + @Override - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); + public void handle(HeartbeatResponse heartbeatResponse, RequestFuture future) { + sensors.heartbeatLatency.record(response.requestLatencyMs()); + short error = heartbeatResponse.errorCode(); + if (error == Errors.NONE.code()) { + log.debug("Received successful heartbeat response."); + future.complete(null); + } else if (error == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || error == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); + coordinatorDead(); + future.raise(Errors.forCode(error)); + } else if (error == Errors.ILLEGAL_GENERATION.code()) { + log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group."); + subscriptions.needReassignment(); + future.raise(Errors.ILLEGAL_GENERATION); + } else if (error == Errors.UNKNOWN_CONSUMER_ID.code()) { + log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group."); + consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + subscriptions.needReassignment(); + future.raise(Errors.UNKNOWN_CONSUMER_ID); } else { - HeartbeatResponse response = new HeartbeatResponse(resp.responseBody()); - if (response.errorCode() == Errors.NONE.code()) { - log.debug("Received successful heartbeat response."); - } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); - coordinatorDead(); - } else if (response.errorCode() == Errors.ILLEGAL_GENERATION.code()) { - log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group."); - subscriptions.needReassignment(); - } else if (response.errorCode() == Errors.UNKNOWN_CONSUMER_ID.code()) { - log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group."); - consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; - subscriptions.needReassignment(); - } else { - throw new KafkaException("Unexpected error in heartbeat response: " - + Errors.forCode(response.errorCode()).exception().getMessage()); - } + future.raise(new KafkaException("Unexpected error in heartbeat response: " + + Errors.forCode(error).exception().getMessage())); } - sensors.heartbeatLatency.record(resp.requestLatencyMs()); } } - private class OffsetCommitCompletionHandler implements RequestCompletionHandler { + private class OffsetCommitResponseHandler extends CoordinatorResponseHandler { private final Map offsets; - private final RequestFuture future; - public OffsetCommitCompletionHandler(Map offsets, RequestFuture future) { + public OffsetCommitResponseHandler(Map offsets) { this.offsets = offsets; - this.future = future; } @Override - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); - future.retryWithNewCoordinator(); - } else { - OffsetCommitResponse commitResponse = new OffsetCommitResponse(resp.responseBody()); - for (Map.Entry entry : commitResponse.responseData().entrySet()) { - TopicPartition tp = entry.getKey(); - short errorCode = entry.getValue(); - long offset = this.offsets.get(tp); - if (errorCode == Errors.NONE.code()) { - log.debug("Committed offset {} for partition {}", offset, tp); - subscriptions.committed(tp, offset); - } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - coordinatorDead(); - future.retryWithNewCoordinator(); - } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code() - || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) { - // do not need to throw the exception but just log the error - log.error("Error committing partition {} at offset {}: {}", - tp, - offset, - Errors.forCode(errorCode).exception().getMessage()); - } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code() - || errorCode == Errors.ILLEGAL_GENERATION.code()) { - // need to re-join group - subscriptions.needReassignment(); - } else { - // re-throw the exception as these should not happen - log.error("Error committing partition {} at offset {}: {}", - tp, - offset, - Errors.forCode(errorCode).exception().getMessage()); - } + public OffsetCommitResponse parse(ClientResponse response) { + return new OffsetCommitResponse(response.responseBody()); + } + + @Override + public void handle(OffsetCommitResponse commitResponse, RequestFuture future) { + sensors.commitLatency.record(response.requestLatencyMs()); + + for (Map.Entry entry : commitResponse.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + short errorCode = entry.getValue(); + long offset = this.offsets.get(tp); + if (errorCode == Errors.NONE.code()) { + log.debug("Committed offset {} for partition {}", offset, tp); + subscriptions.committed(tp, offset); + } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + future.raise(Errors.forCode(errorCode)); + return; + } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code() + || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) { + // do not need to throw the exception but just log the error + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); + future.raise(Errors.forCode(errorCode)); + return; + } else { + // do not need to throw the exception but just log the error + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); } + } + + future.complete(null); + } + } - if (!future.isDone()) - future.complete(null); + private abstract class CoordinatorResponseHandler + extends RequestFutureAdapter { + protected ClientResponse response; + + public abstract R parse(ClientResponse response); + + public abstract void handle(R response, RequestFuture future); + + @Override + public void onSuccess(ClientResponse value, RequestFuture future) { + this.response = value; + + if (value.wasDisconnected()) { + handleCoordinatorDisconnect(value); + future.raise(new DisconnectException()); + return; } - sensors.commitLatency.record(resp.requestLatencyMs()); + + // TODO: We can handle general coordinator errors here, but we + // need an AbstractResponse which gives us access to the error code + R response = parse(value); + handle(response, future); + } + + @Override + public void onFailure(RuntimeException e, RequestFuture future) { + log.debug("Coordinator request failed", e); + coordinatorDead(); + future.raise(e); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java new file mode 100644 index 0000000..830f869 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java @@ -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; + + +public interface DelayedTask { + + /** + * Execute the task. + * @param now current time in milliseconds + * @return true if the task completed successfully, false otherwise + */ + void run(long now); +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java new file mode 100644 index 0000000..91cf422 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java @@ -0,0 +1,82 @@ +/** + * 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.PriorityQueue; + +/** + * Tracks a set of tasks to be executed after a delay. + */ +public class DelayedTaskQueue { + + private PriorityQueue tasks; + + public DelayedTaskQueue() { + tasks = new PriorityQueue(); + } + + /** + * Schedule a task for execution in the future. + * + * @param task the task to execute + * @param at the time at which to + */ + public void add(DelayedTask task, long at) { + tasks.add(new Entry(task, at)); + } + + /** + * Get amount of time in milliseconds until the next event. Returns Long.MAX_VALUE if no tasks are scheduled. + * + * @return the remaining time in milliseconds + */ + public long nextTimeout(long now) { + if (tasks.isEmpty()) + return Long.MAX_VALUE; + else + return Math.max(tasks.peek().timeout - now, 0); + } + + /** + * Run any ready tasks. + * + * @param now the current time + */ + public void poll(long now) { + while (!tasks.isEmpty() && tasks.peek().timeout <= now) { + Entry entry = tasks.poll(); + entry.task.run(now); + } + } + + private static class Entry implements Comparable { + DelayedTask task; + long timeout; + + public Entry(DelayedTask task, long timeout) { + this.task = task; + this.timeout = timeout; + } + + @Override + public int compareTo(Entry entry) { + if (this.timeout < entry.timeout) + return -1; + else if (this.timeout > entry.timeout) + return 1; + else + return 0; + } + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 695eaf6..4ef57ea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -13,17 +13,18 @@ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.Metadata; -import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.InvalidMetadataException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; @@ -38,7 +39,6 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.ListOffsetRequest; import org.apache.kafka.common.requests.ListOffsetResponse; -import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -52,21 +52,24 @@ import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; /** * This class manage the fetching process with the brokers. */ public class Fetcher { + private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; + private static final long LATEST_OFFSET_TIMESTAMP = -1L; private static final Logger log = LoggerFactory.getLogger(Fetcher.class); - private final KafkaClient client; - + private final ConsumerNetworkClient client; private final Time time; private final int minBytes; private final int maxWaitMs; private final int fetchSize; + private final long retryBackoffMs; private final boolean checkCrcs; private final Metadata metadata; private final FetchManagerMetrics sensors; @@ -75,8 +78,7 @@ public class Fetcher { private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; - - public Fetcher(KafkaClient client, + public Fetcher(ConsumerNetworkClient client, int minBytes, int maxWaitMs, int fetchSize, @@ -88,7 +90,8 @@ public class Fetcher { Metrics metrics, String metricGrpPrefix, Map metricTags, - Time time) { + Time time, + long retryBackoffMs) { this.time = time; this.client = client; @@ -105,25 +108,99 @@ public class Fetcher { this.records = new LinkedList>(); this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags); + this.retryBackoffMs = retryBackoffMs; } /** * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one. * * @param cluster The current cluster metadata - * @param now The current time */ - public void initFetches(Cluster cluster, long now) { - for (ClientRequest request : createFetchRequests(cluster)) { - Node node = cluster.nodeById(Integer.parseInt(request.request().destination())); - if (client.ready(node, now)) { - log.trace("Initiating fetch to node {}: {}", node.id(), request); - client.send(request); + public void initFetches(Cluster cluster) { + for (Map.Entry fetchEntry: createFetchRequests(cluster).entrySet()) { + final FetchRequest fetch = fetchEntry.getValue(); + client.send(fetchEntry.getKey(), ApiKeys.FETCH, fetch) + .addListener(new RequestFutureListener() { + @Override + public void onSuccess(ClientResponse response) { + handleFetchResponse(response, fetch); + } + + @Override + public void onFailure(RuntimeException e) { + log.debug("Fetch failed", e); + } + }); + } + } + + /** + * + * @param partitions + */ + public void updateFetchPositions(Set partitions) { + // reset the fetch position to the committed position + for (TopicPartition tp : partitions) { + // Skip if we already have a fetch position + if (subscriptions.fetched(tp) != null) + continue; + + // TODO: If there are several offsets to reset, we could submit offset requests in parallel + if (subscriptions.isOffsetResetNeeded(tp)) { + resetOffset(tp); + } else if (subscriptions.committed(tp) == null) { + // There's no committed position, so we need to reset with the default strategy + subscriptions.needOffsetReset(tp); + resetOffset(tp); + } else { + log.debug("Resetting offset for partition {} to the committed offset {}", + tp, subscriptions.committed(tp)); + subscriptions.seek(tp, subscriptions.committed(tp)); } } } /** + * Reset offsets for the given partition using the offset reset strategy. + * + * @param partition The given partition that needs reset offset + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined + */ + private void resetOffset(TopicPartition partition) { + OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); + final long timestamp; + if (strategy == OffsetResetStrategy.EARLIEST) + timestamp = EARLIEST_OFFSET_TIMESTAMP; + else if (strategy == OffsetResetStrategy.LATEST) + timestamp = LATEST_OFFSET_TIMESTAMP; + else + throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); + + log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase()); + long offset = listOffset(partition, timestamp); + this.subscriptions.seek(partition, offset); + } + + /** + * Fetch a single offset before the given timestamp for the partition. + * + * @param partition The partition that needs fetching offset. + * @param timestamp The timestamp for fetching offset. + * @return The offset of the message that is published before the given timestamp + */ + private long listOffset(TopicPartition partition, long timestamp) { + while (true) { + RequestFuture future = sendListOffsetRequest(partition, timestamp); + client.poll(future); + + if (future.succeeded()) + return future.value(); + + handleRequestFailure(future); + } + } + + /** * Return the fetched records, empty the record buffer and update the consumed position. * * @return The fetched records per partition @@ -163,37 +240,38 @@ public class Fetcher { * @param timestamp The timestamp for fetching offset. * @return A response which can be polled to obtain the corresponding offset. */ - public RequestFuture listOffset(final TopicPartition topicPartition, long timestamp) { + private RequestFuture sendListOffsetRequest(final TopicPartition topicPartition, long timestamp) { Map partitions = new HashMap(1); partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); - long now = time.milliseconds(); PartitionInfo info = metadata.fetch().partition(topicPartition); if (info == null) { metadata.add(topicPartition.topic()); log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); - return RequestFuture.metadataRefreshNeeded(); + return RequestFuture.staleMetadata(); } else if (info.leader() == null) { log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); - return RequestFuture.metadataRefreshNeeded(); - } else if (this.client.ready(info.leader(), now)) { - final RequestFuture future = new RequestFuture(); + return RequestFuture.leaderNotAvailable(); + } else { Node node = info.leader(); ListOffsetRequest request = new ListOffsetRequest(-1, partitions); - RequestSend send = new RequestSend(node.idString(), - this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS), - request.toStruct()); - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - handleListOffsetResponse(topicPartition, resp, future); - } - }; - ClientRequest clientRequest = new ClientRequest(now, true, send, completionHandler); - this.client.send(clientRequest); - return future; + return client.send(node, ApiKeys.LIST_OFFSETS, request) + .compose(new RequestFutureAdapter() { + @Override + public void onSuccess(ClientResponse response, RequestFuture future) { + handleListOffsetResponse(topicPartition, response, future); + } + }); + } + } + + private void handleRequestFailure(RequestFuture future) { + if (!future.isRetriable()) + throw future.exception(); + + if (future.exception() instanceof InvalidMetadataException) { + client.awaitMetadataUpdate(); } else { - // We initiated a connect to the leader, but we need to poll to finish it. - return RequestFuture.pollNeeded(); + Utils.sleep(retryBackoffMs); } } @@ -206,7 +284,7 @@ public class Fetcher { ClientResponse clientResponse, RequestFuture future) { if (clientResponse.wasDisconnected()) { - future.retryAfterMetadataRefresh(); + future.raise(new DisconnectException()); } else { ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody()); short errorCode = lor.responseData().get(topicPartition).errorCode; @@ -222,11 +300,11 @@ public class Fetcher { || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", topicPartition); - future.retryAfterMetadataRefresh(); + future.raise(Errors.forCode(errorCode)); } else { log.error("Attempt to fetch offsets for partition {} failed due to: {}", topicPartition, Errors.forCode(errorCode).exception().getMessage()); - future.retryAfterMetadataRefresh(); + future.raise(new StaleMetadataException()); } } } @@ -235,37 +313,31 @@ public class Fetcher { * Create fetch requests for all nodes for which we have assigned partitions * that have no existing requests in flight. */ - private List createFetchRequests(Cluster cluster) { + private Map createFetchRequests(Cluster cluster) { // create the fetch info - Map> fetchable = new HashMap>(); + Map> fetchable = new HashMap>(); for (TopicPartition partition : subscriptions.assignedPartitions()) { Node node = cluster.leaderFor(partition); if (node == null) { metadata.requestUpdate(); - } else if (this.client.inFlightRequestCount(node.idString()) == 0) { + } else if (this.client.pendingRequestCount(node) == 0) { // if there is a leader and no in-flight requests, issue a new fetch Map fetch = fetchable.get(node.id()); if (fetch == null) { fetch = new HashMap(); - fetchable.put(node.id(), fetch); + fetchable.put(node, fetch); } long offset = this.subscriptions.fetched(partition); fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize)); } } - // create the requests - List requests = new ArrayList(fetchable.size()); - for (Map.Entry> entry : fetchable.entrySet()) { - int nodeId = entry.getKey(); - final FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); - RequestSend send = new RequestSend(Integer.toString(nodeId), this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct()); - RequestCompletionHandler handler = new RequestCompletionHandler() { - public void onComplete(ClientResponse response) { - handleFetchResponse(response, fetch); - } - }; - requests.add(new ClientRequest(time.milliseconds(), true, send, handler)); + // Create the fetches + Map requests = new HashMap(); + for (Map.Entry> entry : fetchable.entrySet()) { + Node node = entry.getKey(); + FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); + requests.put(node, fetch); } return requests; } @@ -353,7 +425,6 @@ public class Fetcher { } } - private class FetchManagerMetrics { public final Metrics metrics; public final String metricGrpName; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java index 51eae19..73ea0b3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -13,7 +13,7 @@ package org.apache.kafka.clients.consumer.internals; /** - * A helper class for managing the heartbeat to the co-ordinator + * A helper class for managing the heartbeat to the coordinator */ public final class Heartbeat { @@ -25,6 +25,7 @@ public final class Heartbeat { private final long timeout; private long lastHeartbeatSend; + private long resetSesssionTime; public Heartbeat(long timeout, long now) { this.timeout = timeout; @@ -36,7 +37,7 @@ public final class Heartbeat { } public boolean shouldHeartbeat(long now) { - return now - lastHeartbeatSend > (1.0 / HEARTBEATS_PER_SESSION_INTERVAL) * this.timeout; + return timeToNextHeartbeat(now) == 0; } public long lastHeartbeatSend() { @@ -52,4 +53,17 @@ public final class Heartbeat { else return hbInterval - timeSinceLastHeartbeat; } + + public boolean sessionTimeoutExpired(long now) { + return now - Math.max(lastHeartbeatSend, resetSesssionTime) > timeout; + } + + public long interval() { + return timeout / HEARTBEATS_PER_SESSION_INTERVAL; + } + + public void resetSession(long now) { + this.resetSesssionTime = now; + } + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java new file mode 100644 index 0000000..0ec6017 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java @@ -0,0 +1,23 @@ +/** + * 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.common.errors.InvalidMetadataException; + +/** + * No brokers were available to complete a request. + */ +public class NoAvailableBrokersException extends InvalidMetadataException { + private static final long serialVersionUID = 1L; + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java index 13fc9af..7d3a5dc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java @@ -12,78 +12,49 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.protocol.Errors; + +import java.util.ArrayList; +import java.util.List; + /** - * Result of an asynchronous request through {@link org.apache.kafka.clients.KafkaClient}. To get the - * result of the request, you must use poll using {@link org.apache.kafka.clients.KafkaClient#poll(long, long)} - * until {@link #isDone()} returns true. Typical usage might look like this: + * Result of an asynchronous request from {@link ConsumerNetworkClient}. Use {@link ConsumerNetworkClient#poll(long)} + * (and variants) to finish a request future. Use {@link #isDone()} to check if the future is complete, and + * {@link #succeeded()} to check if the request completed successfully. Typical usage might look like this: * *

- *     RequestFuture future = sendRequest();
- *     while (!future.isDone()) {
- *         client.poll(timeout, now);
- *     }
+ *     RequestFuture future = client.send(api, request);
+ *     client.poll(future);
  *
- *     switch (future.outcome()) {
- *     case SUCCESS:
- *         // handle request success
- *         break;
- *     case NEED_RETRY:
- *         // retry after taking possible retry action
- *         break;
- *     case EXCEPTION:
- *         // handle exception
-  *     }
+ *     if (future.succeeded()) {
+ *         ClientResponse response = future.value();
+ *         // Handle response
+ *     } else {
+ *         throw future.exception();
+ *     }
  * 
* - * When {@link #isDone()} returns true, there are three possible outcomes (obtained through {@link #outcome()}): - * - *
    - *
  1. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#SUCCESS}: If the request was - * successful, then you can use {@link #value()} to obtain the result.
  2. - *
  3. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#EXCEPTION}: If an unhandled exception - * was encountered, you can use {@link #exception()} to get it.
  4. - *
  5. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#NEED_RETRY}: The request may - * not have been successful, but the failure may be ephemeral and the caller just needs to try the request again. - * In this case, use {@link #retryAction()} to determine what action should be taken (if any) before - * retrying.
  6. - *
- * * @param Return type of the result (Can be Void if there is no response) */ public class RequestFuture { - public static final RequestFuture NEED_NEW_COORDINATOR = newRetryFuture(RetryAction.FIND_COORDINATOR); - public static final RequestFuture NEED_POLL = newRetryFuture(RetryAction.POLL); - public static final RequestFuture NEED_METADATA_REFRESH = newRetryFuture(RetryAction.REFRESH_METADATA); - - public enum RetryAction { - NOOP, // Retry immediately. - POLL, // Retry after calling poll (e.g. to finish a connection) - BACKOFF, // Retry after a delay - FIND_COORDINATOR, // Find a new coordinator before retrying - REFRESH_METADATA // Refresh metadata before retrying - } - - public enum Outcome { - SUCCESS, - NEED_RETRY, - EXCEPTION - } - private Outcome outcome; - private RetryAction retryAction; + private boolean isDone = false; private T value; private RuntimeException exception; + private List> listeners = new ArrayList>(); + /** * Check whether the response is ready to be handled * @return true if the response is ready, false otherwise */ public boolean isDone() { - return outcome != null; + return isDone; } /** - * Get the value corresponding to this request (if it has one, as indicated by {@link #outcome()}). + * Get the value corresponding to this request (only available if the request succeeded) * @return the value if it exists or null */ public T value() { @@ -92,32 +63,31 @@ public class RequestFuture { /** * Check if the request succeeded; - * @return true if a value is available, false otherwise + * @return true if the request completed and was successful */ public boolean succeeded() { - return outcome == Outcome.SUCCESS; + return isDone && exception == null; } /** - * Check if the request completed failed. - * @return true if the request failed (whether or not it can be retried) + * Check if the request failed. + * @return true if the request completed with a failure */ public boolean failed() { - return outcome != Outcome.SUCCESS; + return isDone && exception != null; } /** - * Return the error from this response (assuming {@link #succeeded()} has returned false. If the - * response is not ready or if there is no retryAction, null is returned. - * @return the error if it exists or null + * Check if the request is retriable (convenience method for checking if + * the exception is an instance of {@link RetriableException}. + * @return true if it is retriable, false otherwise */ - public RetryAction retryAction() { - return retryAction; + public boolean isRetriable() { + return exception instanceof RetriableException; } /** - * Get the exception from a failed result. You should check that there is an exception - * with {@link #hasException()} before using this method. + * Get the exception from a failed result (only available if the request failed) * @return The exception if it exists or null */ public RuntimeException exception() { @@ -125,85 +95,96 @@ public class RequestFuture { } /** - * Check whether there was an exception. - * @return true if this request failed with an exception + * Complete the request successfully. After this call, {@link #succeeded()} will return true + * and the value can be obtained through {@link #value()}. + * @param value corresponding value (or null if there is none) */ - public boolean hasException() { - return outcome == Outcome.EXCEPTION; + public void complete(T value) { + this.value = value; + this.isDone = true; + fireSuccess(); } /** - * Check the outcome of the future if it is ready. - * @return the outcome or null if the future is not finished + * Raise an exception. The request will be marked as failed, and the caller can either + * handle the exception or throw it. + * @param e corresponding exception to be passed to caller */ - public Outcome outcome() { - return outcome; + public void raise(RuntimeException e) { + this.exception = e; + this.isDone = true; + fireFailure(); } /** - * The request failed, but should be retried using the provided retry action. - * @param retryAction The action that should be taken by the caller before retrying the request + * Raise an error. The request will be marked as failed. + * @param error corresponding error to be passed to caller */ - public void retry(RetryAction retryAction) { - this.outcome = Outcome.NEED_RETRY; - this.retryAction = retryAction; - } - - public void retryNow() { - retry(RetryAction.NOOP); - } - - public void retryAfterBackoff() { - retry(RetryAction.BACKOFF); + public void raise(Errors error) { + raise(error.exception()); } - public void retryWithNewCoordinator() { - retry(RetryAction.FIND_COORDINATOR); + private void fireSuccess() { + for (RequestFutureListener listener: listeners) + listener.onSuccess(value); } - public void retryAfterMetadataRefresh() { - retry(RetryAction.REFRESH_METADATA); + private void fireFailure() { + for (RequestFutureListener listener: listeners) + listener.onFailure(exception); } /** - * Complete the request successfully. After this call, {@link #succeeded()} will return true - * and the value can be obtained through {@link #value()}. - * @param value corresponding value (or null if there is none) + * Add a listener which will be notified when the future completes + * @param listener */ - public void complete(T value) { - this.outcome = Outcome.SUCCESS; - this.value = value; + public void addListener(RequestFutureListener listener) { + if (isDone) { + if (exception != null) + listener.onFailure(exception); + else + listener.onSuccess(value); + } else { + this.listeners.add(listener); + } } /** - * Raise an exception. The request will be marked as failed, and the caller can either - * handle the exception or throw it. - * @param e The exception that + * Convert from a request future of one type to another type + * @param adapter The adapter which does the conversion + * @param The type of the future adapted to + * @return The new future */ - public void raise(RuntimeException e) { - this.outcome = Outcome.EXCEPTION; - this.exception = e; + public RequestFuture compose(final RequestFutureAdapter adapter) { + return adapter.adapt(this); + } + + public static RequestFuture failure(RuntimeException e) { + RequestFuture future = new RequestFuture(); + future.raise(e); + return future; + } + + public static RequestFuture voidSuccess() { + RequestFuture future = new RequestFuture(); + future.complete(null); + return future; } - private static RequestFuture newRetryFuture(RetryAction retryAction) { - RequestFuture result = new RequestFuture(); - result.retry(retryAction); - return result; + public static RequestFuture coordinatorNotAvailable() { + return failure(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception()); } - @SuppressWarnings("unchecked") - public static RequestFuture pollNeeded() { - return (RequestFuture) NEED_POLL; + public static RequestFuture leaderNotAvailable() { + return failure(Errors.LEADER_NOT_AVAILABLE.exception()); } - @SuppressWarnings("unchecked") - public static RequestFuture metadataRefreshNeeded() { - return (RequestFuture) NEED_METADATA_REFRESH; + public static RequestFuture noBrokersAvailable() { + return failure(new NoAvailableBrokersException()); } - @SuppressWarnings("unchecked") - public static RequestFuture newCoordinatorNeeded() { - return (RequestFuture) NEED_NEW_COORDINATOR; + public static RequestFuture staleMetadata() { + return failure(new StaleMetadataException()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java new file mode 100644 index 0000000..303bd17 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java @@ -0,0 +1,43 @@ +/** + * 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; + +/** + * Adapt from a request future of one type to another. + */ +public abstract class RequestFutureAdapter implements RequestFutureListener { + + private RequestFuture toFuture = new RequestFuture(); + + public abstract void onSuccess(F value, RequestFuture future); + + public void onFailure(RuntimeException e, RequestFuture future) { + future.raise(e); + } + + public RequestFuture adapt(RequestFuture fromFuture) { + fromFuture.addListener(this); + return toFuture; + } + + @Override + public void onSuccess(F value) { + onSuccess(value, toFuture); + } + + @Override + public void onFailure(RuntimeException e) { + onFailure(e, toFuture); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java new file mode 100644 index 0000000..b39261b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java @@ -0,0 +1,23 @@ +/** + * 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; + +/** + * Listener interface to hook into RequestFuture completion. + */ +public interface RequestFutureListener { + + void onSuccess(T value); + + void onFailure(RuntimeException e); +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java new file mode 100644 index 0000000..3312a2c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java @@ -0,0 +1,27 @@ +/** + * 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.common.errors.RetriableException; + +/** + * Exception used in {@link ConsumerNetworkClient} to indicate the failure + * to transmit a request to the networking layer. This could be either because + * the client is still connecting to the given host or its send buffer is full. + */ +public class SendFailedException extends RetriableException { + public static final SendFailedException INSTANCE = new SendFailedException(); + + private static final long serialVersionUID = 1L; + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java new file mode 100644 index 0000000..09114cb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java @@ -0,0 +1,22 @@ +/** + * 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.common.errors.InvalidMetadataException; + +/** + * Thrown when metadata is old and needs to be refreshed. + */ +public class StaleMetadataException extends InvalidMetadataException { + private static final long serialVersionUID = 1L; +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java new file mode 100644 index 0000000..ba9ce82 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java @@ -0,0 +1,40 @@ +/** + * 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.common.errors; + +/** + * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has + * not yet been created. + */ +public class ConsumerCoordinatorNotAvailableException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public ConsumerCoordinatorNotAvailableException() { + super(); + } + + public ConsumerCoordinatorNotAvailableException(String message) { + super(message); + } + + public ConsumerCoordinatorNotAvailableException(String message, Throwable cause) { + super(message, cause); + } + + public ConsumerCoordinatorNotAvailableException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java b/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java new file mode 100644 index 0000000..18d61a2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java @@ -0,0 +1,39 @@ +/** + * 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.common.errors; + + +/** + * Server disconnected before a request could be completed. + */ +public class DisconnectException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public DisconnectException() { + super(); + } + + public DisconnectException(String message, Throwable cause) { + super(message, cause); + } + + public DisconnectException(String message) { + super(message); + } + + public DisconnectException(Throwable cause) { + super(cause); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java new file mode 100644 index 0000000..b6c83b4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java @@ -0,0 +1,40 @@ +/** + * 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.common.errors; + +/** + * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is + * not a coordinator for. + */ +public class NotCoordinatorForConsumerException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public NotCoordinatorForConsumerException() { + super(); + } + + public NotCoordinatorForConsumerException(String message) { + super(message); + } + + public NotCoordinatorForConsumerException(String message, Throwable cause) { + super(message, cause); + } + + public NotCoordinatorForConsumerException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java new file mode 100644 index 0000000..016506e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java @@ -0,0 +1,40 @@ +/** + * 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.common.errors; + +/** + * The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change + * for that offsets topic partition). + */ +public class OffsetLoadInProgressException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public OffsetLoadInProgressException() { + super(); + } + + public OffsetLoadInProgressException(String message) { + super(message); + } + + public OffsetLoadInProgressException(String message, Throwable cause) { + super(message, cause); + } + + public OffsetLoadInProgressException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 4c0ecc3..cd6c0db 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -53,11 +53,11 @@ public enum Errors { NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")), OFFSET_LOAD_IN_PROGRESS(14, - new ApiException("The coordinator is loading offsets and can't process requests.")), + new OffsetLoadInProgressException("The coordinator is loading offsets and can't process requests.")), CONSUMER_COORDINATOR_NOT_AVAILABLE(15, - new ApiException("The coordinator is not available.")), + new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")), NOT_COORDINATOR_FOR_CONSUMER(16, - new ApiException("This is not the correct co-ordinator for this consumer.")), + new NotCoordinatorForConsumerException("This is not the correct co-ordinator for this consumer.")), INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), RECORD_LIST_TOO_LARGE(18, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java new file mode 100644 index 0000000..8c078b5 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java @@ -0,0 +1,108 @@ +/** + * 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.ClientResponse; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.HeartbeatRequest; +import org.apache.kafka.common.requests.HeartbeatResponse; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ConsumerNetworkClientTest { + + private String topicName = "test"; + private MockTime time = new MockTime(); + private MockClient client = new MockClient(time); + private Cluster cluster = TestUtils.singletonCluster(topicName, 1); + private Node node = cluster.nodes().get(0); + private Metadata metadata = new Metadata(0, Long.MAX_VALUE); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + + @Test + public void send() { + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + RequestFuture future = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(1, consumerClient.pendingRequestCount()); + assertEquals(1, consumerClient.pendingRequestCount(node)); + assertFalse(future.isDone()); + + consumerClient.poll(future); + assertTrue(future.isDone()); + assertTrue(future.succeeded()); + + ClientResponse clientResponse = future.value(); + HeartbeatResponse response = new HeartbeatResponse(clientResponse.responseBody()); + assertEquals(Errors.NONE.code(), response.errorCode()); + } + + @Test + public void multiSend() { + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + RequestFuture future1 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + RequestFuture future2 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(2, consumerClient.pendingRequestCount()); + assertEquals(2, consumerClient.pendingRequestCount(node)); + + consumerClient.awaitPendingRequests(node); + assertTrue(future1.succeeded()); + assertTrue(future2.succeeded()); + } + + @Test + public void schedule() { + TestDelayedTask task = new TestDelayedTask(); + consumerClient.schedule(task, time.milliseconds()); + consumerClient.poll(0); + assertEquals(1, task.executions); + + consumerClient.schedule(task, time.milliseconds() + 100); + consumerClient.poll(0); + assertEquals(1, task.executions); + + time.sleep(100); + consumerClient.poll(0); + assertEquals(2, task.executions); + } + + + private HeartbeatRequest heartbeatRequest() { + return new HeartbeatRequest("group", 1, "consumerId"); + } + + private Struct heartbeatResponse(short error) { + HeartbeatResponse response = new HeartbeatResponse(error); + return response.toStruct(); + } + + private static class TestDelayedTask implements DelayedTask { + int executions = 0; + @Override + public void run(long now) { + executions++; + } + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java index d085fe5..be1633c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java @@ -18,13 +18,18 @@ package org.apache.kafka.clients.consumer.internals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.ConsumerCommitCallback; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; @@ -40,6 +45,7 @@ import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import org.junit.Before; import org.junit.Test; @@ -51,16 +57,20 @@ public class CoordinatorTest { private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); private int sessionTimeoutMs = 10; + private long retryBackoffMs = 100; + private long requestTimeoutMs = 5000; private String rebalanceStrategy = "not-matter"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); + private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); - private Coordinator coordinator = new Coordinator(client, + private Coordinator coordinator = new Coordinator(consumerClient, groupId, sessionTimeoutMs, rebalanceStrategy, @@ -68,7 +78,9 @@ public class CoordinatorTest { metrics, "consumer" + groupId, metricTags, - time); + time, + requestTimeoutMs, + retryBackoffMs); @Before public void setup() { @@ -78,81 +90,129 @@ public class CoordinatorTest { @Test public void testNormalHeartbeat() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // normal heartbeat time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.NONE.code())); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + consumerClient.poll(0, time.milliseconds()); + + assertTrue(future.isDone()); + assertTrue(future.succeeded()); } @Test public void testCoordinatorNotAvailable() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // consumer_coordinator_not_available will mark coordinator as unknown time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())); time.sleep(sessionTimeoutMs); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + consumerClient.poll(0, time.milliseconds()); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), future.exception()); assertTrue(coordinator.coordinatorUnknown()); } @Test public void testNotCoordinator() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // not_coordinator will mark coordinator as unknown time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_CONSUMER.code())); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_CONSUMER.code())); time.sleep(sessionTimeoutMs); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + consumerClient.poll(0, time.milliseconds()); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), future.exception()); assertTrue(coordinator.coordinatorUnknown()); } @Test public void testIllegalGeneration() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // illegal_generation will cause re-partition subscriptions.subscribe(topicName); subscriptions.changePartitionAssignment(Collections.singletonList(tp)); time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.ILLEGAL_GENERATION.code())); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.ILLEGAL_GENERATION.code())); time.sleep(sessionTimeoutMs); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + consumerClient.poll(0, time.milliseconds()); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.ILLEGAL_GENERATION.exception(), future.exception()); + assertTrue(subscriptions.partitionAssignmentNeeded()); + } + + @Test + public void testUnknownConsumerId() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // illegal_generation will cause re-partition + subscriptions.subscribe(topicName); + subscriptions.changePartitionAssignment(Collections.singletonList(tp)); + + time.sleep(sessionTimeoutMs); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.UNKNOWN_CONSUMER_ID.code())); + time.sleep(sessionTimeoutMs); + consumerClient.poll(0, time.milliseconds()); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.UNKNOWN_CONSUMER_ID.exception(), future.exception()); assertTrue(subscriptions.partitionAssignmentNeeded()); } @Test public void testCoordinatorDisconnect() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // coordinator disconnect will mark coordinator as unknown time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.NONE.code()), true); // return disconnected + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NONE.code()), true); // return disconnected time.sleep(sessionTimeoutMs); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + consumerClient.poll(0, time.milliseconds()); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertTrue(future.exception() instanceof DisconnectException); assertTrue(coordinator.coordinatorUnknown()); } @@ -162,13 +222,11 @@ public class CoordinatorTest { subscriptions.needReassignment(); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // normal join group client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - coordinator.assignPartitions(time.milliseconds()); - client.poll(0, time.milliseconds()); + coordinator.assignPartitions(); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); @@ -180,165 +238,157 @@ public class CoordinatorTest { subscriptions.needReassignment(); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); - assertTrue(subscriptions.partitionAssignmentNeeded()); + coordinator.ensureCoordinatorKnown(); // diconnected from original coordinator will cause re-discover and join again client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()), true); - coordinator.assignPartitions(time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(subscriptions.partitionAssignmentNeeded()); - - // rediscover the coordinator client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); - - // try assigning partitions again client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - coordinator.assignPartitions(time.milliseconds()); - client.poll(0, time.milliseconds()); + coordinator.assignPartitions(); assertFalse(subscriptions.partitionAssignmentNeeded()); - assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); } @Test public void testCommitOffsetNormal() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); - // With success flag client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - RequestFuture result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertTrue(result.isDone()); - assertTrue(result.succeeded()); - - // Without success flag - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - client.respond(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + + AtomicBoolean success = new AtomicBoolean(false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, callback(success)); + consumerClient.poll(0); + assertTrue(success.get()); } @Test - public void testCommitOffsetError() { + public void testCommitOffsetAsyncCoordinatorNotAvailable() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // async commit with coordinator not available + MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + assertTrue(coordinator.coordinatorUnknown()); - // resume + assertEquals(1, cb.invoked); + assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), cb.exception); + } + + @Test + public void testCommitOffsetAsyncNotCoordinator() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // async commit with not coordinator + MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + assertTrue(coordinator.coordinatorUnknown()); - // resume + assertEquals(1, cb.invoked); + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), cb.exception); + } + + @Test + public void testCommitOffsetAsyncDisconnected() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // async commit with not coordinator + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + + assertTrue(coordinator.coordinatorUnknown()); + assertEquals(1, cb.invoked); + assertTrue(cb.exception instanceof DisconnectException); + } + + @Test + public void testCommitOffsetSyncNotCoordinator() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); - // sync commit with not_coordinator + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - RequestFuture result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertTrue(result.isDone()); - assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); + } - // sync commit with coordinator disconnected - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + @Test + public void testCommitOffsetSyncCoordinatorNotAvailable() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); - assertEquals(0, client.poll(0, time.milliseconds()).size()); - assertTrue(result.isDone()); - assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); + } + @Test + public void testCommitOffsetSyncCoordinatorDisconnected() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); - result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertTrue(result.isDone()); - assertTrue(result.succeeded()); + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); } - @Test public void testFetchOffset() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // normal fetch client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - RequestFuture> result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertEquals(100L, (long) result.value().get(tp)); + Map offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); + assertEquals(100L, (long) offsets.get(tp)); // fetch with loading in progress client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L)); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.failed()); - assertEquals(RequestFuture.RetryAction.BACKOFF, result.retryAction()); - - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertEquals(100L, (long) result.value().get(tp)); + offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); + assertEquals(100L, (long) offsets.get(tp)); // fetch with not coordinator client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L)); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.failed()); - assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); - - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); - - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertEquals(100L, (long) result.value().get(tp)); + offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); + assertEquals(100L, (long) offsets.get(tp)); // fetch with no fetchable offsets client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.value().isEmpty()); + offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); + assertTrue(offsets.isEmpty()); // fetch with offset -1 client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.value().isEmpty()); + offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); + assertTrue(offsets.isEmpty()); } private Struct consumerMetadataResponse(Node node, short error) { @@ -366,4 +416,25 @@ public class CoordinatorTest { OffsetFetchResponse response = new OffsetFetchResponse(Collections.singletonMap(tp, data)); return response.toStruct(); } + + private ConsumerCommitCallback callback(final AtomicBoolean success) { + return new ConsumerCommitCallback() { + @Override + public void onComplete(Exception exception) { + if (exception == null) + success.set(true); + } + }; + } + + private static class MockCommitCallback implements ConsumerCommitCallback { + public int invoked = 0; + public Exception exception = null; + + @Override + public void onComplete(Exception exception) { + invoked++; + this.exception = exception; + } + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java new file mode 100644 index 0000000..3238845 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java @@ -0,0 +1,68 @@ +/** + * 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.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + +public class DelayedTaskQueueTest { + private DelayedTaskQueue scheduler = new DelayedTaskQueue(); + private ArrayList executed = new ArrayList(); + + @Test + public void testScheduling() { + // Empty scheduler + assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(0)); + scheduler.poll(0); + assertEquals(Collections.emptyList(), executed); + + TestTask task1 = new TestTask(); + TestTask task2 = new TestTask(); + TestTask task3 = new TestTask(); + scheduler.add(task1, 20); + assertEquals(20, scheduler.nextTimeout(0)); + scheduler.add(task2, 10); + assertEquals(10, scheduler.nextTimeout(0)); + scheduler.add(task3, 30); + assertEquals(10, scheduler.nextTimeout(0)); + + scheduler.poll(5); + assertEquals(Collections.emptyList(), executed); + assertEquals(5, scheduler.nextTimeout(5)); + + scheduler.poll(10); + assertEquals(Arrays.asList(task2), executed); + assertEquals(10, scheduler.nextTimeout(10)); + + scheduler.poll(20); + assertEquals(Arrays.asList(task2, task1), executed); + assertEquals(20, scheduler.nextTimeout(10)); + + scheduler.poll(30); + assertEquals(Arrays.asList(task2, task1, task3), executed); + assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(30)); + } + + private class TestTask implements DelayedTask { + @Override + public void run(long now) { + executed.add(this); + } + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 405efdc..7a4e586 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -52,6 +52,7 @@ public class FetcherTest { private int minBytes = 1; private int maxWaitMs = 0; private int fetchSize = 1000; + private long retryBackoffMs = 100; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); private Metadata metadata = new Metadata(0, Long.MAX_VALUE); @@ -60,10 +61,11 @@ public class FetcherTest { private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); - private Fetcher fetcher = new Fetcher(client, + private Fetcher fetcher = new Fetcher(consumerClient, minBytes, maxWaitMs, fetchSize, @@ -75,7 +77,8 @@ public class FetcherTest { metrics, "consumer" + groupId, metricTags, - time); + time, + retryBackoffMs); @Before public void setup() throws Exception { @@ -97,9 +100,9 @@ public class FetcherTest { subscriptions.consumed(tp, 0); // normal fetch - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); + consumerClient.poll(0); records = fetcher.fetchedRecords().get(tp); assertEquals(3, records.size()); assertEquals(4L, (long) subscriptions.fetched(tp)); // this is the next fetching position @@ -119,24 +122,24 @@ public class FetcherTest { subscriptions.consumed(tp, 0); // fetch with not leader - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L)); + consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); // fetch with unknown topic partition - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L)); + consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); // fetch with out of range subscriptions.fetched(tp, 5); - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); + consumerClient.poll(0); assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(null, subscriptions.fetched(tp)); @@ -151,9 +154,9 @@ public class FetcherTest { subscriptions.consumed(tp, 5); // fetch with out of range - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); + consumerClient.poll(0); assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(null, subscriptions.fetched(tp)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java index ee1ede0..089b2e4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java @@ -51,4 +51,19 @@ public class HeartbeatTest { assertEquals(0, heartbeat.timeToNextHeartbeat(100)); assertEquals(0, heartbeat.timeToNextHeartbeat(200)); } + + @Test + public void testSessionTimeoutExpired() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep(305); + assertTrue(heartbeat.sessionTimeoutExpired(time.milliseconds())); + } + + @Test + public void resetSession() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep(305); + heartbeat.resetSession(time.milliseconds()); + assertFalse(heartbeat.sessionTimeoutExpired(time.milliseconds())); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java new file mode 100644 index 0000000..79097ed --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java @@ -0,0 +1,57 @@ +/** + * 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.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class RequestFutureTest { + + @Test + public void composeSuccessCase() { + RequestFuture future = new RequestFuture(); + RequestFuture composed = future.compose(new RequestFutureAdapter() { + @Override + public void onSuccess(String value, RequestFuture future) { + future.complete(value.length()); + } + }); + + future.complete("hello"); + + assertTrue(composed.isDone()); + assertTrue(composed.succeeded()); + assertEquals(5, (int) composed.value()); + } + + @Test + public void composeFailureCase() { + RequestFuture future = new RequestFuture(); + RequestFuture composed = future.compose(new RequestFutureAdapter() { + @Override + public void onSuccess(String value, RequestFuture future) { + future.complete(value.length()); + } + }); + + RuntimeException e = new RuntimeException(); + future.raise(e); + + assertTrue(composed.isDone()); + assertTrue(composed.failed()); + assertEquals(e, composed.exception()); + } + +} diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 92ffb91..3c1e055 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -14,15 +14,9 @@ package kafka.api import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.clients.consumer.Consumer -import org.apache.kafka.clients.consumer.KafkaConsumer -import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback -import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.clients.consumer.CommitType +import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.TopicPartition -import org.apache.kafka.clients.consumer.NoOffsetForPartitionException import kafka.utils.{TestUtils, Logging} import kafka.server.KafkaConfig @@ -46,6 +40,8 @@ class ConsumerTest extends IntegrationTestHarness with Logging { val topic = "topic" val part = 0 val tp = new TopicPartition(topic, part) + val part2 = 1 + val tp2 = new TopicPartition(topic, part2) // configure the servers and clients this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown @@ -56,12 +52,13 @@ class ConsumerTest extends IntegrationTestHarness with Logging { this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") - + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + override def setUp() { super.setUp() // create the test topic with all the brokers as replicas - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + TestUtils.createTopic(this.zkClient, topic, 2, serverCount, this.servers) } def testSimpleConsumption() { @@ -74,6 +71,45 @@ class ConsumerTest extends IntegrationTestHarness with Logging { this.consumers(0).seek(tp, 0) consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) + + // check async commit callbacks + val commitCallback = new CountConsumerCommitCallback() + this.consumers(0).commit(CommitType.ASYNC, commitCallback) + + // shouldn't make progress until poll is invoked + Thread.sleep(10) + assertEquals(0, commitCallback.count) + awaitCommitCallback(this.consumers(0), commitCallback) + } + + def testCommitSpecifiedOffsets() { + sendRecords(5, tp) + sendRecords(7, tp2) + + this.consumers(0).subscribe(tp) + this.consumers(0).subscribe(tp2) + + // Need to poll to join the group + this.consumers(0).poll(50) + val pos1 = this.consumers(0).position(tp) + val pos2 = this.consumers(0).position(tp2) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp, 3L)), CommitType.SYNC) + assertEquals(3, this.consumers(0).committed(tp)) + intercept[NoOffsetForPartitionException] { + this.consumers(0).committed(tp2) + } + // positions should not change + assertEquals(pos1, this.consumers(0).position(tp)) + assertEquals(pos2, this.consumers(0).position(tp2)) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 5L)), CommitType.SYNC) + assertEquals(3, this.consumers(0).committed(tp)) + assertEquals(5, this.consumers(0).committed(tp2)) + + // Using async should pick up the committed changes after commit completes + val commitCallback = new CountConsumerCommitCallback() + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 7L)), CommitType.ASYNC, commitCallback) + awaitCommitCallback(this.consumers(0), commitCallback) + assertEquals(7, this.consumers(0).committed(tp2)) } def testAutoOffsetReset() { @@ -150,7 +186,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { def testPartitionReassignmentCallback() { val callback = new TestConsumerReassignmentCallback() - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) consumer0.subscribe(topic) @@ -172,6 +208,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { // this should cause another callback execution while(callback.callsToAssigned < 2) consumer0.poll(50) + assertEquals(2, callback.callsToAssigned) assertEquals(2, callback.callsToRevoked) @@ -191,9 +228,13 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } } - private def sendRecords(numRecords: Int) { + private def sendRecords(numRecords: Int): Unit = { + sendRecords(numRecords, tp) + } + + private def sendRecords(numRecords: Int, tp: TopicPartition) { val futures = (0 until numRecords).map { i => - this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) + this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), i.toString.getBytes, i.toString.getBytes)) } futures.map(_.get) } @@ -218,4 +259,18 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } } + private def awaitCommitCallback(consumer: Consumer[Array[Byte], Array[Byte]], commitCallback: CountConsumerCommitCallback): Unit = { + val startCount = commitCallback.count + val started = System.currentTimeMillis() + while (commitCallback.count == startCount && System.currentTimeMillis() - started < 10000) + this.consumers(0).poll(10000) + assertEquals(startCount + 1, commitCallback.count) + } + + private class CountConsumerCommitCallback extends ConsumerCommitCallback { + var count = 0 + + override def onComplete(exception: Exception): Unit = count += 1 + } + } \ No newline at end of file -- 2.3.2 (Apple Git-55) From b1e12741d287aa4159d7e2db03df6dd78169d005 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 8 Jul 2015 10:51:31 -0700 Subject: [PATCH 2/2] KAFKA-2123; resolve problems from rebase --- .../kafka/clients/consumer/KafkaConsumer.java | 4 +- .../clients/consumer/internals/Coordinator.java | 120 ++++++++------------- .../kafka/clients/consumer/internals/Fetcher.java | 2 +- .../consumer/internals/SubscriptionState.java | 5 +- .../common/errors/IllegalGenerationException.java | 33 ++++++ .../common/errors/UnknownConsumerIdException.java | 33 ++++++ .../org/apache/kafka/common/protocol/Errors.java | 6 +- .../consumer/internals/CoordinatorTest.java | 55 ++++++---- .../kafka/coordinator/ConsumerCoordinator.scala | 2 +- 9 files changed, 156 insertions(+), 104 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index b6b7e98..86f6899 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1061,7 +1061,7 @@ public class KafkaConsumer implements Consumer { } // get new assigned partitions from the coordinator - coordinator.assignPartitions(); + coordinator.ensureAssignment(); // execute the user's callback after rebalance log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); @@ -1085,7 +1085,7 @@ public class KafkaConsumer implements Consumer { // first refresh the committed positions in case they are not up-to-date coordinator.refreshCommittedOffsets(partitions); - // then do any lookups in case some positions are not known + // then do any offset lookups in case some positions are not known fetcher.updateFetchPositions(partitions); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index b46528f..dc1111d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -19,11 +19,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.ConsumerCoordinatorNotAvailableException; import org.apache.kafka.common.errors.DisconnectException; -import org.apache.kafka.common.errors.InvalidMetadataException; -import org.apache.kafka.common.errors.NotCoordinatorForConsumerException; -import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -34,7 +30,6 @@ import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.ConsumerMetadataRequest; import org.apache.kafka.common.requests.ConsumerMetadataResponse; import org.apache.kafka.common.requests.HeartbeatRequest; @@ -115,34 +110,46 @@ public final class Coordinator { // we only need to fetch latest committed offset from coordinator if there // is some commit process in progress, otherwise our current // committed cache is up-to-date - if (subscriptions.refreshCommitsNeeded()) { + while (subscriptions.refreshCommitsNeeded()) { + ensureCoordinatorKnown(); + ensureAssignment(); + // contact coordinator to fetch committed offsets - Map offsets = fetchCommittedOffsets(partitions); + RequestFuture> future = sendOffsetFetchRequest(partitions); + client.poll(future); - // update the position with the offsets - for (Map.Entry entry : offsets.entrySet()) { - TopicPartition tp = entry.getKey(); - this.subscriptions.committed(tp, entry.getValue()); + if (future.succeeded()) { + Map offsets = future.value(); + for (Map.Entry entry : offsets.entrySet()) { + TopicPartition tp = entry.getKey(); + this.subscriptions.committed(tp, entry.getValue()); + } + this.subscriptions.commitsRefreshed(); + } else if (!future.isRetriable()) { + throw future.exception(); } } } /** - * Block until we have received a partition assignment from the coordinator. + * Ensure that we have a valid partition assignment from the coordinator. */ - public void assignPartitions() { - ensureCoordinatorKnown(); + public void ensureAssignment() { + while (subscriptions.partitionAssignmentNeeded()) { + ensureCoordinatorKnown(); - // Ensure that there are no pending requests to the coordinator. This is important - // in particular to avoid resending a pending JoinGroup request. - client.awaitPendingRequests(this.consumerCoordinator); + // Ensure that there are no pending requests to the coordinator. This is important + // in particular to avoid resending a pending JoinGroup request. + if (client.pendingRequestCount(this.consumerCoordinator) > 0) { + client.awaitPendingRequests(this.consumerCoordinator); + continue; + } - while (subscriptions.partitionAssignmentNeeded()) { RequestFuture future = sendJoinGroupRequest(); client.poll(future); - if (future.failed()) - handleRequestFailure(future); + if (future.failed() && !future.isRetriable()) + throw future.exception(); } } @@ -155,25 +162,7 @@ public final class Coordinator { client.poll(future, requestTimeoutMs); if (future.failed()) - handleRequestFailure(future); - } - } - - /** - * Lookup the committed offsets for a set of partitions. This will block until the coordinator has - * responded to the offset fetch request. - * @param partitions List of partitions to get offsets for - * @return Map from partition to its respective offset - */ - public Map fetchCommittedOffsets(Set partitions) { - while (true) { - RequestFuture> future = sendOffsetFetchRequest(partitions); - client.poll(future); - - if (future.succeeded()) - return future.value(); - - handleRequestFailure(future); + client.awaitMetadataUpdate(); } } @@ -196,7 +185,11 @@ public final class Coordinator { DelayedTask heartbeatTask = new DelayedTask() { @Override public void run(long now) { - if (!subscriptions.partitionsAutoAssigned() || coordinatorUnknown()) { + if (!subscriptions.partitionsAutoAssigned() || + subscriptions.partitionAssignmentNeeded() || + coordinatorUnknown()) { + // No need to send if we're not using auto-assignment, or if we are + // expecting a rebalance client.schedule(this, now + interval); } else if (heartbeat.sessionTimeoutExpired(now)) { // We haven't sent a successful heartbeat in one session interval @@ -241,7 +234,8 @@ public final class Coordinator { // create the request for the coordinator log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); - return sendCoordinator(ApiKeys.JOIN_GROUP, request).compose(new JoinGroupResponseHandler()); + return client.send(consumerCoordinator, ApiKeys.JOIN_GROUP, request) + .compose(new JoinGroupResponseHandler()); } private class JoinGroupResponseHandler extends CoordinatorResponseHandler { @@ -320,6 +314,9 @@ public final class Coordinator { private void commitOffsetsSync(Map offsets, ConsumerCommitCallback callback) { while (true) { + ensureCoordinatorKnown(); + ensureAssignment(); + RequestFuture future = sendOffsetCommitRequest(offsets); client.poll(future); @@ -336,27 +333,6 @@ public final class Coordinator { callback.onComplete(future.exception()); return; } - - handleRetriableException((RetriableException) future.exception()); - } - } - - private void handleRequestFailure(RequestFuture future) { - if (!future.isRetriable()) - throw future.exception(); - - handleRetriableException((RetriableException) future.exception()); - } - - private void handleRetriableException(RetriableException e) { - if (e instanceof ConsumerCoordinatorNotAvailableException || - e instanceof NotCoordinatorForConsumerException || - e instanceof DisconnectException) { - ensureCoordinatorKnown(); - } else if (e instanceof InvalidMetadataException) { - client.awaitMetadataUpdate(); - } else { - Utils.sleep(retryBackoffMs); } } @@ -386,7 +362,7 @@ public final class Coordinator { OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - return sendCoordinator(ApiKeys.OFFSET_COMMIT, req) + return client.send(consumerCoordinator, ApiKeys.OFFSET_COMMIT, req) .compose(new OffsetCommitResponseHandler(offsets)); } @@ -406,7 +382,7 @@ public final class Coordinator { OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); // send the request with a callback - return sendCoordinator(ApiKeys.OFFSET_FETCH, request) + return client.send(consumerCoordinator, ApiKeys.OFFSET_FETCH, request) .compose(new OffsetFetchResponseHandler()); } @@ -461,7 +437,7 @@ public final class Coordinator { */ public RequestFuture sendHeartbeatRequest() { HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); - return sendCoordinator(ApiKeys.HEARTBEAT, req) + return client.send(consumerCoordinator, ApiKeys.HEARTBEAT, req) .compose(new HeartbeatCompletionHandler()); } @@ -479,7 +455,7 @@ public final class Coordinator { // find a node to ask about the coordinator Node node = this.client.leastLoadedNode(); if (node == null) { - // TODO: If there are no brokers left, we should use the bootstrap set + // TODO: If there are no brokers left, perhaps we should use the bootstrap set // from configuration? return RequestFuture.noBrokersAvailable(); } else { @@ -547,12 +523,6 @@ public final class Coordinator { coordinatorDead(); } - private RequestFuture sendCoordinator(ApiKeys api, AbstractRequest request) { - return client.send(this.consumerCoordinator, api, request); - } - - - private class HeartbeatCompletionHandler extends Coordinator.CoordinatorResponseHandler { @Override public HeartbeatResponse parse(ClientResponse response) { @@ -603,11 +573,10 @@ public final class Coordinator { @Override public void handle(OffsetCommitResponse commitResponse, RequestFuture future) { sensors.commitLatency.record(response.requestLatencyMs()); - for (Map.Entry entry : commitResponse.responseData().entrySet()) { TopicPartition tp = entry.getKey(); - short errorCode = entry.getValue(); long offset = this.offsets.get(tp); + short errorCode = entry.getValue(); if (errorCode == Errors.NONE.code()) { log.debug("Committed offset {} for partition {}", offset, tp); subscriptions.committed(tp, offset); @@ -631,6 +600,7 @@ public final class Coordinator { return; } else { // do not need to throw the exception but just log the error + future.raise(Errors.forCode(errorCode)); log.error("Error committing partition {} at offset {}: {}", tp, offset, @@ -660,8 +630,6 @@ public final class Coordinator { return; } - // TODO: We can handle general coordinator errors here, but we - // need an AbstractResponse which gives us access to the error code R response = parse(value); handle(response, future); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 4ef57ea..eeff23f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -322,7 +322,7 @@ public class Fetcher { metadata.requestUpdate(); } else if (this.client.pendingRequestCount(node) == 0) { // if there is a leader and no in-flight requests, issue a new fetch - Map fetch = fetchable.get(node.id()); + Map fetch = fetchable.get(node); if (fetch == null) { fetch = new HashMap(); fetchable.put(node, fetch); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 6837453..4d9a425 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -138,7 +138,6 @@ public class SubscriptionState { public void committed(TopicPartition tp, long offset) { this.committed.put(tp, offset); - this.needsFetchCommittedOffsets = false; } public Long committed(TopicPartition tp) { @@ -152,6 +151,10 @@ public class SubscriptionState { public boolean refreshCommitsNeeded() { return this.needsFetchCommittedOffsets; } + + public void commitsRefreshed() { + this.needsFetchCommittedOffsets = false; + } public void seek(TopicPartition tp, long offset) { fetched(tp, offset); diff --git a/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java new file mode 100644 index 0000000..d20b74a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java @@ -0,0 +1,33 @@ +/** + * 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.common.errors; + +public class IllegalGenerationException extends RetriableException { + private static final long serialVersionUID = 1L; + + public IllegalGenerationException() { + super(); + } + + public IllegalGenerationException(String message, Throwable cause) { + super(message, cause); + } + + public IllegalGenerationException(String message) { + super(message); + } + + public IllegalGenerationException(Throwable cause) { + super(cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java new file mode 100644 index 0000000..9bcbd11 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java @@ -0,0 +1,33 @@ +/** + * 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.common.errors; + +public class UnknownConsumerIdException extends RetriableException { + private static final long serialVersionUID = 1L; + + public UnknownConsumerIdException() { + super(); + } + + public UnknownConsumerIdException(String message, Throwable cause) { + super(message, cause); + } + + public UnknownConsumerIdException(String message) { + super(message); + } + + public UnknownConsumerIdException(Throwable cause) { + super(cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index cd6c0db..d6c41c1 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -57,7 +57,7 @@ public enum Errors { CONSUMER_COORDINATOR_NOT_AVAILABLE(15, new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")), NOT_COORDINATOR_FOR_CONSUMER(16, - new NotCoordinatorForConsumerException("This is not the correct co-ordinator for this consumer.")), + new NotCoordinatorForConsumerException("This is not the correct coordinator for this consumer.")), INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), RECORD_LIST_TOO_LARGE(18, @@ -69,13 +69,13 @@ public enum Errors { INVALID_REQUIRED_ACKS(21, new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")), ILLEGAL_GENERATION(22, - new ApiException("Specified consumer generation id is not valid.")), + new IllegalGenerationException("Specified consumer generation id is not valid.")), INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23, new ApiException("The request partition assignment strategy does not match that of the group.")), UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24, new ApiException("The request partition assignment strategy is unknown to the broker.")), UNKNOWN_CONSUMER_ID(25, - new ApiException("The coordinator is not aware of this consumer.")), + new UnknownConsumerIdException("The coordinator is not aware of this consumer.")), INVALID_SESSION_TIMEOUT(26, new ApiException("The session timeout is not within an acceptable range.")), COMMITTING_PARTITIONS_NOT_ASSIGNED(27, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java index be1633c..a72fb08 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java @@ -226,7 +226,7 @@ public class CoordinatorTest { // normal join group client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - coordinator.assignPartitions(); + coordinator.ensureAssignment(); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); @@ -244,7 +244,7 @@ public class CoordinatorTest { client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()), true); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - coordinator.assignPartitions(); + coordinator.ensureAssignment(); assertFalse(subscriptions.partitionAssignmentNeeded()); } @@ -356,39 +356,54 @@ public class CoordinatorTest { } @Test - public void testFetchOffset() { + public void testRefreshOffset() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorKnown(); - // normal fetch + subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - Map offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); - assertEquals(100L, (long) offsets.get(tp)); + coordinator.refreshCommittedOffsets(Collections.singleton(tp)); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } + + @Test + public void testRefreshOffsetLoadInProgress() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); - // fetch with loading in progress + subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L)); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); + coordinator.refreshCommittedOffsets(Collections.singleton(tp)); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } - offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); - assertEquals(100L, (long) offsets.get(tp)); + @Test + public void testRefreshOffsetNotCoordinatorForConsumer() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); - // fetch with not coordinator + subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L)); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); + coordinator.refreshCommittedOffsets(Collections.singleton(tp)); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } - offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); - assertEquals(100L, (long) offsets.get(tp)); - - // fetch with no fetchable offsets - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); - assertTrue(offsets.isEmpty()); + @Test + public void testRefreshOffsetWithNoFetchableOffsets() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); - // fetch with offset -1 + subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - offsets = coordinator.fetchCommittedOffsets(Collections.singleton(tp)); - assertTrue(offsets.isEmpty()); + coordinator.refreshCommittedOffsets(Collections.singleton(tp)); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(null, subscriptions.committed(tp)); } private Struct consumerMetadataResponse(Node node, short error) { diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index 476973b..9fe57ef 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -210,7 +210,7 @@ class ConsumerCoordinator(val brokerId: Int, responseCallback(Errors.UNKNOWN_CONSUMER_ID.code) } else if (!group.has(consumerId)) { responseCallback(Errors.UNKNOWN_CONSUMER_ID.code) - } else if (generationId != group.generationId) { + } else if (generationId != group.generationId || group.is(PreparingRebalance)) { responseCallback(Errors.ILLEGAL_GENERATION.code) } else { val consumer = group.get(consumerId) -- 2.3.2 (Apple Git-55)