From 77b1b6eccbfb34f48390a4205173ca6a7fc67a85 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 1 Jun 2015 15:58:19 -0700 Subject: [PATCH 1/4] KAFKA-2168; move blocking calls into KafkaConsumer to enable async wakeup() --- .../apache/kafka/clients/consumer/Consumer.java | 5 + .../clients/consumer/ConsumerWakeupException.java | 18 + .../kafka/clients/consumer/KafkaConsumer.java | 232 +++++++--- .../kafka/clients/consumer/MockConsumer.java | 9 +- .../clients/consumer/internals/Coordinator.java | 479 ++++++++------------- .../kafka/clients/consumer/internals/Fetcher.java | 153 +++---- .../consumer/internals/OffsetFetchResult.java | 39 ++ .../consumer/internals/SubscriptionState.java | 41 +- .../kafka/clients/consumer/MockConsumerTest.java | 2 +- .../consumer/internals/CoordinatorTest.java | 150 +++++-- .../clients/consumer/internals/FetcherTest.java | 24 +- .../consumer/internals/SubscriptionStateTest.java | 19 +- 12 files changed, 683 insertions(+), 488 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetchResult.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 8f587bc..fd98740 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 @@ -108,4 +108,9 @@ public interface Consumer extends Closeable { */ public void close(); + /** + * @see KafkaConsumer#wakeup() + */ + public void wakeup(); + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java new file mode 100644 index 0000000..6429a9b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java @@ -0,0 +1,18 @@ +/** + * 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; + +import org.apache.kafka.common.KafkaException; + +public class ConsumerWakeupException extends KafkaException{ +} 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 d301be4..f7060c2 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 @@ -12,44 +12,46 @@ */ package org.apache.kafka.clients.consumer; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; - 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.Coordinator; import org.apache.kafka.clients.consumer.internals.Fetcher; +import org.apache.kafka.clients.consumer.internals.OffsetFetchResult; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; -import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; -import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.InetSocketAddress; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +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; + /** * A Kafka client that consumes records from a Kafka cluster. *

@@ -298,10 +300,14 @@ import org.slf4j.LoggerFactory; * *

Multithreaded Processing

* - * The Kafka consumer is threadsafe but coarsely synchronized. All network I/O happens in the thread of the application - * making the call. We have intentionally avoided implementing a particular threading model for processing. + * The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application + * making the call. It is possible, however, to safely use {@link #wakeup()} to abort a long poll from another thread. + * In this case, a {@link ConsumerWakeupException} will be thrown from the thread invoking poll. + * *

- * This leaves several options for implementing multi-threaded processing of records. + * We have intentionally avoided implementing a particular threading model for processing. This leaves several + * options for implementing multi-threaded processing of records. + * * *

1. One Consumer Per Thread

* @@ -363,6 +369,8 @@ public class KafkaConsumer implements Consumer { private final ConsumerRebalanceCallback rebalanceCallback; private long lastCommitAttemptMs; private boolean closed = false; + private final AtomicBoolean wakeup = new AtomicBoolean(false); + /** * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -479,10 +487,10 @@ 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.subscriptions = new SubscriptionState(); + OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase()); + this.subscriptions = new SubscriptionState(offsetResetStrategy); this.coordinator = new Coordinator(this.client, config.getString(ConsumerConfig.GROUP_ID_CONFIG), - this.retryBackoffMs, config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), this.metadata, @@ -507,12 +515,10 @@ public class KafkaConsumer implements Consumer { this.valueDeserializer = valueDeserializer; } this.fetcher = new Fetcher(this.client, - this.retryBackoffMs, config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG), config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG), config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG), config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG), - config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(), this.keyDeserializer, this.valueDeserializer, this.metadata, @@ -541,7 +547,7 @@ public class KafkaConsumer implements Consumer { * then this will give the set of topics currently assigned to the consumer (which may be none if the assignment * hasn't happened yet, or the partitions are in the process of getting reassigned). */ - public synchronized Set subscriptions() { + public Set subscriptions() { return Collections.unmodifiableSet(this.subscriptions.assignedPartitions()); } @@ -560,7 +566,7 @@ public class KafkaConsumer implements Consumer { * @param topics A variable list of topics that the consumer wants to subscribe to */ @Override - public synchronized void subscribe(String... topics) { + public void subscribe(String... topics) { ensureNotClosed(); log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", ")); for (String topic : topics) @@ -577,7 +583,7 @@ public class KafkaConsumer implements Consumer { * @param partitions Partitions to incrementally subscribe to */ @Override - public synchronized void subscribe(TopicPartition... partitions) { + public void subscribe(TopicPartition... partitions) { ensureNotClosed(); log.debug("Subscribed to partitions(s): {}", Utils.join(partitions, ", ")); for (TopicPartition tp : partitions) { @@ -592,7 +598,7 @@ public class KafkaConsumer implements Consumer { * * @param topics Topics to unsubscribe from */ - public synchronized void unsubscribe(String... topics) { + public void unsubscribe(String... topics) { ensureNotClosed(); log.debug("Unsubscribed from topic(s): {}", Utils.join(topics, ", ")); // throw an exception if the topic was never subscribed to @@ -606,7 +612,7 @@ public class KafkaConsumer implements Consumer { * * @param partitions Partitions to unsubscribe from */ - public synchronized void unsubscribe(TopicPartition... partitions) { + public void unsubscribe(TopicPartition... partitions) { ensureNotClosed(); log.debug("Unsubscribed from partitions(s): {}", Utils.join(partitions, ", ")); // throw an exception if the partition was never subscribed to @@ -631,7 +637,7 @@ public class KafkaConsumer implements Consumer { * offset reset policy has been configured. */ @Override - public synchronized ConsumerRecords poll(long timeout) { + public ConsumerRecords poll(long timeout) { ensureNotClosed(); long now = time.milliseconds(); @@ -650,6 +656,11 @@ public class KafkaConsumer implements Consumer { if (!subscriptions.hasAllFetchPositions()) updateFetchPositions(this.subscriptions.missingFetchPositions(), now); + // Some partitions may need to be reset (if they had no previous offset + // or if a seek to beginning/end has been done + if (subscriptions.offsetResetNeeded()) + resetOffsets(); + // maybe autocommit position if (shouldAutoCommit(now)) commit(CommitType.ASYNC); @@ -659,13 +670,13 @@ public class KafkaConsumer implements Consumer { */ Cluster cluster = this.metadata.fetch(); fetcher.initFetches(cluster, now); - client.poll(timeout, now); + poll(timeout, now); /* * initiate a fetch request for any nodes that we just got a response from without blocking */ fetcher.initFetches(cluster, now); - client.poll(0, now); + poll(0, now); return new ConsumerRecords(fetcher.fetchedRecords()); } @@ -685,7 +696,7 @@ public class KafkaConsumer implements Consumer { * @param commitType Control whether the commit is blocking */ @Override - public synchronized void commit(final Map offsets, CommitType commitType) { + public void commit(final Map offsets, CommitType commitType) { ensureNotClosed(); log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); @@ -693,10 +704,9 @@ public class KafkaConsumer implements Consumer { this.lastCommitAttemptMs = now; // commit the offsets with the coordinator - boolean syncCommit = commitType.equals(CommitType.SYNC); - if (!syncCommit) + if (commitType == CommitType.ASYNC); this.subscriptions.needRefreshCommits(); - coordinator.commitOffsets(offsets, syncCommit, now); + commitOffsets(offsets, commitType, now); } /** @@ -709,7 +719,7 @@ public class KafkaConsumer implements Consumer { * @param commitType Whether or not the commit should block until it is acknowledged. */ @Override - public synchronized void commit(CommitType commitType) { + public void commit(CommitType commitType) { ensureNotClosed(); commit(this.subscriptions.allConsumed(), commitType); } @@ -720,7 +730,7 @@ public class KafkaConsumer implements Consumer { * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets */ @Override - public synchronized void seek(TopicPartition partition, long offset) { + public void seek(TopicPartition partition, long offset) { ensureNotClosed(); log.debug("Seeking to offset {} for partition {}", offset, partition); this.subscriptions.seek(partition, offset); @@ -729,26 +739,24 @@ public class KafkaConsumer implements Consumer { /** * Seek to the first offset for each of the given partitions */ - public synchronized void seekToBeginning(TopicPartition... partitions) { + public void seekToBeginning(TopicPartition... partitions) { ensureNotClosed(); Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() : Arrays.asList(partitions); for (TopicPartition tp : parts) { - // TODO: list offset call could be optimized by grouping by node - seek(tp, fetcher.offsetBefore(tp, EARLIEST_OFFSET_TIMESTAMP)); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); } } /** * Seek to the last offset for each of the given partitions */ - public synchronized void seekToEnd(TopicPartition... partitions) { + public void seekToEnd(TopicPartition... partitions) { ensureNotClosed(); Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() : Arrays.asList(partitions); for (TopicPartition tp : parts) { - // TODO: list offset call could be optimized by grouping by node - seek(tp, fetcher.offsetBefore(tp, LATEST_OFFSET_TIMESTAMP)); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); } } @@ -760,13 +768,16 @@ public class KafkaConsumer implements Consumer { * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is * available. */ - public synchronized long position(TopicPartition partition) { + public long position(TopicPartition partition) { ensureNotClosed(); if (!this.subscriptions.assignedPartitions().contains(partition)) throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); Long offset = this.subscriptions.consumed(partition); if (offset == null) { updateFetchPositions(Collections.singleton(partition), time.milliseconds()); + if (subscriptions.offsetResetNeeded(partition)) { + resetOffset(partition, subscriptions.partitionsToReset().get(partition)); + } return this.subscriptions.consumed(partition); } else { return offset; @@ -786,7 +797,7 @@ public class KafkaConsumer implements Consumer { * partition. */ @Override - public synchronized long committed(TopicPartition partition) { + public long committed(TopicPartition partition) { ensureNotClosed(); Set partitionsToFetch; if (subscriptions.assignedPartitions().contains(partition)) { @@ -832,10 +843,20 @@ public class KafkaConsumer implements Consumer { } @Override - public synchronized void close() { + public void close() { close(false); } + /** + * Wakeup a consumer in the middle of a long poll. This is useful when you need to interrupt the consumer in + * order to close it, commit offsets, seek to a new position, etc. + */ + @Override + public void wakeup() { + this.wakeup.set(true); + this.client.wakeup(); + } + private void close(boolean swallowException) { log.trace("Closing the Kafka consumer."); AtomicReference firstException = new AtomicReference(); @@ -862,7 +883,7 @@ public class KafkaConsumer implements Consumer { int version = this.metadata.requestUpdate(); do { long now = time.milliseconds(); - this.client.poll(this.retryBackoffMs, now); + this.poll(this.retryBackoffMs, now); } while (this.metadata.version() == version); } @@ -880,8 +901,7 @@ public class KafkaConsumer implements Consumer { } // get new assigned partitions from the coordinator - this.subscriptions.changePartitionAssignment(coordinator.assignPartitions( - new ArrayList(this.subscriptions.subscribedTopics()), now)); + assignPartitions(); // execute the user's callback after rebalance log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); @@ -911,7 +931,7 @@ public class KafkaConsumer implements Consumer { if (subscriptions.fetched(tp) == null) { if (subscriptions.committed(tp) == null) { // if the committed position is unknown reset the position - fetcher.resetOffset(tp); + subscriptions.needOffsetReset(tp); } else { log.debug("Resetting offset for partition {} to the committed offset {}", tp, subscriptions.committed(tp)); @@ -922,6 +942,59 @@ public class KafkaConsumer implements Consumer { } /** + * Reset any partitions that need it. + */ + private void resetOffsets() { + for (Map.Entry resetEntry : subscriptions.partitionsToReset().entrySet()) { + // TODO: list offset call could be optimized by grouping by node + resetOffset(resetEntry.getKey(), resetEntry.getValue()); + } + } + + /** + * Reset offsets for the given partition using the offset reset strategy. + * + * @param partition The given partition that needs reset offset + * @param strategy The strategy to use for resetting (either EARLIEST or LATEST) + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined + */ + private void resetOffset(TopicPartition partition, OffsetResetStrategy strategy) { + 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()); + this.subscriptions.seek(partition, offsetBefore(partition, timestamp)); + } + + /** + * 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 offsetBefore(TopicPartition partition, long timestamp) { + while (true) { + ensureNotClosed(); + long now = time.milliseconds(); + + Map offsets = new HashMap(); + fetcher.initOffsetFetch(partition, timestamp, offsets); + poll(this.retryBackoffMs, now); + + if (offsets.containsKey(partition)) { + return offsets.get(partition); + } + } + } + + + /** * Refresh the committed offsets for given set of partitions and update the cache */ private void refreshCommittedOffsets(Set partitions, long now) { @@ -930,7 +1003,7 @@ public class KafkaConsumer implements Consumer { // committed cache is up-to-date if (subscriptions.refreshCommitsNeeded()) { // contact coordinator to fetch committed offsets - Map offsets = coordinator.fetchOffsets(partitions, now); + Map offsets = fetchCommittedOffsets(partitions, now); // update the position with the offsets for (Map.Entry entry : offsets.entrySet()) { @@ -940,6 +1013,58 @@ public class KafkaConsumer implements Consumer { } } + private void assignPartitions() { + while (subscriptions.partitionAssignmentNeeded()) { + ensureCoordinatorKnown(); + long now = time.milliseconds(); + coordinator.assignPartitions(now); + poll(retryBackoffMs, now); + } + } + + private void ensureCoordinatorKnown() { + while (coordinator.coordinatorUnknown()) { + coordinator.discoverConsumerCoordinator(); + poll(retryBackoffMs, time.milliseconds()); + } + } + + private Map fetchCommittedOffsets(Set partitions, long now) { + while (true) { + ensureCoordinatorKnown(); + OffsetFetchResult result = new OffsetFetchResult(); + coordinator.fetchOffsets(partitions, now, result); + poll(retryBackoffMs, now); + + if (result.isReady()) + return result.offsets(); + } + } + + private void commitOffsets(Map offsets, CommitType commitType, long now) { + while (true) { + ensureCoordinatorKnown(); + if (commitType == CommitType.SYNC) { + AtomicBoolean success = new AtomicBoolean(false); + coordinator.commitOffsets(offsets, now, success); + poll(retryBackoffMs, now); + if (success.get()) return; + } else { + coordinator.commitOffsets(offsets, now, null); + return; + } + } + } + + private void poll(long timeout, long now) { + this.client.poll(timeout, now); + + if (wakeup.get()) { + wakeup.set(false); + throw new ConsumerWakeupException(); + } + } + /* * Check that the consumer hasn't been closed. */ @@ -947,4 +1072,9 @@ public class KafkaConsumer implements Consumer { if (this.closed) throw new IllegalStateException("This consumer has already been closed."); } + + + public enum OffsetResetStrategy { + LATEST, EARLIEST, NONE + } } 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 f50da82..9e14142 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 @@ -40,8 +40,8 @@ public class MockConsumer implements Consumer { private Map>> records; private boolean closed; - public MockConsumer() { - this.subscriptions = new SubscriptionState(); + public MockConsumer(KafkaConsumer.OffsetResetStrategy offsetResetStrategy) { + this.subscriptions = new SubscriptionState(offsetResetStrategy); this.partitions = new HashMap>(); this.records = new HashMap>>(); this.closed = false; @@ -175,6 +175,11 @@ public class MockConsumer implements Consumer { this.closed = true; } + @Override + public void wakeup() { + + } + private void ensureNotClosed() { if (this.closed) throw new IllegalStateException("This consumer has already been closed."); 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 fac7995..8b06493 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 @@ -31,7 +31,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.protocol.types.Struct; import org.apache.kafka.common.requests.ConsumerMetadataRequest; import org.apache.kafka.common.requests.ConsumerMetadataResponse; import org.apache.kafka.common.requests.HeartbeatRequest; @@ -55,6 +54,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; /** * This class manage the coordination process with the consumer coordinator. @@ -73,7 +73,6 @@ public final class Coordinator { private final String assignmentStrategy; private final SubscriptionState subscriptions; private final CoordinatorMetrics sensors; - private final long retryBackoffMs; private Node consumerCoordinator; private String consumerId; private int generation; @@ -83,7 +82,6 @@ public final class Coordinator { */ public Coordinator(KafkaClient client, String groupId, - long retryBackoffMs, long sessionTimeoutMs, String assignmentStrategy, Metadata metadata, @@ -101,77 +99,83 @@ public final class Coordinator { this.metadata = metadata; this.consumerCoordinator = null; this.subscriptions = subscriptions; - this.retryBackoffMs = retryBackoffMs; this.sessionTimeoutMs = sessionTimeoutMs; this.assignmentStrategy = assignmentStrategy; this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags); } - /** - * Assign partitions for the subscribed topics. - * - * @param subscribedTopics The subscribed topics list - * @param now The current time - * @return The assigned partition info - */ - public List assignPartitions(List subscribedTopics, long now) { + public void assignPartitions(final long now) { + if (coordinatorReady(now)) { + // send a join group request to the coordinator + List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); + log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); - // send a join group request to the coordinator - log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); - - // repeat processing the response until succeed or fatal error - do { JoinGroupRequest request = new JoinGroupRequest(groupId, - (int) this.sessionTimeoutMs, - subscribedTopics, - this.consumerId, - this.assignmentStrategy); - - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now); - JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); - short errorCode = response.errorCode(); - - if (errorCode == Errors.NONE.code()) { - this.consumerId = response.consumerId(); - this.generation = response.generationId(); - - // set the flag to refresh last committed offsets - this.subscriptions.needRefreshCommits(); - - log.debug("Joined group: {}", response); - - // record re-assignment time - this.sensors.partitionReassignments.record(time.milliseconds() - now); - - // return assigned partitions - return response.assignedPartitions(); - } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { - // reset the consumer id and retry immediately - this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; - log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", - groupId); - } 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(); - Utils.sleep(this.retryBackoffMs); - - log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", - groupId); - } 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 - log.error("Attempt to join group {} failed due to: {}", - groupId, Errors.forCode(errorCode).exception().getMessage()); - Errors.forCode(errorCode).maybeThrow(); - } else { - // unexpected error, throw the exception - throw new KafkaException("Unexpected error in join group response: " - + Errors.forCode(response.errorCode()).exception().getMessage()); - } - } while (true); + (int) this.sessionTimeoutMs, + subscribedTopics, + this.consumerId, + this.assignmentStrategy); + + // create the request for the coordinator + log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); + + RequestHeader header = this.client.nextRequestHeader(ApiKeys.JOIN_GROUP); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request.toStruct()); + + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + handleCoordinatorDisconnect(resp); + return; + } + + // process the response + JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); + short errorCode = response.errorCode(); + + if (errorCode == Errors.NONE.code()) { + Coordinator.this.consumerId = response.consumerId(); + Coordinator.this.generation = response.generationId(); + + // set the flag to refresh last committed offsets + subscriptions.needRefreshCommits(); + + log.debug("Joined group: {}", response); + + // record re-assignment time + sensors.partitionReassignments.record(time.milliseconds() - now); + + // update subscription partition assignment + subscriptions.changePartitionAssignment(response.assignedPartitions()); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { + // reset the consumer id and retry immediately + Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", + groupId); + } 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); + } 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 + log.error("Attempt to join group {} failed due to: {}", + groupId, Errors.forCode(errorCode).exception().getMessage()); + Errors.forCode(errorCode).maybeThrow(); + } else { + // unexpected error, throw the exception + throw new KafkaException("Unexpected error in join group response: " + + Errors.forCode(response.errorCode()).exception().getMessage()); + } + } + }; + this.client.send(new ClientRequest(now, true, send, completionHandler)); + } } /** @@ -182,11 +186,11 @@ public final class Coordinator { * the commit succeeds. * * @param offsets The list of offsets per partition that should be committed. - * @param blocking Control whether the commit is blocking * @param now The current time + * @param success Nullable parameter to track the result of the operation */ - public void commitOffsets(final Map offsets, boolean blocking, long now) { - if (!offsets.isEmpty()) { + public void commitOffsets(final Map offsets, long now, final AtomicBoolean success) { + if (!offsets.isEmpty() && coordinatorReady(now)) { // create the offset commit request Map offsetData; offsetData = new HashMap(offsets.size()); @@ -200,86 +204,91 @@ public final class Coordinator { // send request and possibly wait for response if it is blocking RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets); - - if (blocking) { - boolean done; - do { - ClientResponse response = blockingCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); - - // check for errors - done = true; - OffsetCommitResponse commitResponse = new OffsetCommitResponse(response.responseBody()); - for (short errorCode : commitResponse.responseData().values()) { - if (errorCode != Errors.NONE.code()) - done = false; - } - if (!done) { - log.debug("Error in offset commit, backing off for {} ms before retrying again.", - this.retryBackoffMs); - Utils.sleep(this.retryBackoffMs); + if (success != null) { + handler = new CommitOffsetCompletionHandler(offsets) { + @Override + public void onComplete(ClientResponse resp) { + super.onComplete(resp); + success.set(true); + + OffsetCommitResponse commitResponse = new OffsetCommitResponse(resp.responseBody()); + for (short errorCode : commitResponse.responseData().values()) { + if (errorCode != Errors.NONE.code()) + success.set(false); + } } - } while (!done); - } else { - this.client.send(initiateCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now)); + }; } + + RequestHeader header = this.client.nextRequestHeader(ApiKeys.OFFSET_COMMIT); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, req.toStruct()); + this.client.send(new ClientRequest(now, true, send, handler)); } } - /** - * Fetch the committed offsets of the given set of partitions. - * - * @param partitions The list of partitions which need to ask for committed offsets - * @param now The current time - * @return The fetched offset values - */ - public Map fetchOffsets(Set partitions, long now) { - log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); - - while (true) { + public void fetchOffsets(Set partitions, long now, final OffsetFetchResult result) { + if (coordinatorReady(now)) { + 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 and block on waiting for response - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.OFFSET_FETCH, request.toStruct(), null, now); - - // parse the response to get the offsets - boolean offsetsReady = true; - OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); - Map offsets = new HashMap(response.responseData().size()); - for (Map.Entry entry : response.responseData().entrySet()) { - TopicPartition tp = entry.getKey(); - OffsetFetchResponse.PartitionData data = entry.getValue(); - if (data.hasError()) { - log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) + // send the request with a callback + RequestHeader header = this.client.nextRequestHeader(ApiKeys.OFFSET_FETCH); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request.toStruct()); + + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + handleCoordinatorDisconnect(resp); + return; + } + + OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); + handleOffsetResponse(response, result); + } + }; + + this.client.send(new ClientRequest(now, true, send, completionHandler)); + } + } + + private void handleOffsetResponse(OffsetFetchResponse response, OffsetFetchResult result) { + // parse the response to get the offsets + boolean offsetsReady = true; + + Map offsets = new HashMap(response.responseData().size()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + OffsetFetchResponse.PartitionData data = entry.getValue(); + if (data.hasError()) { + log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) .exception() .getMessage()); - if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { - // just retry - offsetsReady = false; - Utils.sleep(this.retryBackoffMs); - } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - // re-discover the coordinator and retry - coordinatorDead(); - offsetsReady = false; - Utils.sleep(this.retryBackoffMs); - } else if (data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { - // just ignore this partition - log.debug("Unknown topic or partition for " + tp); - } else { - throw new KafkaException("Unexpected error in fetch offset response: " - + Errors.forCode(data.errorCode).exception().getMessage()); - } - } else if (data.offset >= 0) { - // record the position with the offset (-1 indicates no committed offset to fetch) - offsets.put(tp, data.offset); + if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { + // just retry + offsetsReady = false; + } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + // re-discover the coordinator and retry + coordinatorDead(); + offsetsReady = false; + } else if (data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + // just ignore this partition + log.debug("Unknown topic or partition for " + tp); } else { - log.debug("No committed offset for partition " + tp); + throw new KafkaException("Unexpected error in fetch offset response: " + + Errors.forCode(data.errorCode).exception().getMessage()); } + } else if (data.offset >= 0) { + // record the position with the offset (-1 indicates no committed offset to fetch) + offsets.put(tp, data.offset); + } else { + log.debug("No committed offset for partition " + tp); } - - if (offsetsReady) - return offsets; } + + if (offsetsReady) + result.setOffsets(offsets); } /** @@ -288,9 +297,14 @@ public final class Coordinator { * @param now The current time */ public void maybeHeartbeat(long now) { - if (heartbeat.shouldHeartbeat(now)) { + if (heartbeat.shouldHeartbeat(now) && coordinatorReady(now)) { HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); - this.client.send(initiateCoordinatorRequest(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now)); + + RequestHeader header = this.client.nextRequestHeader(ApiKeys.HEARTBEAT); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, req.toStruct()); + ClientRequest request = new ClientRequest(now, true, send, new HeartbeatCompletionHandler()); + + this.client.send(request); this.heartbeat.sentHeartbeat(now); } } @@ -299,53 +313,45 @@ public final class Coordinator { return this.consumerCoordinator == null; } - /** - * Repeatedly attempt to send a request to the coordinator until a response is received (retry if we are - * disconnected). Note that this means any requests sent this way must be idempotent. - * - * @return The response - */ - private ClientResponse blockingCoordinatorRequest(ApiKeys api, - Struct request, - RequestCompletionHandler handler, - long now) { - while (true) { - ClientRequest coordinatorRequest = initiateCoordinatorRequest(api, request, handler, now); - ClientResponse coordinatorResponse = sendAndReceive(coordinatorRequest, now); - if (coordinatorResponse.wasDisconnected()) { - handleCoordinatorDisconnect(coordinatorResponse); - Utils.sleep(this.retryBackoffMs); - } else { - return coordinatorResponse; - } - } + private boolean coordinatorReady(long now) { + return !coordinatorUnknown() && this.client.ready(this.consumerCoordinator, now); } - /** - * Ensure the consumer coordinator is known and we have a ready connection to it. - */ - private void ensureCoordinatorReady() { - while (true) { - if (this.consumerCoordinator == null) - discoverCoordinator(); - - while (true) { - boolean ready = this.client.ready(this.consumerCoordinator, time.milliseconds()); - if (ready) { - return; - } else { - log.debug("No connection to coordinator, attempting to connect."); - this.client.poll(this.retryBackoffMs, time.milliseconds()); + public void discoverConsumerCoordinator() { + // initiate the consumer metadata request + // find a node to ask about the coordinator + Node node = this.client.leastLoadedNode(time.milliseconds()); + if (this.client.ready(node, time.milliseconds())) { + // create a consumer metadata request + log.debug("Issuing consumer metadata request to broker {}", node.id()); - // if the coordinator connection has failed, we need to - // break the inner loop to re-discover the coordinator - if (this.client.connectionFailed(this.consumerCoordinator)) { - log.debug("Coordinator connection failed. Attempting to re-discover."); - coordinatorDead(); - break; + ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId); + RequestSend send = new RequestSend(node.id(), + this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), + metadataRequest.toStruct()); + + long now = time.milliseconds(); + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse response) { + // parse the response to get the coordinator info if it is not disconnected, + // otherwise we need to request metadata update + if (!response.wasDisconnected()) { + ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(response.responseBody()); + // use MAX_VALUE - node.id as the coordinator id to mimic separate connections + // for the coordinator in the underlying network client layer + // TODO: this needs to be better handled in KAFKA-1935 + if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) + consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), + consumerMetadataResponse.node().host(), + consumerMetadataResponse.node().port()); + } else { + metadata.requestUpdate(); } } - } + }; + ClientRequest request = new ClientRequest(now, true, send, completionHandler); + client.send(request); } } @@ -359,54 +365,6 @@ public final class Coordinator { } } - /** - * Keep discovering the consumer coordinator until it is found. - */ - private void discoverCoordinator() { - while (this.consumerCoordinator == null) { - log.debug("No coordinator known, attempting to discover one."); - Node coordinator = fetchConsumerCoordinator(); - - if (coordinator == null) { - log.debug("No coordinator found, backing off."); - Utils.sleep(this.retryBackoffMs); - } else { - log.debug("Found coordinator: " + coordinator); - this.consumerCoordinator = coordinator; - } - } - } - - /** - * Get the current consumer coordinator information via consumer metadata request. - * - * @return the consumer coordinator node - */ - private Node fetchConsumerCoordinator() { - - // initiate the consumer metadata request - ClientRequest request = initiateConsumerMetadataRequest(); - - // send the request and wait for its response - ClientResponse response = sendAndReceive(request, request.createdTime()); - - // parse the response to get the coordinator info if it is not disconnected, - // otherwise we need to request metadata update - if (!response.wasDisconnected()) { - ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(response.responseBody()); - // use MAX_VALUE - node.id as the coordinator id to mimic separate connections - // for the coordinator in the underlying network client layer - // TODO: this needs to be better handled in KAFKA-1935 - if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) - return new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), - consumerMetadataResponse.node().host(), - consumerMetadataResponse.node().port()); - } else { - this.metadata.requestUpdate(); - } - - return null; - } /** * Handle the case when the request gets cancelled due to coordinator disconnection. @@ -422,73 +380,6 @@ public final class Coordinator { coordinatorDead(); } - /** - * Initiate a consumer metadata request to the least loaded node. - * - * @return The created request - */ - private ClientRequest initiateConsumerMetadataRequest() { - - // find a node to ask about the coordinator - Node node = this.client.leastLoadedNode(time.milliseconds()); - while (node == null || !this.client.ready(node, time.milliseconds())) { - long now = time.milliseconds(); - this.client.poll(this.retryBackoffMs, now); - node = this.client.leastLoadedNode(now); - - // if there is no ready node, backoff before retry - if (node == null) - Utils.sleep(this.retryBackoffMs); - } - - // create a consumer metadata request - log.debug("Issuing consumer metadata request to broker {}", node.id()); - - ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.groupId); - RequestSend send = new RequestSend(node.id(), - this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), - request.toStruct()); - long now = time.milliseconds(); - return new ClientRequest(now, true, send, null); - } - - /** - * Initiate a request to the coordinator. - */ - private ClientRequest initiateCoordinatorRequest(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { - - // first make sure the coordinator is known and ready - ensureCoordinatorReady(); - - // create the request for the coordinator - log.debug("Issuing request ({}: {}) to coordinator {}", api, request, this.consumerCoordinator.id()); - - RequestHeader header = this.client.nextRequestHeader(api); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request); - return new ClientRequest(now, true, send, handler); - } - - /** - * Attempt to send a request and receive its response. - * - * @return The response - */ - private ClientResponse sendAndReceive(ClientRequest clientRequest, long now) { - - // send the request - this.client.send(clientRequest); - - // drain all responses from the destination node - List responses = this.client.completeAll(clientRequest.request().destination(), now); - if (responses.isEmpty()) { - throw new IllegalStateException("This should not happen."); - } else { - // other requests should be handled by the callback, and - // we only care about the response of the last request - return responses.get(responses.size() - 1); - } - } - private class HeartbeatCompletionHandler implements RequestCompletionHandler { @Override public void onComplete(ClientResponse resp) { 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 c5e577f..dfb99c6 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 @@ -19,7 +19,6 @@ 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.common.Cluster; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; @@ -61,9 +60,6 @@ import java.util.Map; public class Fetcher { private static final Logger log = LoggerFactory.getLogger(Fetcher.class); - private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; - private static final long LATEST_OFFSET_TIMESTAMP = -1L; - private final KafkaClient client; @@ -72,23 +68,19 @@ public class Fetcher { private final int maxWaitMs; private final int fetchSize; private final boolean checkCrcs; - private final long retryBackoffMs; private final Metadata metadata; private final FetchManagerMetrics sensors; private final SubscriptionState subscriptions; private final List> records; - private final AutoOffsetResetStrategy offsetResetStrategy; private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; public Fetcher(KafkaClient client, - long retryBackoffMs, int minBytes, int maxWaitMs, int fetchSize, boolean checkCrcs, - String offsetReset, Deserializer keyDeserializer, Deserializer valueDeserializer, Metadata metadata, @@ -102,17 +94,16 @@ public class Fetcher { this.client = client; this.metadata = metadata; this.subscriptions = subscriptions; - this.retryBackoffMs = retryBackoffMs; this.minBytes = minBytes; this.maxWaitMs = maxWaitMs; this.fetchSize = fetchSize; this.checkCrcs = checkCrcs; - this.offsetResetStrategy = AutoOffsetResetStrategy.valueOf(offsetReset); this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; this.records = new LinkedList>(); + this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags); } @@ -166,84 +157,68 @@ public class Fetcher { } /** - * 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 + * Initialize a list offset request for the provided partition. Results from this fetch can be obtained by + * calling {@link #fetchedOffsets()}. + * @param topicPartition partition to get offsets for + * @param timestamp timestamp for fetching offsets */ - public void resetOffset(TopicPartition partition) { - long timestamp; - if (this.offsetResetStrategy == AutoOffsetResetStrategy.EARLIEST) - timestamp = EARLIEST_OFFSET_TIMESTAMP; - else if (this.offsetResetStrategy == AutoOffsetResetStrategy.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, this.offsetResetStrategy.name() - .toLowerCase()); - this.subscriptions.seek(partition, offsetBefore(partition, timestamp)); - } - - /** - * Fetch a single offset before the given timestamp for the partition. - * - * @param topicPartition 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 - */ - public long offsetBefore(TopicPartition topicPartition, long timestamp) { - log.debug("Fetching offsets for partition {}.", topicPartition); + public void initOffsetFetch(final TopicPartition topicPartition, long timestamp, final Map result) { Map partitions = new HashMap(1); partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); - while (true) { - 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); - awaitMetadataUpdate(); - } else if (info.leader() == null) { - log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); - awaitMetadataUpdate(); - } else if (this.client.ready(info.leader(), now)) { - Node node = info.leader(); - ListOffsetRequest request = new ListOffsetRequest(-1, partitions); - RequestSend send = new RequestSend(node.id(), + 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); + metadata.requestUpdate(); + } else if (info.leader() == null) { + log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); + metadata.requestUpdate(); + } else if (this.client.ready(info.leader(), now)) { + Node node = info.leader(); + ListOffsetRequest request = new ListOffsetRequest(-1, partitions); + RequestSend send = new RequestSend(node.id(), this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS), request.toStruct()); - ClientRequest clientRequest = new ClientRequest(now, true, send, null); - this.client.send(clientRequest); - List responses = this.client.completeAll(node.id(), now); - if (responses.isEmpty()) - throw new IllegalStateException("This should not happen."); - ClientResponse response = responses.get(responses.size() - 1); - if (response.wasDisconnected()) { - awaitMetadataUpdate(); - } else { - ListOffsetResponse lor = new ListOffsetResponse(response.responseBody()); - short errorCode = lor.responseData().get(topicPartition).errorCode; - if (errorCode == Errors.NONE.code()) { - List offsets = lor.responseData().get(topicPartition).offsets; - if (offsets.size() != 1) - throw new IllegalStateException("This should not happen."); - long offset = offsets.get(0); - log.debug("Fetched offset {} for partition {}", offset, topicPartition); - return offset; - } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() - || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { - log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", - topicPartition); - awaitMetadataUpdate(); - } else { - log.error("Attempt to fetch offsets for partition {} failed due to: {}", - topicPartition, Errors.forCode(errorCode).exception().getMessage()); - awaitMetadataUpdate(); - } + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse response) { + handleOffsetFetchResponse(topicPartition, response, result); } + }; + ClientRequest clientRequest = new ClientRequest(now, true, send, completionHandler); + this.client.send(clientRequest); + } + + } + + /** + * Callback for the response of the list offset call above. + * @param topicPartition The partition that was fetched + * @param response The response from the server. + */ + private void handleOffsetFetchResponse(TopicPartition topicPartition, ClientResponse response, Map result) { + if (response.wasDisconnected()) { + metadata.requestUpdate(); + } else { + ListOffsetResponse lor = new ListOffsetResponse(response.responseBody()); + short errorCode = lor.responseData().get(topicPartition).errorCode; + if (errorCode == Errors.NONE.code()) { + List offsets = lor.responseData().get(topicPartition).offsets; + if (offsets.size() != 1) + throw new IllegalStateException("This should not happen."); + long offset = offsets.get(0); + log.debug("Fetched offset {} for partition {}", offset, topicPartition); + result.put(topicPartition, offset); + } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", + topicPartition); + metadata.requestUpdate(); } else { - log.debug("Leader for partition {} is not ready, retry fetching offsets", topicPartition); - client.poll(this.retryBackoffMs, now); + log.error("Attempt to fetch offsets for partition {} failed due to: {}", + topicPartition, Errors.forCode(errorCode).exception().getMessage()); + metadata.requestUpdate(); } } } @@ -327,7 +302,7 @@ public class Fetcher { } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { // TODO: this could be optimized by grouping all out-of-range partitions log.info("Fetch offset {} is out of range, resetting offset", subscriptions.fetched(tp)); - resetOffset(tp); + subscriptions.needOffsetReset(tp); } else if (partition.errorCode == Errors.UNKNOWN.code()) { log.warn("Unknown error fetching data for topic-partition {}", tp); } else { @@ -356,17 +331,6 @@ public class Fetcher { return new ConsumerRecord(partition.topic(), partition.partition(), offset, key, value); } - /* - * Request a metadata update and wait until it has occurred - */ - private void awaitMetadataUpdate() { - int version = this.metadata.requestUpdate(); - do { - long now = time.milliseconds(); - this.client.poll(this.retryBackoffMs, now); - } while (this.metadata.version() == version); - } - private static class PartitionRecords { public long fetchOffset; public TopicPartition partition; @@ -379,9 +343,6 @@ public class Fetcher { } } - private static enum AutoOffsetResetStrategy { - LATEST, EARLIEST, NONE - } private class FetchManagerMetrics { public final Metrics metrics; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetchResult.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetchResult.java new file mode 100644 index 0000000..ca00cac --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetchResult.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.clients.consumer.internals; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; +import java.util.Set; + +/** + * Used to convey results from an offset fetch to the polling thread by way of a callback. In particular, + * see the usage in {@link org.apache.kafka.clients.consumer.KafkaConsumer#fetchCommittedOffsets(Set, long)}. + */ +public class OffsetFetchResult { + private Map offsets; + + public void setOffsets(Map offsets) { + this.offsets = offsets; + } + + public boolean isReady() { + return offsets != null; + } + + public Map offsets() { + return offsets; + } + +} 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 cee7541..f25067c 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 @@ -12,14 +12,15 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; + import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.kafka.common.TopicPartition; - /** * A class for tracking the topics, partitions, and offsets for the consumer */ @@ -49,7 +50,14 @@ public class SubscriptionState { /* do we need to request the latest committed offsets from the coordinator? */ private boolean needsFetchCommittedOffsets; - public SubscriptionState() { + /* Partitions that need to be reset before fetching */ + private Map resetPartitions; + + /* Default offset reset strategy */ + private KafkaConsumer.OffsetResetStrategy offsetResetStrategy; + + public SubscriptionState(KafkaConsumer.OffsetResetStrategy offsetResetStrategy) { + this.offsetResetStrategy = offsetResetStrategy; this.subscribedTopics = new HashSet(); this.subscribedPartitions = new HashSet(); this.assignedPartitions = new HashSet(); @@ -58,6 +66,7 @@ public class SubscriptionState { this.committed = new HashMap(); this.needsPartitionAssignment = false; this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up + this.resetPartitions = new HashMap(); } public void subscribe(String topic) { @@ -102,12 +111,14 @@ public class SubscriptionState { this.committed.remove(tp); this.fetched.remove(tp); this.consumed.remove(tp); + this.resetPartitions.remove(tp); } public void clearAssignment() { this.assignedPartitions.clear(); this.committed.clear(); this.fetched.clear(); + this.consumed.clear(); this.needsPartitionAssignment = !subscribedTopics().isEmpty(); } @@ -145,6 +156,7 @@ public class SubscriptionState { public void seek(TopicPartition tp, long offset) { fetched(tp, offset); consumed(tp, offset); + resetPartitions.remove(tp); } public Set assignedPartitions() { @@ -169,6 +181,28 @@ public class SubscriptionState { return this.consumed; } + public void needOffsetReset(TopicPartition partition, KafkaConsumer.OffsetResetStrategy offsetResetStrategy) { + this.resetPartitions.put(partition, offsetResetStrategy); + this.fetched.remove(partition); + this.consumed.remove(partition); + } + + public void needOffsetReset(TopicPartition partition) { + needOffsetReset(partition, offsetResetStrategy); + } + + public boolean offsetResetNeeded(TopicPartition partition) { + return resetPartitions.containsKey(partition); + } + + public boolean offsetResetNeeded() { + return !resetPartitions.isEmpty(); + } + + public Map partitionsToReset() { + return this.resetPartitions; + } + public boolean hasAllFetchPositions() { return this.fetched.size() >= this.assignedPartitions.size(); } @@ -192,4 +226,5 @@ public class SubscriptionState { this.needsPartitionAssignment = false; } + } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index 677edd3..865b3df 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -25,7 +25,7 @@ import org.junit.Test; public class MockConsumerTest { - private MockConsumer consumer = new MockConsumer(); + private MockConsumer consumer = new MockConsumer(KafkaConsumer.OffsetResetStrategy.EARLIEST); @Test public void testSimpleMock() { 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 b06c4a7..140d000 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 @@ -17,10 +17,13 @@ package org.apache.kafka.clients.consumer.internals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -36,9 +39,11 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; import java.util.Collections; +import java.util.HashMap; 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; @@ -49,7 +54,6 @@ public class CoordinatorTest { private String topicName = "test"; private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); - private long retryBackoffMs = 0L; private long sessionTimeoutMs = 10L; private String rebalanceStrategy = "not-matter"; private MockTime time = new MockTime(); @@ -57,13 +61,12 @@ public class CoordinatorTest { private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(); + private SubscriptionState subscriptions = new SubscriptionState(KafkaConsumer.OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); private Coordinator coordinator = new Coordinator(client, groupId, - retryBackoffMs, sessionTimeoutMs, rebalanceStrategy, metadata, @@ -82,6 +85,8 @@ public class CoordinatorTest { @Test public void testNormalHeartbeat() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal heartbeat time.sleep(sessionTimeoutMs); @@ -94,6 +99,8 @@ public class CoordinatorTest { @Test public void testCoordinatorNotAvailable() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // consumer_coordinator_not_available will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -108,6 +115,8 @@ public class CoordinatorTest { @Test public void testNotCoordinator() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // not_coordinator will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -122,6 +131,8 @@ public class CoordinatorTest { @Test public void testIllegalGeneration() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // illegal_generation will cause re-partition subscriptions.subscribe(topicName); @@ -139,6 +150,8 @@ public class CoordinatorTest { @Test public void testCoordinatorDisconnect() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // coordinator disconnect will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -152,39 +165,67 @@ public class CoordinatorTest { @Test public void testNormalJoinGroup() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal join group client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - assertEquals(Collections.singletonList(tp), - coordinator.assignPartitions(Collections.singletonList(topicName), time.milliseconds())); - assertEquals(0, client.inFlightRequestCount()); + coordinator.assignPartitions(time.milliseconds()); + client.poll(0, time.milliseconds()); + + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); } @Test public void testReJoinGroup() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + assertTrue(subscriptions.partitionAssignmentNeeded()); // 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())); - assertEquals(Collections.singletonList(tp), - coordinator.assignPartitions(Collections.singletonList(topicName), time.milliseconds())); - assertEquals(0, client.inFlightRequestCount()); + coordinator.assignPartitions(time.milliseconds()); + client.poll(0, time.milliseconds()); + 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()); - // sync commit + // With success flag + AtomicBoolean success = new AtomicBoolean(false); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), true, time.milliseconds()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), success); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertTrue(success.get()); - // async commit - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + // Without success flag + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), null); client.respond(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); assertEquals(1, client.poll(0, time.milliseconds()).size()); } @@ -192,34 +233,56 @@ public class CoordinatorTest { @Test public void testCommitOffsetError() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // async commit with coordinator not available client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), null); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(coordinator.coordinatorUnknown()); // resume client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // async commit with not coordinator client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), null); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(coordinator.coordinatorUnknown()); // resume client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // sync commit with not_coordinator 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()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), true, time.milliseconds()); + AtomicBoolean success = new AtomicBoolean(false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), success); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertFalse(success.get()); // sync commit with coordinator disconnected 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), true, time.milliseconds()); + success = new AtomicBoolean(false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), success); + + assertEquals(0, client.poll(0, time.milliseconds()).size()); + assertFalse(success.get()); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + + + success = new AtomicBoolean(false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), success); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertTrue(success.get()); } @@ -227,33 +290,72 @@ public class CoordinatorTest { public void testFetchOffset() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal fetch client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + OffsetFetchResult result = new OffsetFetchResult(); + coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + client.poll(0, time.milliseconds()); + assertTrue(result.isReady()); + assertEquals(100L, (long) result.offsets().get(tp)); // fetch with loading in progress + result = new OffsetFetchResult(); client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L)); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + + coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + client.poll(0, time.milliseconds()); + assertFalse(result.isReady()); + + coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + client.poll(0, time.milliseconds()); + assertTrue(result.isReady()); + assertEquals(100L, (long) result.offsets().get(tp)); // fetch with not coordinator + result = new OffsetFetchResult(); 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)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + + coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + client.poll(0, time.milliseconds()); + assertFalse(result.isReady()); + + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + + coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + client.poll(0, time.milliseconds()); + assertTrue(result.isReady()); + assertEquals(100L, (long) result.offsets().get(tp)); // fetch with no fetchable offsets + result = new OffsetFetchResult(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + client.poll(0, time.milliseconds()); + assertTrue(result.isReady()); + assertTrue(result.offsets().isEmpty()); // fetch with offset topic unknown + result = new OffsetFetchResult(); client.prepareResponse(offsetFetchResponse(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), "", 100L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + client.poll(0, time.milliseconds()); + assertTrue(result.isReady()); + assertTrue(result.offsets().isEmpty()); // fetch with offset -1 + result = new OffsetFetchResult(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + client.poll(0, time.milliseconds()); + assertTrue(result.isReady()); + assertTrue(result.offsets().isEmpty()); } private Struct consumerMetadataResponse(Node node, short error) { 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 4195410..5ccd08d 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 @@ -17,10 +17,12 @@ package org.apache.kafka.clients.consumer.internals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -49,29 +51,25 @@ public class FetcherTest { private String topicName = "test"; private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); - private long retryBackoffMs = 0L; private int minBytes = 1; private int maxWaitMs = 0; private int fetchSize = 1000; - private String offsetReset = "EARLIEST"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(); + private SubscriptionState subscriptions = new SubscriptionState(KafkaConsumer.OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); private Fetcher fetcher = new Fetcher(client, - retryBackoffMs, minBytes, maxWaitMs, fetchSize, true, // check crc - offsetReset, new ByteArrayDeserializer(), new ByteArrayDeserializer(), metadata, @@ -140,11 +138,11 @@ public class FetcherTest { subscriptions.fetched(tp, 5); fetcher.initFetches(cluster, time.milliseconds()); client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.prepareResponse(listOffsetResponse(Collections.singletonList(0L), Errors.NONE.code())); client.poll(0, time.milliseconds()); + assertTrue(subscriptions.offsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); - assertEquals(0L, (long) subscriptions.fetched(tp)); - assertEquals(0L, (long) subscriptions.consumed(tp)); + assertEquals(null, subscriptions.fetched(tp)); + assertEquals(null, subscriptions.consumed(tp)); } @Test @@ -157,11 +155,11 @@ public class FetcherTest { // fetch with out of range fetcher.initFetches(cluster, time.milliseconds()); client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.prepareResponse(listOffsetResponse(Collections.singletonList(0L), Errors.NONE.code())); client.poll(0, time.milliseconds()); + assertTrue(subscriptions.offsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); - assertEquals(0L, (long) subscriptions.fetched(tp)); - assertEquals(0L, (long) subscriptions.consumed(tp)); + assertEquals(null, subscriptions.fetched(tp)); + assertEquals(null, subscriptions.consumed(tp)); } private Struct fetchResponse(ByteBuffer buffer, short error, long hw) { @@ -169,9 +167,5 @@ public class FetcherTest { return response.toStruct(); } - private Struct listOffsetResponse(List offsets, short error) { - ListOffsetResponse response = new ListOffsetResponse(Collections.singletonMap(tp, new ListOffsetResponse.PartitionData(error, offsets))); - return response.toStruct(); - } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index e000cf8..ab4d1e7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -22,12 +22,13 @@ import static java.util.Arrays.asList; import java.util.Collections; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; import org.junit.Test; public class SubscriptionStateTest { - private final SubscriptionState state = new SubscriptionState(); + private final SubscriptionState state = new SubscriptionState(KafkaConsumer.OffsetResetStrategy.EARLIEST); private final TopicPartition tp0 = new TopicPartition("test", 0); private final TopicPartition tp1 = new TopicPartition("test", 1); @@ -43,7 +44,21 @@ public class SubscriptionStateTest { assertTrue(state.assignedPartitions().isEmpty()); assertAllPositions(tp0, null); } - + + @Test + public void partitionReset() { + state.subscribe(tp0); + state.seek(tp0, 5); + assertEquals(5l, (long) state.fetched(tp0)); + assertEquals(5l, (long) state.consumed(tp0)); + state.needOffsetReset(tp0); + assertTrue(state.offsetResetNeeded()); + assertTrue(state.offsetResetNeeded(tp0)); + assertEquals(null, state.fetched(tp0)); + assertEquals(null, state.consumed(tp0)); + } + + @Test public void topicSubscription() { state.subscribe("test"); assertEquals(1, state.subscribedTopics().size()); -- 2.3.2 (Apple Git-55) From fd1d0a01a3281b6bac6b4e4d33615e3dc958b646 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 2 Jun 2015 17:08:04 -0700 Subject: [PATCH 2/4] KAFKA-2168; refactored callback handling to prevent unnecessary requests --- .../kafka/clients/consumer/ConsumerRecords.java | 7 + .../clients/consumer/ConsumerWakeupException.java | 2 +- .../kafka/clients/consumer/KafkaConsumer.java | 196 ++++++++++----- .../clients/consumer/internals/BrokerResponse.java | 78 ++++++ .../clients/consumer/internals/Coordinator.java | 273 ++++++++++++--------- .../consumer/internals/CoordinatorResponse.java | 27 ++ .../consumer/internals/DelayedResponse.java | 40 +++ .../kafka/clients/consumer/internals/Fetcher.java | 41 ++-- .../consumer/internals/OffsetFetchResult.java | 39 --- .../consumer/internals/SubscriptionState.java | 2 +- .../consumer/internals/CoordinatorTest.java | 74 +++--- .../clients/consumer/internals/FetcherTest.java | 10 +- .../consumer/internals/SubscriptionStateTest.java | 4 +- 13 files changed, 509 insertions(+), 284 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResponse.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResponse.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResponse.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetchResult.java 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 1ca75f8..eb75d2e 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 @@ -27,6 +27,8 @@ import java.util.Map; * {@link Consumer#poll(long)} operation. */ public class ConsumerRecords implements Iterable> { + public static final ConsumerRecords EMPTY = + new ConsumerRecords(Collections.EMPTY_MAP); private final Map>> records; @@ -103,4 +105,9 @@ public class ConsumerRecords implements Iterable> { } } + @SuppressWarnings("unchecked") + public static ConsumerRecords empty() { + return (ConsumerRecords) EMPTY; + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java index 6429a9b..7046561 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java @@ -14,5 +14,5 @@ package org.apache.kafka.clients.consumer; import org.apache.kafka.common.KafkaException; -public class ConsumerWakeupException extends KafkaException{ +public class ConsumerWakeupException extends KafkaException { } 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 f7060c2..a245807 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 @@ -16,8 +16,10 @@ 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.Coordinator; +import org.apache.kafka.clients.consumer.internals.CoordinatorResponse; +import org.apache.kafka.clients.consumer.internals.DelayedResponse; import org.apache.kafka.clients.consumer.internals.Fetcher; -import org.apache.kafka.clients.consumer.internals.OffsetFetchResult; +import org.apache.kafka.clients.consumer.internals.BrokerResponse; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; @@ -41,7 +43,6 @@ import java.net.InetSocketAddress; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -371,6 +372,7 @@ public class KafkaConsumer implements Consumer { private boolean closed = false; private final AtomicBoolean wakeup = new AtomicBoolean(false); + private long requestTimeoutMs = 5000L; // TODO: Add this to client configuration? /** * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -640,6 +642,7 @@ public class KafkaConsumer implements Consumer { public ConsumerRecords poll(long timeout) { ensureNotClosed(); long now = time.milliseconds(); + log.debug("Entering consumer poll"); if (subscriptions.partitionsAutoAssigned()) { if (subscriptions.partitionAssignmentNeeded()) { @@ -654,7 +657,7 @@ public class KafkaConsumer implements Consumer { // 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(), now); + updateFetchPositions(this.subscriptions.missingFetchPositions()); // Some partitions may need to be reset (if they had no previous offset // or if a seek to beginning/end has been done @@ -665,20 +668,31 @@ public class KafkaConsumer implements Consumer { if (shouldAutoCommit(now)) commit(CommitType.ASYNC); - /* - * initiate any needed fetches, then block for the timeout the user specified - */ + // Poll for new data until the timeout expires Cluster cluster = this.metadata.fetch(); - fetcher.initFetches(cluster, now); - poll(timeout, now); + long remaining = timeout; + while (remaining > 0) { + // TODO: Long polling could prevent heartbeats. Probably we should move maybeHeartbeat + // into this loop and only call poll with a max timeout of the heartbeat interval - /* - * initiate a fetch request for any nodes that we just got a response from without blocking - */ - fetcher.initFetches(cluster, now); - poll(0, now); + long start = time.milliseconds(); - return new ConsumerRecords(fetcher.fetchedRecords()); + // Init any new fetches (won't resend pending fetches) + fetcher.initFetches(cluster, start); + poll(remaining, start); + + Map>> records = + fetcher.fetchedRecords(); + + // If data is available, then return it immediately + if (!records.isEmpty()) { + return new ConsumerRecords(records); + } + + remaining -= time.milliseconds() - start; + } + + return ConsumerRecords.empty(); } /** @@ -700,13 +714,12 @@ public class KafkaConsumer implements Consumer { ensureNotClosed(); log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); - long now = time.milliseconds(); - this.lastCommitAttemptMs = now; + this.lastCommitAttemptMs = time.milliseconds(); // commit the offsets with the coordinator - if (commitType == CommitType.ASYNC); + if (commitType == CommitType.ASYNC) this.subscriptions.needRefreshCommits(); - commitOffsets(offsets, commitType, now); + commitOffsets(offsets, commitType); } /** @@ -774,10 +787,10 @@ public class KafkaConsumer implements Consumer { throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); Long offset = this.subscriptions.consumed(partition); if (offset == null) { - updateFetchPositions(Collections.singleton(partition), time.milliseconds()); - if (subscriptions.offsetResetNeeded(partition)) { + updateFetchPositions(Collections.singleton(partition)); + if (subscriptions.offsetResetNeeded(partition)) resetOffset(partition, subscriptions.partitionsToReset().get(partition)); - } + return this.subscriptions.consumed(partition); } else { return offset; @@ -808,7 +821,7 @@ public class KafkaConsumer implements Consumer { } else { partitionsToFetch = Collections.singleton(partition); } - refreshCommittedOffsets(partitionsToFetch, time.milliseconds()); + refreshCommittedOffsets(partitionsToFetch); Long committed = this.subscriptions.committed(partition); if (committed == null) throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); @@ -918,17 +931,16 @@ public class KafkaConsumer implements Consumer { * or reset it using the offset reset policy the user has configured. * * @param partitions The partitions that needs updating fetch positions - * @param now The current time * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is * defined */ - private void updateFetchPositions(Set partitions, long now) { + private void updateFetchPositions(Set partitions) { // first refresh the committed positions in case they are not up-to-date - refreshCommittedOffsets(partitions, now); + refreshCommittedOffsets(partitions); // reset the fetch position to the committed position for (TopicPartition tp : partitions) { - if (subscriptions.fetched(tp) == null) { + if (!subscriptions.offsetResetNeeded(tp) && subscriptions.fetched(tp) == null) { if (subscriptions.committed(tp) == null) { // if the committed position is unknown reset the position subscriptions.needOffsetReset(tp); @@ -980,30 +992,26 @@ public class KafkaConsumer implements Consumer { */ private long offsetBefore(TopicPartition partition, long timestamp) { while (true) { - ensureNotClosed(); - long now = time.milliseconds(); - - Map offsets = new HashMap(); - fetcher.initOffsetFetch(partition, timestamp, offsets); - poll(this.retryBackoffMs, now); + BrokerResponse response = fetcher.offsetBefore(partition, timestamp); + poll(response, requestTimeoutMs); - if (offsets.containsKey(partition)) { - return offsets.get(partition); + if (response.isReady()) { + if (response.hasValue()) return response.value(); + handleBrokerResponse(response); } } } - /** * Refresh the committed offsets for given set of partitions and update the cache */ - private void refreshCommittedOffsets(Set partitions, long now) { + 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, now); + Map offsets = fetchCommittedOffsets(partitions); // update the position with the offsets for (Map.Entry entry : offsets.entrySet()) { @@ -1016,46 +1024,114 @@ public class KafkaConsumer implements Consumer { private void assignPartitions() { while (subscriptions.partitionAssignmentNeeded()) { ensureCoordinatorKnown(); - long now = time.milliseconds(); - coordinator.assignPartitions(now); - poll(retryBackoffMs, now); + CoordinatorResponse response = coordinator.assignPartitions(time.milliseconds()); + + // Block indefinitely for the join group request (which can take as long as + // a session timeout) + poll(response); + + if (response.isReady()) { + handleCoordinatorResponse(response); + } } } private void ensureCoordinatorKnown() { while (coordinator.coordinatorUnknown()) { - coordinator.discoverConsumerCoordinator(); - poll(retryBackoffMs, time.milliseconds()); + BrokerResponse response = coordinator.discoverConsumerCoordinator(); + poll(response, requestTimeoutMs); + if (response.isReady()) { + handleBrokerResponse(response); + } } } - private Map fetchCommittedOffsets(Set partitions, long now) { + private Map fetchCommittedOffsets(Set partitions) { while (true) { - ensureCoordinatorKnown(); - OffsetFetchResult result = new OffsetFetchResult(); - coordinator.fetchOffsets(partitions, now, result); - poll(retryBackoffMs, now); - - if (result.isReady()) - return result.offsets(); + long now = time.milliseconds(); + CoordinatorResponse> response = coordinator.fetchOffsets(partitions, now); + poll(response, requestTimeoutMs); + + if (response.isReady()) { + if (response.hasValue()) return response.value(); + handleCoordinatorResponse(response); + } else { + log.debug("Fetch committed offsets failed"); + } } } - private void commitOffsets(Map offsets, CommitType commitType, long now) { + private void commitOffsets(Map offsets, CommitType commitType) { while (true) { - ensureCoordinatorKnown(); - if (commitType == CommitType.SYNC) { - AtomicBoolean success = new AtomicBoolean(false); - coordinator.commitOffsets(offsets, now, success); - poll(retryBackoffMs, now); - if (success.get()) return; - } else { - coordinator.commitOffsets(offsets, now, null); - return; + long now = time.milliseconds(); + CoordinatorResponse response = coordinator.commitOffsets(offsets, now); + if (commitType == CommitType.SYNC) + poll(response, requestTimeoutMs); + else + poll(requestTimeoutMs, now); + + // Retry for both commit types if there are coordinator connection issues + if (response.isReady()) { + if (response.hasValue() && response.value()) return; + handleCoordinatorResponse(response); + } else if (commitType == CommitType.ASYNC) { + break; } } } + private void handleCoordinatorResponse(CoordinatorResponse response) { + if (response.newCoordinatorNeeded()) { + ensureCoordinatorKnown(); + } else { + handleBrokerResponse(response); + } + } + + private void handleBrokerResponse(BrokerResponse response) { + if (response.throwExceptionNeeded()) { + throw response.exception(); + } else if (response.metadataRefreshNeeded()) { + awaitMetadataUpdate(); + } else if (response.retryNeeded()) { + Utils.sleep(retryBackoffMs); + } + } + + /** + * Poll until a response is ready or timeout expires + * @param response The response to poll for + * @param timeout The time in milliseconds to wait for the response + * @param + */ + private void poll(DelayedResponse response, long timeout) { + long remaining = timeout; + do { + long start = time.milliseconds(); + poll(remaining, start); + if (response.isReady()) return; + remaining -= time.milliseconds() - start; + } while (remaining > 0 && !response.isReady()); + } + + /** + * Poll indefinitely until the response is ready. + * @param response The response to poll for. + * @param + */ + private void poll(DelayedResponse response) { + do { + long now = time.milliseconds(); + poll(-1, now); + } while (!response.isReady()); + } + + /** + * 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 poll(long timeout, long now) { this.client.poll(timeout, now); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResponse.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResponse.java new file mode 100644 index 0000000..89f2eb2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResponse.java @@ -0,0 +1,78 @@ +/** + * 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.KafkaException; + +public class BrokerResponse implements DelayedResponse { + + private T value = null; + protected boolean ready = false; + private boolean needMetadataRefresh = false; + private boolean needRetry = false; + + private boolean needThrow = false; + private KafkaException exception; + + @Override + public T value() { + return value; + } + + @Override + public boolean isReady() { + return ready; + } + + @Override + public boolean hasValue() { + return value != null; + } + + public void respond(T result) { + this.value = result; + this.ready = true; + } + + public boolean metadataRefreshNeeded() { + return needMetadataRefresh; + } + + public boolean retryNeeded() { + return needRetry; + } + + public void needMetadataRefresh() { + this.needMetadataRefresh = true; + this.ready = true; + } + + public void needRetry() { + this.needRetry = true; + this.ready = true; + } + + public void raise(KafkaException e) { + this.needThrow = true; + this.exception = e; + } + + public boolean throwExceptionNeeded() { + return needThrow; + } + + public KafkaException exception() { + return exception; + } + +} 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 8b06493..d01f130 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 @@ -54,7 +54,6 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; /** * This class manage the coordination process with the consumer coordinator. @@ -105,77 +104,87 @@ public final class Coordinator { this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags); } - public void assignPartitions(final long now) { - if (coordinatorReady(now)) { - // send a join group request to the coordinator - List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); - log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); + public CoordinatorResponse assignPartitions(final long now) { + final CoordinatorResponse response = newCoordinatorResponse(now); + if (response.isReady()) return response; - JoinGroupRequest request = new JoinGroupRequest(groupId, - (int) this.sessionTimeoutMs, - subscribedTopics, - this.consumerId, - this.assignmentStrategy); + // send a join group request to the coordinator + List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); + log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); - // create the request for the coordinator - log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); + JoinGroupRequest request = new JoinGroupRequest(groupId, + (int) this.sessionTimeoutMs, + subscribedTopics, + this.consumerId, + this.assignmentStrategy); - RequestHeader header = this.client.nextRequestHeader(ApiKeys.JOIN_GROUP); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request.toStruct()); + // create the request for the coordinator + log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); - return; - } + RequestHeader header = this.client.nextRequestHeader(ApiKeys.JOIN_GROUP); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request.toStruct()); - // process the response - JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); - short errorCode = response.errorCode(); + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + handleCoordinatorDisconnect(resp); + response.needNewCoordinator(); + return; + } - if (errorCode == Errors.NONE.code()) { - Coordinator.this.consumerId = response.consumerId(); - Coordinator.this.generation = response.generationId(); + // process the response + JoinGroupResponse joinResponse = new JoinGroupResponse(resp.responseBody()); + short errorCode = joinResponse.errorCode(); - // set the flag to refresh last committed offsets - subscriptions.needRefreshCommits(); + if (errorCode == Errors.NONE.code()) { + Coordinator.this.consumerId = joinResponse.consumerId(); + Coordinator.this.generation = joinResponse.generationId(); - log.debug("Joined group: {}", response); + // set the flag to refresh last committed offsets + subscriptions.needRefreshCommits(); - // record re-assignment time - sensors.partitionReassignments.record(time.milliseconds() - now); + log.debug("Joined group: {}", joinResponse); - // update subscription partition assignment - subscriptions.changePartitionAssignment(response.assignedPartitions()); - } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { - // reset the consumer id and retry immediately - Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; - log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", - groupId); - } 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); - } 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 - log.error("Attempt to join group {} failed due to: {}", - groupId, Errors.forCode(errorCode).exception().getMessage()); - Errors.forCode(errorCode).maybeThrow(); - } else { - // unexpected error, throw the exception - throw new KafkaException("Unexpected error in join group response: " - + Errors.forCode(response.errorCode()).exception().getMessage()); - } + // record re-assignment time + sensors.partitionReassignments.record(time.milliseconds() - now); + + // update subscription partition assignment + subscriptions.changePartitionAssignment(joinResponse.assignedPartitions()); + + response.respond(null); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { + // reset the consumer id and retry immediately + Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", + groupId); + + response.needRetry(); + } 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); + response.needNewCoordinator(); + } 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(); + log.error("Attempt to join group {} failed due to: {}", + groupId, e.getMessage()); + response.raise(e); + } else { + log.error("Unexpected error in join group response: " + + Errors.forCode(joinResponse.errorCode()).exception().getMessage()); + response.needRetry(); } - }; - this.client.send(new ClientRequest(now, true, send, completionHandler)); - } + } + }; + + this.client.send(new ClientRequest(now, true, send, completionHandler)); + return response; } /** @@ -187,10 +196,15 @@ public final class Coordinator { * * @param offsets The list of offsets per partition that should be committed. * @param now The current time - * @param success Nullable parameter to track the result of the operation + * @return A delayed response whose value indicates whether the commit was successful or not */ - public void commitOffsets(final Map offsets, long now, final AtomicBoolean success) { - if (!offsets.isEmpty() && coordinatorReady(now)) { + public CoordinatorResponse commitOffsets(final Map offsets, long now) { + final CoordinatorResponse response = newCoordinatorResponse(now); + if (response.isReady()) return response; + + if (offsets.isEmpty()) { + response.respond(true); + } else { // create the offset commit request Map offsetData; offsetData = new HashMap(offsets.size()); @@ -202,58 +216,57 @@ public final class Coordinator { OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - // send request and possibly wait for response if it is blocking - RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets); - if (success != null) { - handler = new CommitOffsetCompletionHandler(offsets) { - @Override - public void onComplete(ClientResponse resp) { - super.onComplete(resp); - success.set(true); - - OffsetCommitResponse commitResponse = new OffsetCommitResponse(resp.responseBody()); - for (short errorCode : commitResponse.responseData().values()) { - if (errorCode != Errors.NONE.code()) - success.set(false); - } - } - }; - } - + RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets, response); RequestHeader header = this.client.nextRequestHeader(ApiKeys.OFFSET_COMMIT); RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, req.toStruct()); this.client.send(new ClientRequest(now, true, send, handler)); } + + return response; + } + + private CoordinatorResponse newCoordinatorResponse(long now) { + CoordinatorResponse response = new CoordinatorResponse(); + if (coordinatorUnknown()) { + response.needNewCoordinator(); + } else if (!this.client.ready(this.consumerCoordinator, now)) { + response.needRetry(); + } + return response; } - public void fetchOffsets(Set partitions, long now, final OffsetFetchResult result) { - if (coordinatorReady(now)) { - 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 - RequestHeader header = this.client.nextRequestHeader(ApiKeys.OFFSET_FETCH); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request.toStruct()); + public CoordinatorResponse> fetchOffsets(Set partitions, long now) { + final CoordinatorResponse> response = newCoordinatorResponse(now); + if (response.isReady()) return response; - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); - return; - } + 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 + RequestHeader header = this.client.nextRequestHeader(ApiKeys.OFFSET_FETCH); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request.toStruct()); - OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); - handleOffsetResponse(response, result); + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + response.needRetry(); + handleCoordinatorDisconnect(resp); + } else { + OffsetFetchResponse offsetResponse = new OffsetFetchResponse(resp.responseBody()); + handleOffsetResponse(offsetResponse, response); } - }; + } + }; - this.client.send(new ClientRequest(now, true, send, completionHandler)); - } + this.client.send(new ClientRequest(now, true, send, completionHandler)); + + return response; } - private void handleOffsetResponse(OffsetFetchResponse response, OffsetFetchResult result) { + private void handleOffsetResponse(OffsetFetchResponse response, CoordinatorResponse> result) { // parse the response to get the offsets boolean offsetsReady = true; @@ -288,7 +301,9 @@ public final class Coordinator { } if (offsetsReady) - result.setOffsets(offsets); + result.respond(offsets); + else + result.needRetry(); } /** @@ -317,11 +332,15 @@ public final class Coordinator { return !coordinatorUnknown() && this.client.ready(this.consumerCoordinator, now); } - public void discoverConsumerCoordinator() { + public BrokerResponse discoverConsumerCoordinator() { // initiate the consumer metadata request // find a node to ask about the coordinator + final BrokerResponse response = new BrokerResponse(); Node node = this.client.leastLoadedNode(time.milliseconds()); - if (this.client.ready(node, time.milliseconds())) { + + if (!this.client.ready(node, time.milliseconds())) { + response.needRetry(); + } else { // create a consumer metadata request log.debug("Issuing consumer metadata request to broker {}", node.id()); @@ -333,26 +352,34 @@ public final class Coordinator { long now = time.milliseconds(); RequestCompletionHandler completionHandler = new RequestCompletionHandler() { @Override - public void onComplete(ClientResponse response) { + public void onComplete(ClientResponse resp) { + log.debug("Consumer metadata response {}", resp); + // parse the response to get the coordinator info if it is not disconnected, // otherwise we need to request metadata update - if (!response.wasDisconnected()) { - ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(response.responseBody()); + if (!resp.wasDisconnected()) { + ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody()); // use MAX_VALUE - node.id as the coordinator id to mimic separate connections // for the coordinator in the underlying network client layer // TODO: this needs to be better handled in KAFKA-1935 - if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) - consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), + if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) { + Coordinator.this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), consumerMetadataResponse.node().host(), consumerMetadataResponse.node().port()); + response.respond(null); + } else { + response.needRetry(); + } } else { - metadata.requestUpdate(); + response.needMetadataRefresh(); } } }; ClientRequest request = new ClientRequest(now, true, send, completionHandler); client.send(request); } + return response; + } /** @@ -372,9 +399,9 @@ public final class Coordinator { private void handleCoordinatorDisconnect(ClientResponse response) { int correlation = response.request().request().header().correlationId(); log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected", - response.request(), - correlation, - response.request().request().destination()); + response.request(), + correlation, + response.request().request().destination()); // mark the coordinator as dead coordinatorDead(); @@ -412,18 +439,22 @@ public final class Coordinator { private class CommitOffsetCompletionHandler implements RequestCompletionHandler { private final Map offsets; + private final CoordinatorResponse response; - public CommitOffsetCompletionHandler(Map offsets) { + public CommitOffsetCompletionHandler(Map offsets, CoordinatorResponse response) { this.offsets = offsets; + this.response = response; } @Override public void onComplete(ClientResponse resp) { if (resp.wasDisconnected()) { handleCoordinatorDisconnect(resp); + response.needNewCoordinator(); } else { - OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody()); - for (Map.Entry entry : response.responseData().entrySet()) { + boolean succeeded = true; + 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); @@ -432,15 +463,21 @@ public final class Coordinator { subscriptions.committed(tp, offset); } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + succeeded = false; coordinatorDead(); + response.needNewCoordinator(); } else { // do not need to throw the exception but just log the error + succeeded = false; log.error("Error committing partition {} at offset {}: {}", tp, offset, Errors.forCode(errorCode).exception().getMessage()); } } + + if (!response.isReady()) + response.respond(succeeded); } sensors.commitLatency.record(resp.requestLatencyMs()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResponse.java new file mode 100644 index 0000000..8723a6a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResponse.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; + +public class CoordinatorResponse extends BrokerResponse { + + private boolean needNewCoordinator = false; + + public void needNewCoordinator() { + this.needNewCoordinator = true; + this.ready = true; + } + + public boolean newCoordinatorNeeded() { + return needNewCoordinator; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResponse.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResponse.java new file mode 100644 index 0000000..bc52dad --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResponse.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.clients.consumer.internals; + +/** + * Result of an asynchronous request. Can be waited by calling poll. + * @param + */ +public interface DelayedResponse { + + /** + * Get the value corresponding to this request (if it has one, as indicated by {@link #hasValue()}). + * @return the value or null if none exists + */ + T value(); + + /** + * Check whether the response is ready to be handled + * @return true if the response is ready, false otherwise + */ + boolean isReady(); + + /** + * Delayed responses may have a value. This checks whether one is available. It only makes sense + * to check this if the response is ready (as indicated by {@link #isReady()}. + * @return true if a value is available, false otherwise + */ + boolean hasValue(); + +} 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 dfb99c6..13fb0fb 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 @@ -157,12 +157,15 @@ public class Fetcher { } /** - * Initialize a list offset request for the provided partition. Results from this fetch can be obtained by - * calling {@link #fetchedOffsets()}. - * @param topicPartition partition to get offsets for - * @param timestamp timestamp for fetching offsets + * Fetch a single offset before the given timestamp for the partition. + * + * @param topicPartition The partition that needs fetching offset. + * @param timestamp The timestamp for fetching offset. + * @return A response which can be polled to obtain the corresponding offset. */ - public void initOffsetFetch(final TopicPartition topicPartition, long timestamp, final Map result) { + public BrokerResponse offsetBefore(final TopicPartition topicPartition, long timestamp) { + final BrokerResponse response = new BrokerResponse(); + Map partitions = new HashMap(1); partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); long now = time.milliseconds(); @@ -170,10 +173,10 @@ public class Fetcher { if (info == null) { metadata.add(topicPartition.topic()); log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); - metadata.requestUpdate(); + response.needMetadataRefresh(); } else if (info.leader() == null) { log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); - metadata.requestUpdate(); + response.needMetadataRefresh(); } else if (this.client.ready(info.leader(), now)) { Node node = info.leader(); ListOffsetRequest request = new ListOffsetRequest(-1, partitions); @@ -182,26 +185,31 @@ public class Fetcher { request.toStruct()); RequestCompletionHandler completionHandler = new RequestCompletionHandler() { @Override - public void onComplete(ClientResponse response) { - handleOffsetFetchResponse(topicPartition, response, result); + public void onComplete(ClientResponse resp) { + handleOffsetFetchResponse(topicPartition, resp, response); } }; ClientRequest clientRequest = new ClientRequest(now, true, send, completionHandler); this.client.send(clientRequest); + } else { + response.needRetry(); } + return response; } /** * Callback for the response of the list offset call above. * @param topicPartition The partition that was fetched - * @param response The response from the server. + * @param clientResponse The response from the server. */ - private void handleOffsetFetchResponse(TopicPartition topicPartition, ClientResponse response, Map result) { - if (response.wasDisconnected()) { - metadata.requestUpdate(); + private void handleOffsetFetchResponse(TopicPartition topicPartition, + ClientResponse clientResponse, + BrokerResponse response) { + if (clientResponse.wasDisconnected()) { + response.needMetadataRefresh(); } else { - ListOffsetResponse lor = new ListOffsetResponse(response.responseBody()); + ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody()); short errorCode = lor.responseData().get(topicPartition).errorCode; if (errorCode == Errors.NONE.code()) { List offsets = lor.responseData().get(topicPartition).offsets; @@ -209,12 +217,13 @@ public class Fetcher { throw new IllegalStateException("This should not happen."); long offset = offsets.get(0); log.debug("Fetched offset {} for partition {}", offset, topicPartition); - result.put(topicPartition, offset); + + response.respond(offset); } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", topicPartition); - metadata.requestUpdate(); + response.needMetadataRefresh(); } else { log.error("Attempt to fetch offsets for partition {} failed due to: {}", topicPartition, Errors.forCode(errorCode).exception().getMessage()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetchResult.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetchResult.java deleted file mode 100644 index ca00cac..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetchResult.java +++ /dev/null @@ -1,39 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals; - -import org.apache.kafka.common.TopicPartition; - -import java.util.Map; -import java.util.Set; - -/** - * Used to convey results from an offset fetch to the polling thread by way of a callback. In particular, - * see the usage in {@link org.apache.kafka.clients.consumer.KafkaConsumer#fetchCommittedOffsets(Set, long)}. - */ -public class OffsetFetchResult { - private Map offsets; - - public void setOffsets(Map offsets) { - this.offsets = offsets; - } - - public boolean isReady() { - return offsets != null; - } - - public Map offsets() { - return offsets; - } - -} 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 f25067c..dfcafaa 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 @@ -200,7 +200,7 @@ public class SubscriptionState { } public Map partitionsToReset() { - return this.resetPartitions; + return new HashMap(this.resetPartitions); } public boolean hasAllFetchPositions() { 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 140d000..5f32e7c 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 @@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -39,11 +38,9 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; import java.util.Collections; -import java.util.HashMap; 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; @@ -218,14 +215,14 @@ public class CoordinatorTest { client.poll(0, time.milliseconds()); // With success flag - AtomicBoolean success = new AtomicBoolean(false); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), success); + CoordinatorResponse result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertTrue(success.get()); + assertTrue(result.isReady()); + assertTrue(result.value()); // Without success flag - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), null); + 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()); } @@ -238,7 +235,7 @@ public class CoordinatorTest { // async commit with coordinator not available client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), null); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(coordinator.coordinatorUnknown()); // resume @@ -248,7 +245,7 @@ public class CoordinatorTest { // async commit with not coordinator client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), null); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(coordinator.coordinatorUnknown()); // resume @@ -260,29 +257,28 @@ public class CoordinatorTest { 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()))); - AtomicBoolean success = new AtomicBoolean(false); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), success); + CoordinatorResponse result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertFalse(success.get()); + assertTrue(result.isReady()); + assertTrue(result.newCoordinatorNeeded()); // sync commit with coordinator disconnected client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - success = new AtomicBoolean(false); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), success); + result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(0, client.poll(0, time.milliseconds()).size()); - assertFalse(success.get()); + assertTrue(result.isReady()); + assertTrue(result.newCoordinatorNeeded()); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); coordinator.discoverConsumerCoordinator(); client.poll(0, time.milliseconds()); - - success = new AtomicBoolean(false); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds(), success); + result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertTrue(success.get()); + assertTrue(result.isReady()); + assertTrue(result.value()); } @@ -295,67 +291,63 @@ public class CoordinatorTest { // normal fetch client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - OffsetFetchResult result = new OffsetFetchResult(); - coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + CoordinatorResponse> result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); - assertEquals(100L, (long) result.offsets().get(tp)); + assertEquals(100L, (long) result.value().get(tp)); // fetch with loading in progress - result = new OffsetFetchResult(); client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L)); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); - assertFalse(result.isReady()); + assertTrue(result.isReady()); + assertTrue(result.retryNeeded()); - coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); - assertEquals(100L, (long) result.offsets().get(tp)); + assertEquals(100L, (long) result.value().get(tp)); // fetch with not coordinator - result = new OffsetFetchResult(); 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.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); - assertFalse(result.isReady()); + assertTrue(result.isReady()); + assertTrue(result.retryNeeded()); coordinator.discoverConsumerCoordinator(); client.poll(0, time.milliseconds()); - coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); - assertEquals(100L, (long) result.offsets().get(tp)); + assertEquals(100L, (long) result.value().get(tp)); // fetch with no fetchable offsets - result = new OffsetFetchResult(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); - assertTrue(result.offsets().isEmpty()); + assertTrue(result.value().isEmpty()); // fetch with offset topic unknown - result = new OffsetFetchResult(); client.prepareResponse(offsetFetchResponse(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), "", 100L)); - coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); - assertTrue(result.offsets().isEmpty()); + assertTrue(result.value().isEmpty()); // fetch with offset -1 - result = new OffsetFetchResult(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds(), result); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); - assertTrue(result.offsets().isEmpty()); + assertTrue(result.value().isEmpty()); } private Struct consumerMetadataResponse(Node node, short error) { 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 5ccd08d..87a103c 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 @@ -16,9 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -32,10 +29,11 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.FetchResponse; -import org.apache.kafka.common.requests.ListOffsetResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; import java.nio.ByteBuffer; import java.util.Collections; @@ -43,8 +41,8 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import org.junit.Before; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class FetcherTest { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index ab4d1e7..0b5e166 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -49,8 +49,8 @@ public class SubscriptionStateTest { public void partitionReset() { state.subscribe(tp0); state.seek(tp0, 5); - assertEquals(5l, (long) state.fetched(tp0)); - assertEquals(5l, (long) state.consumed(tp0)); + assertEquals(5L, (long) state.fetched(tp0)); + assertEquals(5L, (long) state.consumed(tp0)); state.needOffsetReset(tp0); assertTrue(state.offsetResetNeeded()); assertTrue(state.offsetResetNeeded(tp0)); -- 2.3.2 (Apple Git-55) From ed296aab844033b1180618d310f86f15f34efa7d Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 3 Jun 2015 16:38:55 -0700 Subject: [PATCH 3/4] KAFKA-2168; address review comments --- .../kafka/clients/consumer/KafkaConsumer.java | 184 +++++----- .../kafka/clients/consumer/MockConsumer.java | 2 +- .../clients/consumer/OffsetResetStrategy.java | 17 + .../clients/consumer/internals/BrokerResponse.java | 78 ----- .../clients/consumer/internals/BrokerResult.java | 39 +++ .../clients/consumer/internals/Coordinator.java | 377 +++++++++++---------- .../consumer/internals/CoordinatorResponse.java | 27 -- .../consumer/internals/CoordinatorResult.java | 50 +++ .../consumer/internals/DelayedResponse.java | 40 --- .../clients/consumer/internals/DelayedResult.java | 147 ++++++++ .../kafka/clients/consumer/internals/Fetcher.java | 24 +- .../consumer/internals/SubscriptionState.java | 20 +- .../kafka/clients/consumer/MockConsumerTest.java | 2 +- .../consumer/internals/CoordinatorTest.java | 28 +- .../clients/consumer/internals/FetcherTest.java | 8 +- .../consumer/internals/SubscriptionStateTest.java | 8 +- 16 files changed, 582 insertions(+), 469 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResponse.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResult.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResponse.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResult.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResponse.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResult.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 a245807..2b8eac7 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 @@ -16,10 +16,10 @@ 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.Coordinator; -import org.apache.kafka.clients.consumer.internals.CoordinatorResponse; -import org.apache.kafka.clients.consumer.internals.DelayedResponse; +import org.apache.kafka.clients.consumer.internals.CoordinatorResult; +import org.apache.kafka.clients.consumer.internals.DelayedResult; import org.apache.kafka.clients.consumer.internals.Fetcher; -import org.apache.kafka.clients.consumer.internals.BrokerResponse; +import org.apache.kafka.clients.consumer.internals.BrokerResult; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; @@ -495,7 +495,6 @@ public class KafkaConsumer implements Consumer { config.getString(ConsumerConfig.GROUP_ID_CONFIG), config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - this.metadata, this.subscriptions, metrics, metricGrpPrefix, @@ -641,42 +640,39 @@ public class KafkaConsumer implements Consumer { @Override public ConsumerRecords poll(long timeout) { ensureNotClosed(); - long now = time.milliseconds(); - log.debug("Entering consumer poll"); - - 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); - } - } - - // 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()); - - // Some partitions may need to be reset (if they had no previous offset - // or if a seek to beginning/end has been done - if (subscriptions.offsetResetNeeded()) - resetOffsets(); - - // maybe autocommit position - if (shouldAutoCommit(now)) - commit(CommitType.ASYNC); + log.trace("Entering consumer poll"); // Poll for new data until the timeout expires Cluster cluster = this.metadata.fetch(); long remaining = timeout; while (remaining > 0) { - // TODO: Long polling could prevent heartbeats. Probably we should move maybeHeartbeat - // into this loop and only call poll with a max timeout of the heartbeat interval - long start = time.milliseconds(); + // TODO: Sub-requests should take into account the poll timeout + + if (subscriptions.partitionsAutoAssigned()) { + if (subscriptions.partitionAssignmentNeeded()) { + // TODO: partition assignment must wait on JoinGroup responses which can block + // as long as a session timeout, which means we either have to wait longer + // than the timeout in this poll call or leave the request pending and return + + // rebalance to get partition assignment + reassignPartitions(start); + } else { + // try to heartbeat with the coordinator if needed + coordinator.maybeHeartbeat(start); + } + } + + // 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(start)) + commit(CommitType.ASYNC); + // Init any new fetches (won't resend pending fetches) fetcher.initFetches(cluster, start); poll(remaining, start); @@ -788,9 +784,6 @@ public class KafkaConsumer implements Consumer { Long offset = this.subscriptions.consumed(partition); if (offset == null) { updateFetchPositions(Collections.singleton(partition)); - if (subscriptions.offsetResetNeeded(partition)) - resetOffset(partition, subscriptions.partitionsToReset().get(partition)); - return this.subscriptions.consumed(partition); } else { return offset; @@ -845,6 +838,7 @@ public class KafkaConsumer implements Consumer { */ @Override public List partitionsFor(String topic) { + ensureNotClosed(); Cluster cluster = this.metadata.fetch(); List parts = cluster.partitionsForTopic(topic); if (parts == null) { @@ -940,37 +934,35 @@ public class KafkaConsumer implements Consumer { // reset the fetch position to the committed position for (TopicPartition tp : partitions) { - if (!subscriptions.offsetResetNeeded(tp) && subscriptions.fetched(tp) == null) { - if (subscriptions.committed(tp) == null) { - // if the committed position is unknown reset the position - subscriptions.needOffsetReset(tp); - } else { - log.debug("Resetting offset for partition {} to the committed offset {}", - tp, subscriptions.committed(tp)); - subscriptions.seek(tp, subscriptions.committed(tp)); - } + // 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()) { + 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 any partitions that need it. - */ - private void resetOffsets() { - for (Map.Entry resetEntry : subscriptions.partitionsToReset().entrySet()) { - // TODO: list offset call could be optimized by grouping by node - resetOffset(resetEntry.getKey(), resetEntry.getValue()); - } - } - - /** * Reset offsets for the given partition using the offset reset strategy. * * @param partition The given partition that needs reset offset * @param strategy The strategy to use for resetting (either EARLIEST or LATEST) * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined */ - private void resetOffset(TopicPartition partition, OffsetResetStrategy strategy) { + private void resetOffset(TopicPartition partition) { + OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); + long timestamp; if (strategy == OffsetResetStrategy.EARLIEST) timestamp = EARLIEST_OFFSET_TIMESTAMP; @@ -992,12 +984,12 @@ public class KafkaConsumer implements Consumer { */ private long offsetBefore(TopicPartition partition, long timestamp) { while (true) { - BrokerResponse response = fetcher.offsetBefore(partition, timestamp); + BrokerResult response = fetcher.offsetBefore(partition, timestamp); poll(response, requestTimeoutMs); if (response.isReady()) { - if (response.hasValue()) return response.value(); - handleBrokerResponse(response); + if (response.succeeded()) return response.value(); + handleBrokerFailure(response); } } } @@ -1023,25 +1015,24 @@ public class KafkaConsumer implements Consumer { private void assignPartitions() { while (subscriptions.partitionAssignmentNeeded()) { - ensureCoordinatorKnown(); - CoordinatorResponse response = coordinator.assignPartitions(time.milliseconds()); + CoordinatorResult response = coordinator.assignPartitions(time.milliseconds()); // Block indefinitely for the join group request (which can take as long as // a session timeout) poll(response); if (response.isReady()) { - handleCoordinatorResponse(response); + handleCoordinatorFailure(response); } } } private void ensureCoordinatorKnown() { while (coordinator.coordinatorUnknown()) { - BrokerResponse response = coordinator.discoverConsumerCoordinator(); + BrokerResult response = coordinator.discoverConsumerCoordinator(); poll(response, requestTimeoutMs); if (response.isReady()) { - handleBrokerResponse(response); + handleBrokerFailure(response); } } } @@ -1049,12 +1040,12 @@ public class KafkaConsumer implements Consumer { private Map fetchCommittedOffsets(Set partitions) { while (true) { long now = time.milliseconds(); - CoordinatorResponse> response = coordinator.fetchOffsets(partitions, now); + CoordinatorResult> response = coordinator.fetchOffsets(partitions, now); poll(response, requestTimeoutMs); if (response.isReady()) { - if (response.hasValue()) return response.value(); - handleCoordinatorResponse(response); + if (response.succeeded()) return response.value(); + handleCoordinatorFailure(response); } else { log.debug("Fetch committed offsets failed"); } @@ -1064,7 +1055,7 @@ public class KafkaConsumer implements Consumer { private void commitOffsets(Map offsets, CommitType commitType) { while (true) { long now = time.milliseconds(); - CoordinatorResponse response = coordinator.commitOffsets(offsets, now); + CoordinatorResult response = coordinator.commitOffsets(offsets, now); if (commitType == CommitType.SYNC) poll(response, requestTimeoutMs); else @@ -1072,29 +1063,48 @@ public class KafkaConsumer implements Consumer { // Retry for both commit types if there are coordinator connection issues if (response.isReady()) { - if (response.hasValue() && response.value()) return; - handleCoordinatorResponse(response); + if (response.succeeded()) return; + handleCoordinatorFailure(response); } else if (commitType == CommitType.ASYNC) { break; } } } - private void handleCoordinatorResponse(CoordinatorResponse response) { - if (response.newCoordinatorNeeded()) { - ensureCoordinatorKnown(); - } else { - handleBrokerResponse(response); + private void handleCoordinatorFailure(CoordinatorResult response) { + if (response.failed()) { + if (response.hasException()) { + throw response.exception(); + } else if (response.hasRemedy()) { + switch (response.remedy()) { + case FIND_COORDINATOR: + ensureCoordinatorKnown(); + break; + + case RETRY: + Utils.sleep(retryBackoffMs); + break; + } + + } } } - private void handleBrokerResponse(BrokerResponse response) { - if (response.throwExceptionNeeded()) { - throw response.exception(); - } else if (response.metadataRefreshNeeded()) { - awaitMetadataUpdate(); - } else if (response.retryNeeded()) { - Utils.sleep(retryBackoffMs); + private void handleBrokerFailure(BrokerResult result) { + if (result.failed()) { + if (result.hasException()) { + throw result.exception(); + } else if (result.hasRemedy()) { + switch (result.remedy()) { + case REFRESH_METADATA: + awaitMetadataUpdate(); + break; + + case RETRY: + Utils.sleep(retryBackoffMs); + break; + } + } } } @@ -1102,9 +1112,8 @@ public class KafkaConsumer implements Consumer { * Poll until a response is ready or timeout expires * @param response The response to poll for * @param timeout The time in milliseconds to wait for the response - * @param */ - private void poll(DelayedResponse response, long timeout) { + private void poll(DelayedResult response, long timeout) { long remaining = timeout; do { long start = time.milliseconds(); @@ -1117,9 +1126,8 @@ public class KafkaConsumer implements Consumer { /** * Poll indefinitely until the response is ready. * @param response The response to poll for. - * @param */ - private void poll(DelayedResponse response) { + private void poll(DelayedResult response) { do { long now = time.milliseconds(); poll(-1, now); @@ -1149,8 +1157,4 @@ public class KafkaConsumer implements Consumer { throw new IllegalStateException("This consumer has already been closed."); } - - public enum OffsetResetStrategy { - LATEST, EARLIEST, NONE - } } 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 9e14142..46e26a6 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 @@ -40,7 +40,7 @@ public class MockConsumer implements Consumer { private Map>> records; private boolean closed; - public MockConsumer(KafkaConsumer.OffsetResetStrategy offsetResetStrategy) { + public MockConsumer(OffsetResetStrategy offsetResetStrategy) { this.subscriptions = new SubscriptionState(offsetResetStrategy); this.partitions = new HashMap>(); this.records = new HashMap>>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java new file mode 100644 index 0000000..542da7f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java @@ -0,0 +1,17 @@ +/** + * 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; + +public enum OffsetResetStrategy { + LATEST, EARLIEST, NONE +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResponse.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResponse.java deleted file mode 100644 index 89f2eb2..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResponse.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals; - -import org.apache.kafka.common.KafkaException; - -public class BrokerResponse implements DelayedResponse { - - private T value = null; - protected boolean ready = false; - private boolean needMetadataRefresh = false; - private boolean needRetry = false; - - private boolean needThrow = false; - private KafkaException exception; - - @Override - public T value() { - return value; - } - - @Override - public boolean isReady() { - return ready; - } - - @Override - public boolean hasValue() { - return value != null; - } - - public void respond(T result) { - this.value = result; - this.ready = true; - } - - public boolean metadataRefreshNeeded() { - return needMetadataRefresh; - } - - public boolean retryNeeded() { - return needRetry; - } - - public void needMetadataRefresh() { - this.needMetadataRefresh = true; - this.ready = true; - } - - public void needRetry() { - this.needRetry = true; - this.ready = true; - } - - public void raise(KafkaException e) { - this.needThrow = true; - this.exception = e; - } - - public boolean throwExceptionNeeded() { - return needThrow; - } - - public KafkaException exception() { - return exception; - } - -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResult.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResult.java new file mode 100644 index 0000000..152d0bd --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/BrokerResult.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.clients.consumer.internals; + +public class BrokerResult extends DelayedResult { + public static final BrokerResult RETRY_NEEDED = new BrokerResult(); + + static { + RETRY_NEEDED.fail(BrokerRemedy.RETRY); + } + + public enum BrokerRemedy { + REFRESH_METADATA, RETRY + } + + public void needRetry() { + fail(BrokerRemedy.RETRY); + } + + public void needMetadataRefresh() { + fail(BrokerRemedy.REFRESH_METADATA); + } + + @SuppressWarnings("unchecked") + public static BrokerResult retryNeeded() { + return (BrokerResult) RETRY_NEEDED; + } + +} 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 d01f130..3df786f 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 @@ -15,7 +15,6 @@ 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.common.KafkaException; import org.apache.kafka.common.MetricName; @@ -31,6 +30,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.ConsumerMetadataRequest; import org.apache.kafka.common.requests.ConsumerMetadataResponse; import org.apache.kafka.common.requests.HeartbeatRequest; @@ -66,7 +66,6 @@ public final class Coordinator { private final Time time; private final String groupId; - private final Metadata metadata; private final Heartbeat heartbeat; private final long sessionTimeoutMs; private final String assignmentStrategy; @@ -83,7 +82,6 @@ public final class Coordinator { String groupId, long sessionTimeoutMs, String assignmentStrategy, - Metadata metadata, SubscriptionState subscriptions, Metrics metrics, String metricGrpPrefix, @@ -95,7 +93,6 @@ public final class Coordinator { this.generation = -1; this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; this.groupId = groupId; - this.metadata = metadata; this.consumerCoordinator = null; this.subscriptions = subscriptions; this.sessionTimeoutMs = sessionTimeoutMs; @@ -104,9 +101,15 @@ public final class Coordinator { this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags); } - public CoordinatorResponse assignPartitions(final long now) { - final CoordinatorResponse response = newCoordinatorResponse(now); - if (response.isReady()) return response; + /** + * Get a new partition assignment. This will send a JoinGroup request to the coordinator (if it + * is available), which will update subscription assignments if it is completed successfully. + * @param now The current time in milliseconds + * @return A delayed response whose completion indicates the result of the JoinGroup request. + */ + public CoordinatorResult assignPartitions(final long now) { + final CoordinatorResult result = newCoordinatorResult(now); + if (result.isReady()) return result; // send a join group request to the coordinator List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); @@ -121,89 +124,86 @@ public final class Coordinator { // create the request for the coordinator log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); - RequestHeader header = this.client.nextRequestHeader(ApiKeys.JOIN_GROUP); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request.toStruct()); - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { @Override public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); - response.needNewCoordinator(); - return; - } - - // process the response - JoinGroupResponse joinResponse = new JoinGroupResponse(resp.responseBody()); - short errorCode = joinResponse.errorCode(); - - if (errorCode == Errors.NONE.code()) { - Coordinator.this.consumerId = joinResponse.consumerId(); - Coordinator.this.generation = joinResponse.generationId(); - - // set the flag to refresh last committed offsets - subscriptions.needRefreshCommits(); - - log.debug("Joined group: {}", joinResponse); - - // record re-assignment time - sensors.partitionReassignments.record(time.milliseconds() - now); - - // update subscription partition assignment - subscriptions.changePartitionAssignment(joinResponse.assignedPartitions()); - - response.respond(null); - } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { - // reset the consumer id and retry immediately - Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; - log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", - groupId); - - response.needRetry(); - } 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); - response.needNewCoordinator(); - } 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(); - log.error("Attempt to join group {} failed due to: {}", - groupId, e.getMessage()); - response.raise(e); - } else { - log.error("Unexpected error in join group response: " - + Errors.forCode(joinResponse.errorCode()).exception().getMessage()); - response.needRetry(); - } + handleJoinResponse(resp, result, now); } }; - this.client.send(new ClientRequest(now, true, send, completionHandler)); - return response; + sendCoordinator(ApiKeys.JOIN_GROUP, request.toStruct(), completionHandler, now); + return result; + } + + private void handleJoinResponse(ClientResponse response, CoordinatorResult result, long requestTime) { + if (response.wasDisconnected()) { + handleCoordinatorDisconnect(response); + result.needNewCoordinator(); + } else { + // process the response + JoinGroupResponse joinResponse = new JoinGroupResponse(response.responseBody()); + short errorCode = joinResponse.errorCode(); + + if (errorCode == Errors.NONE.code()) { + Coordinator.this.consumerId = joinResponse.consumerId(); + Coordinator.this.generation = joinResponse.generationId(); + + // set the flag to refresh last committed offsets + subscriptions.needRefreshCommits(); + + log.debug("Joined group: {}", response); + + // record re-assignment time + this.sensors.partitionReassignments.record(time.milliseconds() - requestTime); + + // update partition assignment + subscriptions.changePartitionAssignment(joinResponse.assignedPartitions()); + result.complete(null); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { + // reset the consumer id and retry immediately + Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", + groupId); + + result.needRetry(); + } 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); + result.needNewCoordinator(); + } 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(); + log.error("Attempt to join group {} failed due to: {}", + groupId, e.getMessage()); + result.raise(e); + } else { + // unexpected error, throw the exception + result.raise(new KafkaException("Unexpected error in join group response: " + + Errors.forCode(joinResponse.errorCode()).exception().getMessage())); + } + } } /** - * Commit offsets for the specified list of topics and partitions. - * - * A non-blocking commit will attempt to commit offsets asychronously. 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. + * Commit offsets for the specified list of topics and partitions. This method returns a delayed + * response which can be polled on for 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 delayed response whose value indicates whether the commit was successful or not */ - public CoordinatorResponse commitOffsets(final Map offsets, long now) { - final CoordinatorResponse response = newCoordinatorResponse(now); - if (response.isReady()) return response; + public CoordinatorResult commitOffsets(final Map offsets, long now) { + final CoordinatorResult result = newCoordinatorResult(now); + if (result.isReady()) return result; if (offsets.isEmpty()) { - response.respond(true); + result.complete(null); } else { // create the offset commit request Map offsetData; @@ -216,94 +216,90 @@ public final class Coordinator { OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets, response); - RequestHeader header = this.client.nextRequestHeader(ApiKeys.OFFSET_COMMIT); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, req.toStruct()); - this.client.send(new ClientRequest(now, true, send, handler)); + RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets, result); + sendCoordinator(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); } - return response; + return result; } - private CoordinatorResponse newCoordinatorResponse(long now) { - CoordinatorResponse response = new CoordinatorResponse(); + private CoordinatorResult newCoordinatorResult(long now) { if (coordinatorUnknown()) { - response.needNewCoordinator(); + return CoordinatorResult.newCoordinatorNeeded(); } else if (!this.client.ready(this.consumerCoordinator, now)) { - response.needRetry(); + return CoordinatorResult.retryNeeded(); } - return response; + return new CoordinatorResult(); } - - public CoordinatorResponse> fetchOffsets(Set partitions, long now) { - final CoordinatorResponse> response = newCoordinatorResponse(now); - if (response.isReady()) return response; + /** + * Fetch the committed offsets for a set of partitions. + * + * @param partitions The set of partitions to get offsets for. + * @param now The current time in milliseconds + * @return A delayed response containing the committed offsets. + */ + public CoordinatorResult> fetchOffsets(Set partitions, long now) { + final CoordinatorResult> result = newCoordinatorResult(now); + if (result.isReady()) return result; 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 - RequestHeader header = this.client.nextRequestHeader(ApiKeys.OFFSET_FETCH); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request.toStruct()); - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { @Override public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - response.needRetry(); - handleCoordinatorDisconnect(resp); - } else { - OffsetFetchResponse offsetResponse = new OffsetFetchResponse(resp.responseBody()); - handleOffsetResponse(offsetResponse, response); - } + handleOffsetResponse(resp, result); } }; - - this.client.send(new ClientRequest(now, true, send, completionHandler)); - - return response; + sendCoordinator(ApiKeys.OFFSET_FETCH, request.toStruct(), completionHandler, now); + return result; } - private void handleOffsetResponse(OffsetFetchResponse response, CoordinatorResponse> result) { - // parse the response to get the offsets - boolean offsetsReady = true; - - Map offsets = new HashMap(response.responseData().size()); - for (Map.Entry entry : response.responseData().entrySet()) { - TopicPartition tp = entry.getKey(); - OffsetFetchResponse.PartitionData data = entry.getValue(); - if (data.hasError()) { - log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) - .exception() - .getMessage()); - if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { - // just retry - offsetsReady = false; - } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - // re-discover the coordinator and retry - coordinatorDead(); - offsetsReady = false; - } else if (data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { - // just ignore this partition - log.debug("Unknown topic or partition for " + tp); + private void handleOffsetResponse(ClientResponse resp, CoordinatorResult> result) { + if (resp.wasDisconnected()) { + result.needRetry(); + handleCoordinatorDisconnect(resp); + } else { + // parse the response to get the offsets + OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); + Map offsets = new HashMap(response.responseData().size()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + OffsetFetchResponse.PartitionData data = entry.getValue(); + if (data.hasError()) { + log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) + .exception() + .getMessage()); + if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { + // just retry + result.needRetry(); + } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + // re-discover the coordinator and retry + coordinatorDead(); + result.needNewCoordinator(); + } else if (data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + // just ignore this partition + log.debug("Unknown topic or partition for " + tp); + } else { + result.raise(new IllegalStateException("Unexpected error code " + data.errorCode + + " while fetching offset")); + } + } else if (data.offset >= 0) { + // record the position with the offset (-1 indicates no committed offset to fetch) + offsets.put(tp, data.offset); } else { - throw new KafkaException("Unexpected error in fetch offset response: " - + Errors.forCode(data.errorCode).exception().getMessage()); + result.raise(new KafkaException("Unexpected error in fetch offset response: " + + Errors.forCode(data.errorCode).exception().getMessage())); } - } else if (data.offset >= 0) { - // record the position with the offset (-1 indicates no committed offset to fetch) - offsets.put(tp, data.offset); - } else { - log.debug("No committed offset for partition " + tp); } - } - if (offsetsReady) - result.respond(offsets); - else - result.needRetry(); + if (!result.isReady()) { + result.complete(offsets); + } + } } /** @@ -314,12 +310,7 @@ public final class Coordinator { public void maybeHeartbeat(long now) { if (heartbeat.shouldHeartbeat(now) && coordinatorReady(now)) { HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); - - RequestHeader header = this.client.nextRequestHeader(ApiKeys.HEARTBEAT); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, req.toStruct()); - ClientRequest request = new ClientRequest(now, true, send, new HeartbeatCompletionHandler()); - - this.client.send(request); + sendCoordinator(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now); this.heartbeat.sentHeartbeat(now); } } @@ -332,54 +323,57 @@ public final class Coordinator { return !coordinatorUnknown() && this.client.ready(this.consumerCoordinator, now); } - public BrokerResponse discoverConsumerCoordinator() { + /** + * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to + * one of the brokers. + * @return A delayed response which indicates the completion of the metadata request + */ + public BrokerResult discoverConsumerCoordinator() { // initiate the consumer metadata request // find a node to ask about the coordinator - final BrokerResponse response = new BrokerResponse(); - Node node = this.client.leastLoadedNode(time.milliseconds()); + long now = time.milliseconds(); + Node node = this.client.leastLoadedNode(now); - if (!this.client.ready(node, time.milliseconds())) { - response.needRetry(); + if (!this.client.ready(node, now)) { + return BrokerResult.retryNeeded(); } else { + final BrokerResult result = new BrokerResult(); + // create a consumer metadata request log.debug("Issuing consumer metadata request to broker {}", node.id()); - ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId); - RequestSend send = new RequestSend(node.id(), - this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), - metadataRequest.toStruct()); - - long now = time.milliseconds(); RequestCompletionHandler completionHandler = new RequestCompletionHandler() { @Override public void onComplete(ClientResponse resp) { - log.debug("Consumer metadata response {}", resp); - - // parse the response to get the coordinator info if it is not disconnected, - // otherwise we need to request metadata update - if (!resp.wasDisconnected()) { - ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody()); - // use MAX_VALUE - node.id as the coordinator id to mimic separate connections - // for the coordinator in the underlying network client layer - // TODO: this needs to be better handled in KAFKA-1935 - if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) { - Coordinator.this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), - consumerMetadataResponse.node().host(), - consumerMetadataResponse.node().port()); - response.respond(null); - } else { - response.needRetry(); - } - } else { - response.needMetadataRefresh(); - } + handleConsumerMetadataResponse(resp, result); } }; - ClientRequest request = new ClientRequest(now, true, send, completionHandler); - client.send(request); + send(node, ApiKeys.CONSUMER_METADATA, metadataRequest.toStruct(), completionHandler, now); + return result; } - return response; + } + private void handleConsumerMetadataResponse(ClientResponse resp, BrokerResult result) { + log.debug("Consumer metadata response {}", resp); + + // parse the response to get the coordinator info if it is not disconnected, + // otherwise we need to request metadata update + if (resp.wasDisconnected()) { + result.needMetadataRefresh(); + } else { + ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody()); + // use MAX_VALUE - node.id as the coordinator id to mimic separate connections + // for the coordinator in the underlying network client layer + // TODO: this needs to be better handled in KAFKA-1935 + if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) { + Coordinator.this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), + consumerMetadataResponse.node().host(), + consumerMetadataResponse.node().port()); + result.complete(null); + } else { + result.needRetry(); + } + } } /** @@ -392,7 +386,6 @@ public final class Coordinator { } } - /** * Handle the case when the request gets cancelled due to coordinator disconnection. */ @@ -407,6 +400,17 @@ public final class Coordinator { coordinatorDead(); } + + private void sendCoordinator(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { + send(this.consumerCoordinator, api, request, handler, now); + } + + private void send(Node node, ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { + RequestHeader header = this.client.nextRequestHeader(api); + RequestSend send = new RequestSend(node.id(), header, request); + this.client.send(new ClientRequest(now, true, send, handler)); + } + private class HeartbeatCompletionHandler implements RequestCompletionHandler { @Override public void onComplete(ClientResponse resp) { @@ -439,20 +443,19 @@ public final class Coordinator { private class CommitOffsetCompletionHandler implements RequestCompletionHandler { private final Map offsets; - private final CoordinatorResponse response; + private final CoordinatorResult result; - public CommitOffsetCompletionHandler(Map offsets, CoordinatorResponse response) { + public CommitOffsetCompletionHandler(Map offsets, CoordinatorResult result) { this.offsets = offsets; - this.response = response; + this.result = result; } @Override public void onComplete(ClientResponse resp) { if (resp.wasDisconnected()) { handleCoordinatorDisconnect(resp); - response.needNewCoordinator(); + result.needNewCoordinator(); } else { - boolean succeeded = true; OffsetCommitResponse commitResponse = new OffsetCommitResponse(resp.responseBody()); for (Map.Entry entry : commitResponse.responseData().entrySet()) { TopicPartition tp = entry.getKey(); @@ -463,12 +466,11 @@ public final class Coordinator { subscriptions.committed(tp, offset); } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - succeeded = false; coordinatorDead(); - response.needNewCoordinator(); + result.needNewCoordinator(); } else { // do not need to throw the exception but just log the error - succeeded = false; + result.needRetry(); log.error("Error committing partition {} at offset {}: {}", tp, offset, @@ -476,8 +478,9 @@ public final class Coordinator { } } - if (!response.isReady()) - response.respond(succeeded); + if (!result.isReady()) { + result.complete(null); + } } sensors.commitLatency.record(resp.requestLatencyMs()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResponse.java deleted file mode 100644 index 8723a6a..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResponse.java +++ /dev/null @@ -1,27 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals; - -public class CoordinatorResponse extends BrokerResponse { - - private boolean needNewCoordinator = false; - - public void needNewCoordinator() { - this.needNewCoordinator = true; - this.ready = true; - } - - public boolean newCoordinatorNeeded() { - return needNewCoordinator; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResult.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResult.java new file mode 100644 index 0000000..97c5f3e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorResult.java @@ -0,0 +1,50 @@ +/** + * 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; + +/** + * Delayed response subclass for responses from the consumer group coordinator. + * @param + */ +public class CoordinatorResult extends DelayedResult { + public static final CoordinatorResult NEED_NEW_COORDINATOR = new CoordinatorResult(); + public static final CoordinatorResult NEED_RETRY = new CoordinatorResult(); + + static { + NEED_NEW_COORDINATOR.fail(CoordinatorRemedy.FIND_COORDINATOR); + NEED_RETRY.fail(CoordinatorRemedy.RETRY); + } + + public enum CoordinatorRemedy { + RETRY, FIND_COORDINATOR + } + + public void needRetry() { + fail(CoordinatorRemedy.RETRY); + } + + public void needNewCoordinator() { + fail(CoordinatorRemedy.FIND_COORDINATOR); + } + + @SuppressWarnings("unchecked") + public static CoordinatorResult retryNeeded() { + return (CoordinatorResult) NEED_RETRY; + } + + @SuppressWarnings("unchecked") + public static CoordinatorResult newCoordinatorNeeded() { + return (CoordinatorResult) NEED_NEW_COORDINATOR; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResponse.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResponse.java deleted file mode 100644 index bc52dad..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResponse.java +++ /dev/null @@ -1,40 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals; - -/** - * Result of an asynchronous request. Can be waited by calling poll. - * @param - */ -public interface DelayedResponse { - - /** - * Get the value corresponding to this request (if it has one, as indicated by {@link #hasValue()}). - * @return the value or null if none exists - */ - T value(); - - /** - * Check whether the response is ready to be handled - * @return true if the response is ready, false otherwise - */ - boolean isReady(); - - /** - * Delayed responses may have a value. This checks whether one is available. It only makes sense - * to check this if the response is ready (as indicated by {@link #isReady()}. - * @return true if a value is available, false otherwise - */ - boolean hasValue(); - -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResult.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResult.java new file mode 100644 index 0000000..2414c40 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResult.java @@ -0,0 +1,147 @@ +/** + * 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; + +/** + * Result of an asynchronous request. It is similar to a Future, but it does not provide the mechanics to + * handle itself. Instead, you must use {@link org.apache.kafka.clients.KafkaClient#poll(long, long)} until + * a response is ready. Blocking usage would typically look something like this: + * + *
+ *     DelayedResult response = initializeRequest();
+ *     do {
+ *         client.poll(timeout, now);
+ *     } while (!response.isReady());
+ * 
+ * + * When {@link #isReady()} returns true, then one of {@link #failed()} or {@link #succeeded()} + * will return true. If the result succeeded and there is a value corresponding with the result, then it + * can be obtained with {@link #value()}. + * + * If the result failed, then there are two possibilities: either there is an exception to handle, or + * there is a remedy to apply. Remedies are used to signal the caller that the request should be retried + * or that some other action should be taken. Exceptions indicate an error that generally cannot be handled + * and must be propagated. + * + * @param Return type of the result (Can be Void if there is no response) + */ +public class DelayedResult { + private boolean ready = false; + private boolean succeeded = false; + private R remedy; + private T value; + private RuntimeException exception; + + /** + * Check whether the response is ready to be handled + * @return true if the response is ready, false otherwise + */ + public boolean isReady() { + return ready; + } + + /** + * Get the value corresponding to this request (if it has one, as indicated by {@link #succeeded()}). + * @return the value if it exists or null + */ + public T value() { + return value; + } + + /** + * Check if a value was returned from the response. Use after {@link #isReady()} returns true. + * @return true if a value is available, false otherwise + */ + public boolean succeeded() { + return ready && succeeded; + } + + /** + * Return the error from this response (assuming {@link #failed()} has returned true. If the + * response is not ready or if there is no error, null is returned. + * @return the error if it exists or null + */ + public R remedy() { + return remedy; + } + + /** + * Check if there is a remedy for a failed response. + * @return true if the result is ready and there is a remedy + */ + public boolean hasRemedy() { + return ready && remedy != null; + } + + /** + * Check if the result failed (which is true both when {@link #fail(Object)} and {@link #raise(RuntimeException)} + * are used. Use after {@link #isReady()} returns true. + * @return true if a failure occurred, false otherwise + */ + public boolean failed() { + return ready && !succeeded; + } + + /** + * Get the exception from a failed result. You should check that there is an exception + * with {@link #hasException()} before using this method. + * @return The exception if it exists or null + */ + public RuntimeException exception() { + return exception; + } + + /** + * Check whether there is an exception. This is only possible if the result is ready and + * {@link #failed()} returns true. + * @return true if the result is ready and there is an exception, false otherwise + */ + public boolean hasException() { + return ready && exception != null; + } + + /** + * Fail the result with a possible remediation. This allows the caller to take some action in + * response to the failure (such as retrying the request). + * @param remedy The action that can be taken by the caller to recover from the failure + */ + public void fail(R remedy) { + this.ready = true; + this.succeeded = false; + this.remedy = remedy; + } + + /** + * 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 void complete(T value) { + this.ready = true; + this.succeeded = true; + this.value = value; + } + + /** + * Raise an exception. The result will be marked as failed, and the caller can either + * handle the exception or throw it. + * @param e The exception that + */ + public void raise(RuntimeException e){ + this.ready = true; + this.succeeded = false; + this.exception = e; + } + + +} 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 13fb0fb..e7a9c70 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 @@ -163,8 +163,8 @@ public class Fetcher { * @param timestamp The timestamp for fetching offset. * @return A response which can be polled to obtain the corresponding offset. */ - public BrokerResponse offsetBefore(final TopicPartition topicPartition, long timestamp) { - final BrokerResponse response = new BrokerResponse(); + public BrokerResult offsetBefore(final TopicPartition topicPartition, long timestamp) { + final BrokerResult result = new BrokerResult(); Map partitions = new HashMap(1); partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); @@ -173,10 +173,10 @@ public class Fetcher { if (info == null) { metadata.add(topicPartition.topic()); log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); - response.needMetadataRefresh(); + result.needMetadataRefresh(); } else if (info.leader() == null) { log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); - response.needMetadataRefresh(); + result.needMetadataRefresh(); } else if (this.client.ready(info.leader(), now)) { Node node = info.leader(); ListOffsetRequest request = new ListOffsetRequest(-1, partitions); @@ -186,16 +186,16 @@ public class Fetcher { RequestCompletionHandler completionHandler = new RequestCompletionHandler() { @Override public void onComplete(ClientResponse resp) { - handleOffsetFetchResponse(topicPartition, resp, response); + handleOffsetFetchResponse(topicPartition, resp, result); } }; ClientRequest clientRequest = new ClientRequest(now, true, send, completionHandler); this.client.send(clientRequest); } else { - response.needRetry(); + result.needRetry(); } - return response; + return result; } /** @@ -205,9 +205,9 @@ public class Fetcher { */ private void handleOffsetFetchResponse(TopicPartition topicPartition, ClientResponse clientResponse, - BrokerResponse response) { + BrokerResult result) { if (clientResponse.wasDisconnected()) { - response.needMetadataRefresh(); + result.needMetadataRefresh(); } else { ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody()); short errorCode = lor.responseData().get(topicPartition).errorCode; @@ -218,16 +218,16 @@ public class Fetcher { long offset = offsets.get(0); log.debug("Fetched offset {} for partition {}", offset, topicPartition); - response.respond(offset); + result.complete(offset); } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", topicPartition); - response.needMetadataRefresh(); + result.needMetadataRefresh(); } else { log.error("Attempt to fetch offsets for partition {} failed due to: {}", topicPartition, Errors.forCode(errorCode).exception().getMessage()); - metadata.requestUpdate(); + result.needMetadataRefresh(); } } } 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 dfcafaa..6837453 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 @@ -12,7 +12,7 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; import java.util.HashMap; @@ -51,12 +51,12 @@ public class SubscriptionState { private boolean needsFetchCommittedOffsets; /* Partitions that need to be reset before fetching */ - private Map resetPartitions; + private Map resetPartitions; /* Default offset reset strategy */ - private KafkaConsumer.OffsetResetStrategy offsetResetStrategy; + private OffsetResetStrategy offsetResetStrategy; - public SubscriptionState(KafkaConsumer.OffsetResetStrategy offsetResetStrategy) { + public SubscriptionState(OffsetResetStrategy offsetResetStrategy) { this.offsetResetStrategy = offsetResetStrategy; this.subscribedTopics = new HashSet(); this.subscribedPartitions = new HashSet(); @@ -66,7 +66,7 @@ public class SubscriptionState { this.committed = new HashMap(); this.needsPartitionAssignment = false; this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up - this.resetPartitions = new HashMap(); + this.resetPartitions = new HashMap(); } public void subscribe(String topic) { @@ -181,7 +181,7 @@ public class SubscriptionState { return this.consumed; } - public void needOffsetReset(TopicPartition partition, KafkaConsumer.OffsetResetStrategy offsetResetStrategy) { + public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) { this.resetPartitions.put(partition, offsetResetStrategy); this.fetched.remove(partition); this.consumed.remove(partition); @@ -191,16 +191,16 @@ public class SubscriptionState { needOffsetReset(partition, offsetResetStrategy); } - public boolean offsetResetNeeded(TopicPartition partition) { + public boolean isOffsetResetNeeded(TopicPartition partition) { return resetPartitions.containsKey(partition); } - public boolean offsetResetNeeded() { + public boolean isOffsetResetNeeded() { return !resetPartitions.isEmpty(); } - public Map partitionsToReset() { - return new HashMap(this.resetPartitions); + public OffsetResetStrategy resetStrategy(TopicPartition partition) { + return resetPartitions.get(partition); } public boolean hasAllFetchPositions() { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index 865b3df..26b6b40 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -25,7 +25,7 @@ import org.junit.Test; public class MockConsumerTest { - private MockConsumer consumer = new MockConsumer(KafkaConsumer.OffsetResetStrategy.EARLIEST); + private MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST); @Test public void testSimpleMock() { 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 5f32e7c..2adc784 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 @@ -20,9 +20,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; -import org.apache.kafka.clients.consumer.KafkaConsumer; +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; @@ -55,10 +54,9 @@ public class CoordinatorTest { private String rebalanceStrategy = "not-matter"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); - private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(KafkaConsumer.OffsetResetStrategy.EARLIEST); + private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); @@ -66,7 +64,6 @@ public class CoordinatorTest { groupId, sessionTimeoutMs, rebalanceStrategy, - metadata, subscriptions, metrics, "consumer" + groupId, @@ -75,7 +72,6 @@ public class CoordinatorTest { @Before public void setup() { - metadata.update(cluster, time.milliseconds()); client.setNode(node); } @@ -216,10 +212,10 @@ public class CoordinatorTest { // With success flag client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - CoordinatorResponse result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + CoordinatorResult result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(result.isReady()); - assertTrue(result.value()); + assertTrue(result.succeeded()); // Without success flag coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); @@ -257,10 +253,10 @@ public class CoordinatorTest { 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()))); - CoordinatorResponse result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + CoordinatorResult result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(result.isReady()); - assertTrue(result.newCoordinatorNeeded()); + assertEquals(CoordinatorResult.CoordinatorRemedy.FIND_COORDINATOR, result.remedy()); // sync commit with coordinator disconnected client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); @@ -269,7 +265,7 @@ public class CoordinatorTest { assertEquals(0, client.poll(0, time.milliseconds()).size()); assertTrue(result.isReady()); - assertTrue(result.newCoordinatorNeeded()); + assertEquals(CoordinatorResult.CoordinatorRemedy.FIND_COORDINATOR, result.remedy()); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); coordinator.discoverConsumerCoordinator(); @@ -278,7 +274,7 @@ public class CoordinatorTest { result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(result.isReady()); - assertTrue(result.value()); + assertTrue(result.succeeded()); } @@ -291,7 +287,7 @@ public class CoordinatorTest { // normal fetch client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - CoordinatorResponse> result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + CoordinatorResult> result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); assertEquals(100L, (long) result.value().get(tp)); @@ -303,7 +299,8 @@ public class CoordinatorTest { result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); - assertTrue(result.retryNeeded()); + assertTrue(result.failed()); + assertEquals(CoordinatorResult.CoordinatorRemedy.RETRY, result.remedy()); result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); @@ -318,7 +315,8 @@ public class CoordinatorTest { result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); client.poll(0, time.milliseconds()); assertTrue(result.isReady()); - assertTrue(result.retryNeeded()); + assertTrue(result.failed()); + assertEquals(CoordinatorResult.CoordinatorRemedy.FIND_COORDINATOR, result.remedy()); coordinator.discoverConsumerCoordinator(); client.poll(0, time.milliseconds()); 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 87a103c..405efdc 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 @@ -19,7 +19,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.KafkaConsumer; +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; @@ -57,7 +57,7 @@ public class FetcherTest { private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(KafkaConsumer.OffsetResetStrategy.EARLIEST); + private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); @@ -137,7 +137,7 @@ public class FetcherTest { fetcher.initFetches(cluster, time.milliseconds()); client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); client.poll(0, time.milliseconds()); - assertTrue(subscriptions.offsetResetNeeded(tp)); + assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(null, subscriptions.fetched(tp)); assertEquals(null, subscriptions.consumed(tp)); @@ -154,7 +154,7 @@ public class FetcherTest { fetcher.initFetches(cluster, time.milliseconds()); client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); client.poll(0, time.milliseconds()); - assertTrue(subscriptions.offsetResetNeeded(tp)); + assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(null, subscriptions.fetched(tp)); assertEquals(null, subscriptions.consumed(tp)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index 0b5e166..319751c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -22,13 +22,13 @@ import static java.util.Arrays.asList; import java.util.Collections; -import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; import org.junit.Test; public class SubscriptionStateTest { - private final SubscriptionState state = new SubscriptionState(KafkaConsumer.OffsetResetStrategy.EARLIEST); + private final SubscriptionState state = new SubscriptionState(OffsetResetStrategy.EARLIEST); private final TopicPartition tp0 = new TopicPartition("test", 0); private final TopicPartition tp1 = new TopicPartition("test", 1); @@ -52,8 +52,8 @@ public class SubscriptionStateTest { assertEquals(5L, (long) state.fetched(tp0)); assertEquals(5L, (long) state.consumed(tp0)); state.needOffsetReset(tp0); - assertTrue(state.offsetResetNeeded()); - assertTrue(state.offsetResetNeeded(tp0)); + assertTrue(state.isOffsetResetNeeded()); + assertTrue(state.isOffsetResetNeeded(tp0)); assertEquals(null, state.fetched(tp0)); assertEquals(null, state.consumed(tp0)); } -- 2.3.2 (Apple Git-55) From ab68e33c57f2cdcfa17ed33be62c48e37ae32b1b Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 3 Jun 2015 21:06:17 -0700 Subject: [PATCH 4/4] KAFKA-2186; fix rebase error and checkstyle issue --- .../org/apache/kafka/clients/consumer/internals/Coordinator.java | 7 +++---- .../org/apache/kafka/clients/consumer/internals/DelayedResult.java | 2 +- 2 files changed, 4 insertions(+), 5 deletions(-) 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 3df786f..2ba2883 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 @@ -284,15 +284,14 @@ public final class Coordinator { // just ignore this partition log.debug("Unknown topic or partition for " + tp); } else { - result.raise(new IllegalStateException("Unexpected error code " + data.errorCode - + " while fetching offset")); + result.raise(new KafkaException("Unexpected error in fetch offset response: " + + Errors.forCode(data.errorCode).exception().getMessage())); } } else if (data.offset >= 0) { // record the position with the offset (-1 indicates no committed offset to fetch) offsets.put(tp, data.offset); } else { - result.raise(new KafkaException("Unexpected error in fetch offset response: " - + Errors.forCode(data.errorCode).exception().getMessage())); + log.debug("No committed offset for partition " + tp); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResult.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResult.java index 2414c40..b8166fb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedResult.java @@ -137,7 +137,7 @@ public class DelayedResult { * handle the exception or throw it. * @param e The exception that */ - public void raise(RuntimeException e){ + public void raise(RuntimeException e) { this.ready = true; this.succeeded = false; this.exception = e; -- 2.3.2 (Apple Git-55)