mirror of https://github.com/apache/kafka.git
Merged in upstream trunk.
This commit is contained in:
commit
8b62019f0c
|
@ -54,7 +54,7 @@ The release file can be found inside ./core/build/distributions/.
|
||||||
### Cleaning the build ###
|
### Cleaning the build ###
|
||||||
./gradlew clean
|
./gradlew clean
|
||||||
|
|
||||||
### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.5 or 2.11.6) ###
|
### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.5 or 2.11.7) ###
|
||||||
#### (If building a jar with a version other than 2.10, need to set SCALA_BINARY_VERSION variable or change it in bin/kafka-run-class.sh to run quick start.) ####
|
#### (If building a jar with a version other than 2.10, need to set SCALA_BINARY_VERSION variable or change it in bin/kafka-run-class.sh to run quick start.) ####
|
||||||
./gradlew -PscalaVersion=2.9.1 jar
|
./gradlew -PscalaVersion=2.9.1 jar
|
||||||
./gradlew -PscalaVersion=2.9.1 test
|
./gradlew -PscalaVersion=2.9.1 test
|
||||||
|
|
36
build.gradle
36
build.gradle
|
@ -59,7 +59,7 @@ rat {
|
||||||
// And some of the files that we have checked in should also be excluded from this check
|
// And some of the files that we have checked in should also be excluded from this check
|
||||||
excludes.addAll([
|
excludes.addAll([
|
||||||
'**/.git/**',
|
'**/.git/**',
|
||||||
'build/rat/rat-report.xml',
|
'build/**',
|
||||||
'gradlew',
|
'gradlew',
|
||||||
'gradlew.bat',
|
'gradlew.bat',
|
||||||
'**/README.md',
|
'**/README.md',
|
||||||
|
@ -163,7 +163,7 @@ subprojects {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_6'] ) {
|
for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_7'] ) {
|
||||||
String svInDot = sv.replaceAll( "_", ".")
|
String svInDot = sv.replaceAll( "_", ".")
|
||||||
|
|
||||||
tasks.create(name: "jar_core_${sv}", type: GradleBuild) {
|
tasks.create(name: "jar_core_${sv}", type: GradleBuild) {
|
||||||
|
@ -203,20 +203,20 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_6'] ) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_6', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar', 'tools:jar']) {
|
tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar', 'tools:jar']) {
|
||||||
}
|
}
|
||||||
|
|
||||||
tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_6', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar', 'tools:srcJar']) { }
|
tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar', 'tools:srcJar']) { }
|
||||||
|
|
||||||
tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_6', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar', 'tools:docsJar']) { }
|
tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar', 'tools:docsJar']) { }
|
||||||
|
|
||||||
tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_6', 'clients:test', 'log4j-appender:test', 'tools:test']) {
|
tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test', 'tools:test']) {
|
||||||
}
|
}
|
||||||
|
|
||||||
tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_6']) {
|
tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) {
|
||||||
}
|
}
|
||||||
|
|
||||||
tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_6', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives', 'tools:uploadArchives']) {
|
tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives', 'tools:uploadArchives']) {
|
||||||
}
|
}
|
||||||
|
|
||||||
project(':core') {
|
project(':core') {
|
||||||
|
@ -225,8 +225,6 @@ project(':core') {
|
||||||
apply plugin: 'scala'
|
apply plugin: 'scala'
|
||||||
archivesBaseName = "kafka_${baseScalaVersion}"
|
archivesBaseName = "kafka_${baseScalaVersion}"
|
||||||
|
|
||||||
def (major, minor, trivial) = scalaVersion.tokenize('.')
|
|
||||||
|
|
||||||
dependencies {
|
dependencies {
|
||||||
compile project(':clients')
|
compile project(':clients')
|
||||||
compile project(':log4j-appender')
|
compile project(':log4j-appender')
|
||||||
|
@ -235,19 +233,19 @@ project(':core') {
|
||||||
compile 'com.101tec:zkclient:0.5'
|
compile 'com.101tec:zkclient:0.5'
|
||||||
compile 'com.yammer.metrics:metrics-core:2.2.0'
|
compile 'com.yammer.metrics:metrics-core:2.2.0'
|
||||||
compile 'net.sf.jopt-simple:jopt-simple:3.2'
|
compile 'net.sf.jopt-simple:jopt-simple:3.2'
|
||||||
|
if (scalaVersion.startsWith('2.11')) {
|
||||||
|
compile 'org.scala-lang.modules:scala-xml_2.11:1.0.4'
|
||||||
|
compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.4'
|
||||||
|
}
|
||||||
|
|
||||||
testCompile 'junit:junit:4.6'
|
testCompile 'junit:junit:4.6'
|
||||||
testCompile 'org.easymock:easymock:3.0'
|
testCompile 'org.easymock:easymock:3.0'
|
||||||
testCompile 'org.objenesis:objenesis:1.2'
|
testCompile 'org.objenesis:objenesis:1.2'
|
||||||
if (scalaVersion.startsWith('2.10')) {
|
if (scalaVersion.startsWith('2.9'))
|
||||||
testCompile 'org.scalatest:scalatest_2.10:1.9.1'
|
testCompile "org.scalatest:scalatest_$scalaVersion:1.9.1"
|
||||||
} else if (scalaVersion.startsWith('2.11')) {
|
else
|
||||||
compile 'org.scala-lang.modules:scala-xml_2.11:1.0.3'
|
testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5"
|
||||||
compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.3'
|
|
||||||
testCompile "org.scalatest:scalatest_2.11:2.2.0"
|
|
||||||
} else {
|
|
||||||
testCompile "org.scalatest:scalatest_$scalaVersion:1.8"
|
|
||||||
}
|
|
||||||
testRuntime "$slf4jlog4j"
|
testRuntime "$slf4jlog4j"
|
||||||
|
|
||||||
zinc 'com.typesafe.zinc:zinc:0.3.7'
|
zinc 'com.typesafe.zinc:zinc:0.3.7'
|
||||||
|
|
|
@ -63,11 +63,21 @@ public interface Consumer<K, V> extends Closeable {
|
||||||
*/
|
*/
|
||||||
public void commit(CommitType commitType);
|
public void commit(CommitType commitType);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @see KafkaConsumer#commit(CommitType, ConsumerCommitCallback)
|
||||||
|
*/
|
||||||
|
public void commit(CommitType commitType, ConsumerCommitCallback callback);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @see KafkaConsumer#commit(Map, CommitType)
|
* @see KafkaConsumer#commit(Map, CommitType)
|
||||||
*/
|
*/
|
||||||
public void commit(Map<TopicPartition, Long> offsets, CommitType commitType);
|
public void commit(Map<TopicPartition, Long> offsets, CommitType commitType);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @see KafkaConsumer#commit(Map, CommitType, ConsumerCommitCallback)
|
||||||
|
*/
|
||||||
|
public void commit(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @see KafkaConsumer#seek(TopicPartition, long)
|
* @see KafkaConsumer#seek(TopicPartition, long)
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -0,0 +1,33 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A callback interface that the user can implement to trigger custom actions when a commit request completes. The callback
|
||||||
|
* may be executed in any thread calling {@link Consumer#poll(long) poll()}.
|
||||||
|
*/
|
||||||
|
public interface ConsumerCommitCallback {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A callback method the user can implement to provide asynchronous handling of commit request completion.
|
||||||
|
* This method will be called when the commit request sent to the server has been acknowledged.
|
||||||
|
*
|
||||||
|
* @param offsets A map of the offsets that this callback applies to
|
||||||
|
* @param exception The exception thrown during processing of the request, or null if the commit completed successfully
|
||||||
|
*/
|
||||||
|
void onComplete(Map<TopicPartition, Long> offsets, Exception exception);
|
||||||
|
}
|
|
@ -173,7 +173,8 @@ public class ConsumerConfig extends AbstractConfig {
|
||||||
SESSION_TIMEOUT_MS_DOC)
|
SESSION_TIMEOUT_MS_DOC)
|
||||||
.define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
|
.define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
|
||||||
Type.STRING,
|
Type.STRING,
|
||||||
"blah",
|
"range",
|
||||||
|
in("range", "roundrobin"),
|
||||||
Importance.MEDIUM,
|
Importance.MEDIUM,
|
||||||
PARTITION_ASSIGNMENT_STRATEGY_DOC)
|
PARTITION_ASSIGNMENT_STRATEGY_DOC)
|
||||||
.define(METADATA_MAX_AGE_CONFIG,
|
.define(METADATA_MAX_AGE_CONFIG,
|
||||||
|
|
|
@ -74,6 +74,7 @@ public interface ConsumerRebalanceCallback {
|
||||||
* {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its
|
* {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its
|
||||||
* {@link #onPartitionsAssigned(Consumer, Collection)} callback.
|
* {@link #onPartitionsAssigned(Consumer, Collection)} callback.
|
||||||
*
|
*
|
||||||
|
* @param consumer Reference to the consumer for convenience
|
||||||
* @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously
|
* @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously
|
||||||
* assigned to the consumer)
|
* assigned to the consumer)
|
||||||
*/
|
*/
|
||||||
|
@ -87,6 +88,7 @@ public interface ConsumerRebalanceCallback {
|
||||||
* <p>
|
* <p>
|
||||||
* For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}
|
* For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}
|
||||||
*
|
*
|
||||||
|
* @param consumer Reference to the consumer for convenience
|
||||||
* @param partitions The list of partitions that were assigned to the consumer on the last rebalance
|
* @param partitions The list of partitions that were assigned to the consumer on the last rebalance
|
||||||
*/
|
*/
|
||||||
public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
|
public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
|
||||||
|
|
|
@ -105,6 +105,10 @@ public class ConsumerRecords<K, V> implements Iterable<ConsumerRecord<K, V>> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean isEmpty() {
|
||||||
|
return records.isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public static <K, V> ConsumerRecords<K, V> empty() {
|
public static <K, V> ConsumerRecords<K, V> empty() {
|
||||||
return (ConsumerRecords<K, V>) EMPTY;
|
return (ConsumerRecords<K, V>) EMPTY;
|
||||||
|
|
|
@ -15,9 +15,10 @@ package org.apache.kafka.clients.consumer;
|
||||||
import org.apache.kafka.clients.ClientUtils;
|
import org.apache.kafka.clients.ClientUtils;
|
||||||
import org.apache.kafka.clients.Metadata;
|
import org.apache.kafka.clients.Metadata;
|
||||||
import org.apache.kafka.clients.NetworkClient;
|
import org.apache.kafka.clients.NetworkClient;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient;
|
||||||
import org.apache.kafka.clients.consumer.internals.Coordinator;
|
import org.apache.kafka.clients.consumer.internals.Coordinator;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.DelayedTask;
|
||||||
import org.apache.kafka.clients.consumer.internals.Fetcher;
|
import org.apache.kafka.clients.consumer.internals.Fetcher;
|
||||||
import org.apache.kafka.clients.consumer.internals.RequestFuture;
|
|
||||||
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
||||||
import org.apache.kafka.common.Cluster;
|
import org.apache.kafka.common.Cluster;
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
@ -49,8 +50,8 @@ import java.util.Map;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
import static org.apache.kafka.common.utils.Utils.min;
|
import static org.apache.kafka.common.utils.Utils.min;
|
||||||
|
@ -65,8 +66,7 @@ import static org.apache.kafka.common.utils.Utils.min;
|
||||||
* The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to.
|
* The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to.
|
||||||
* Failure to close the consumer after use will leak these connections.
|
* Failure to close the consumer after use will leak these connections.
|
||||||
* <p>
|
* <p>
|
||||||
* The consumer is thread safe but generally will be used only from within a single thread. The consumer client has no
|
* The consumer is not thread-safe. See <a href="#multithreaded">Multi-threaded Processing</a> for more details.
|
||||||
* threads of it's own, all work is done in the caller's thread when calls are made on the various methods exposed.
|
|
||||||
*
|
*
|
||||||
* <h3>Offsets and Consumer Position</h3>
|
* <h3>Offsets and Consumer Position</h3>
|
||||||
* Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of
|
* Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of
|
||||||
|
@ -301,7 +301,8 @@ import static org.apache.kafka.common.utils.Utils.min;
|
||||||
* methods for seeking to the earliest and latest offset the server maintains are also available (
|
* methods for seeking to the earliest and latest offset the server maintains are also available (
|
||||||
* {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively).
|
* {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively).
|
||||||
*
|
*
|
||||||
* <h3>Multithreaded Processing</h3>
|
*
|
||||||
|
* <h3><a name="multithreaded">Multi-threaded Processing</a></h3>
|
||||||
*
|
*
|
||||||
* The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application
|
* The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application
|
||||||
* making the call. It is the responsibility of the user to ensure that multi-threaded access
|
* making the call. It is the responsibility of the user to ensure that multi-threaded access
|
||||||
|
@ -393,8 +394,7 @@ import static org.apache.kafka.common.utils.Utils.min;
|
||||||
public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class);
|
private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class);
|
||||||
private static final long EARLIEST_OFFSET_TIMESTAMP = -2L;
|
private static final long NO_CURRENT_THREAD = -1L;
|
||||||
private static final long LATEST_OFFSET_TIMESTAMP = -1L;
|
|
||||||
private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1);
|
private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1);
|
||||||
|
|
||||||
private final Coordinator coordinator;
|
private final Coordinator coordinator;
|
||||||
|
@ -403,21 +403,18 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
private final Fetcher<K, V> fetcher;
|
private final Fetcher<K, V> fetcher;
|
||||||
|
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final NetworkClient client;
|
private final ConsumerNetworkClient client;
|
||||||
private final Metrics metrics;
|
private final Metrics metrics;
|
||||||
private final SubscriptionState subscriptions;
|
private final SubscriptionState subscriptions;
|
||||||
private final Metadata metadata;
|
private final Metadata metadata;
|
||||||
private final long retryBackoffMs;
|
private final long retryBackoffMs;
|
||||||
private final boolean autoCommit;
|
private final boolean autoCommit;
|
||||||
private final long autoCommitIntervalMs;
|
private final long autoCommitIntervalMs;
|
||||||
private final ConsumerRebalanceCallback rebalanceCallback;
|
|
||||||
private long lastCommitAttemptMs;
|
|
||||||
private boolean closed = false;
|
private boolean closed = false;
|
||||||
private final AtomicBoolean wakeup = new AtomicBoolean(false);
|
|
||||||
|
|
||||||
// currentThread holds the threadId of the current thread accessing KafkaConsumer
|
// currentThread holds the threadId of the current thread accessing KafkaConsumer
|
||||||
// and is used to prevent multi-threaded access
|
// and is used to prevent multi-threaded access
|
||||||
private final AtomicReference<Long> currentThread = new AtomicReference<Long>();
|
private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD);
|
||||||
// refcount is used to allow reentrant access by the thread who has acquired currentThread
|
// refcount is used to allow reentrant access by the thread who has acquired currentThread
|
||||||
private final AtomicInteger refcount = new AtomicInteger(0);
|
private final AtomicInteger refcount = new AtomicInteger(0);
|
||||||
|
|
||||||
|
@ -505,14 +502,11 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
try {
|
try {
|
||||||
log.debug("Starting the Kafka consumer");
|
log.debug("Starting the Kafka consumer");
|
||||||
if (callback == null)
|
if (callback == null)
|
||||||
this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
|
callback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
|
||||||
ConsumerRebalanceCallback.class);
|
ConsumerRebalanceCallback.class);
|
||||||
else
|
|
||||||
this.rebalanceCallback = callback;
|
|
||||||
this.time = new SystemTime();
|
this.time = new SystemTime();
|
||||||
this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
|
this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
|
||||||
this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
|
this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
|
||||||
this.lastCommitAttemptMs = time.milliseconds();
|
|
||||||
|
|
||||||
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
|
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
|
||||||
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
|
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
|
||||||
|
@ -533,7 +527,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
String metricGrpPrefix = "consumer";
|
String metricGrpPrefix = "consumer";
|
||||||
Map<String, String> metricsTags = new LinkedHashMap<String, String>();
|
Map<String, String> metricsTags = new LinkedHashMap<String, String>();
|
||||||
metricsTags.put("client-id", clientId);
|
metricsTags.put("client-id", clientId);
|
||||||
this.client = new NetworkClient(
|
NetworkClient netClient = new NetworkClient(
|
||||||
new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags),
|
new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags),
|
||||||
this.metadata,
|
this.metadata,
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -541,6 +535,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG),
|
config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG),
|
||||||
config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG),
|
config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG),
|
||||||
config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG));
|
config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG));
|
||||||
|
this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs);
|
||||||
OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase());
|
OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase());
|
||||||
this.subscriptions = new SubscriptionState(offsetResetStrategy);
|
this.subscriptions = new SubscriptionState(offsetResetStrategy);
|
||||||
this.coordinator = new Coordinator(this.client,
|
this.coordinator = new Coordinator(this.client,
|
||||||
|
@ -551,8 +546,10 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
metrics,
|
metrics,
|
||||||
metricGrpPrefix,
|
metricGrpPrefix,
|
||||||
metricsTags,
|
metricsTags,
|
||||||
this.time);
|
this.time,
|
||||||
|
requestTimeoutMs,
|
||||||
|
retryBackoffMs,
|
||||||
|
wrapRebalanceCallback(callback));
|
||||||
if (keyDeserializer == null) {
|
if (keyDeserializer == null) {
|
||||||
this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
|
this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
|
||||||
Deserializer.class);
|
Deserializer.class);
|
||||||
|
@ -579,10 +576,14 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
metrics,
|
metrics,
|
||||||
metricGrpPrefix,
|
metricGrpPrefix,
|
||||||
metricsTags,
|
metricsTags,
|
||||||
this.time);
|
this.time,
|
||||||
|
this.retryBackoffMs);
|
||||||
|
|
||||||
config.logUnused();
|
config.logUnused();
|
||||||
|
|
||||||
|
if (autoCommit)
|
||||||
|
scheduleAutoCommitTask(autoCommitIntervalMs);
|
||||||
|
|
||||||
log.debug("Kafka consumer created");
|
log.debug("Kafka consumer created");
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
// call close methods if internal objects are already constructed
|
// call close methods if internal objects are already constructed
|
||||||
|
@ -717,27 +718,25 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
if (timeout < 0)
|
if (timeout < 0)
|
||||||
throw new IllegalArgumentException("Timeout must not be negative");
|
throw new IllegalArgumentException("Timeout must not be negative");
|
||||||
|
|
||||||
// Poll for new data until the timeout expires
|
// poll for new data until the timeout expires
|
||||||
long remaining = timeout;
|
long remaining = timeout;
|
||||||
while (remaining >= 0) {
|
while (remaining >= 0) {
|
||||||
long start = time.milliseconds();
|
long start = time.milliseconds();
|
||||||
long pollTimeout = min(remaining, timeToNextCommit(start), coordinator.timeToNextHeartbeat(start));
|
Map<TopicPartition, List<ConsumerRecord<K, V>>> records = pollOnce(remaining);
|
||||||
|
|
||||||
Map<TopicPartition, List<ConsumerRecord<K, V>>> records = pollOnce(pollTimeout, start);
|
|
||||||
long end = time.milliseconds();
|
long end = time.milliseconds();
|
||||||
|
|
||||||
if (!records.isEmpty()) {
|
if (!records.isEmpty()) {
|
||||||
// If data is available, then return it, but first send off the
|
// if data is available, then return it, but first send off the
|
||||||
// next round of fetches to enable pipelining while the user is
|
// next round of fetches to enable pipelining while the user is
|
||||||
// handling the fetched records.
|
// handling the fetched records.
|
||||||
fetcher.initFetches(metadata.fetch(), end);
|
fetcher.initFetches(metadata.fetch());
|
||||||
pollClient(0, end);
|
client.poll(0);
|
||||||
return new ConsumerRecords<K, V>(records);
|
return new ConsumerRecords<K, V>(records);
|
||||||
}
|
}
|
||||||
|
|
||||||
remaining -= end - start;
|
remaining -= end - start;
|
||||||
|
|
||||||
// Nothing was available, so we should backoff before retrying
|
// nothing was available, so we should backoff before retrying
|
||||||
if (remaining > 0) {
|
if (remaining > 0) {
|
||||||
Utils.sleep(min(remaining, retryBackoffMs));
|
Utils.sleep(min(remaining, retryBackoffMs));
|
||||||
remaining -= time.milliseconds() - end;
|
remaining -= time.milliseconds() - end;
|
||||||
|
@ -750,46 +749,42 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Do one round of polling. In addition to checking for new data, this does any needed
|
* Do one round of polling. In addition to checking for new data, this does any needed
|
||||||
* heart-beating, auto-commits, and offset updates.
|
* heart-beating, auto-commits, and offset updates.
|
||||||
* @param timeout The maximum time to block in the underlying poll
|
* @param timeout The maximum time to block in the underlying poll
|
||||||
* @param now Current time in millis
|
|
||||||
* @return The fetched records (may be empty)
|
* @return The fetched records (may be empty)
|
||||||
*/
|
*/
|
||||||
private Map<TopicPartition, List<ConsumerRecord<K, V>>> pollOnce(long timeout, long now) {
|
private Map<TopicPartition, List<ConsumerRecord<K, V>>> pollOnce(long timeout) {
|
||||||
Cluster cluster = this.metadata.fetch();
|
|
||||||
|
|
||||||
// TODO: Sub-requests should take into account the poll timeout (KAFKA-1894)
|
// TODO: Sub-requests should take into account the poll timeout (KAFKA-1894)
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
if (subscriptions.partitionsAutoAssigned()) {
|
// ensure we have partitions assigned if we expect to
|
||||||
if (subscriptions.partitionAssignmentNeeded()) {
|
if (subscriptions.partitionsAutoAssigned())
|
||||||
// rebalance to get partition assignment
|
coordinator.ensurePartitionAssignment();
|
||||||
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
|
// fetch positions if we have partitions we're subscribed to that we
|
||||||
// don't know the offset for
|
// don't know the offset for
|
||||||
if (!subscriptions.hasAllFetchPositions())
|
if (!subscriptions.hasAllFetchPositions())
|
||||||
updateFetchPositions(this.subscriptions.missingFetchPositions());
|
updateFetchPositions(this.subscriptions.missingFetchPositions());
|
||||||
|
|
||||||
// maybe autocommit position
|
// init any new fetches (won't resend pending fetches)
|
||||||
if (shouldAutoCommit(now))
|
Cluster cluster = this.metadata.fetch();
|
||||||
commit(CommitType.ASYNC);
|
fetcher.initFetches(cluster);
|
||||||
|
client.poll(timeout);
|
||||||
// Init any new fetches (won't resend pending fetches)
|
|
||||||
fetcher.initFetches(cluster, now);
|
|
||||||
|
|
||||||
pollClient(timeout, now);
|
|
||||||
|
|
||||||
return fetcher.fetchedRecords();
|
return fetcher.fetchedRecords();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void scheduleAutoCommitTask(final long interval) {
|
||||||
|
DelayedTask task = new DelayedTask() {
|
||||||
|
public void run(long now) {
|
||||||
|
commit(CommitType.ASYNC);
|
||||||
|
client.schedule(this, now + interval);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
client.schedule(task, time.milliseconds() + interval);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Commits the specified offsets for the specified list of topics and partitions to Kafka.
|
* Commits the specified offsets for the specified list of topics and partitions to Kafka.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -797,25 +792,42 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
* rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
|
* rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
|
||||||
* should not be used.
|
* should not be used.
|
||||||
* <p>
|
* <p>
|
||||||
* A non-blocking commit will attempt to commit offsets asynchronously. No error will be thrown if the commit fails.
|
* Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
|
||||||
* A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until
|
* commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use
|
||||||
* the commit succeeds.
|
* {@link #commit(Map, CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC})
|
||||||
|
* block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown
|
||||||
|
* to the caller).
|
||||||
*
|
*
|
||||||
* @param offsets The list of offsets per partition that should be committed to Kafka.
|
* @param offsets The list of offsets per partition that should be committed to Kafka.
|
||||||
* @param commitType Control whether the commit is blocking
|
* @param commitType Control whether the commit is blocking
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType) {
|
public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType) {
|
||||||
|
commit(offsets, commitType, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Commits the specified offsets for the specified list of topics and partitions to Kafka.
|
||||||
|
* <p>
|
||||||
|
* This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
|
||||||
|
* rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
|
||||||
|
* should not be used.
|
||||||
|
* <p>
|
||||||
|
* Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
|
||||||
|
* commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e.
|
||||||
|
* {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In
|
||||||
|
* this case, the error is either passed to the callback (if provided) or thrown to the caller.
|
||||||
|
*
|
||||||
|
* @param offsets The list of offsets per partition that should be committed to Kafka.
|
||||||
|
* @param commitType Control whether the commit is blocking
|
||||||
|
* @param callback Callback to invoke when the commit completes
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void commit(final Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
|
||||||
acquire();
|
acquire();
|
||||||
try {
|
try {
|
||||||
log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets);
|
log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets);
|
||||||
|
coordinator.commitOffsets(offsets, commitType, callback);
|
||||||
this.lastCommitAttemptMs = time.milliseconds();
|
|
||||||
|
|
||||||
// commit the offsets with the coordinator
|
|
||||||
if (commitType == CommitType.ASYNC)
|
|
||||||
this.subscriptions.needRefreshCommits();
|
|
||||||
commitOffsets(offsets, commitType);
|
|
||||||
} finally {
|
} finally {
|
||||||
release();
|
release();
|
||||||
}
|
}
|
||||||
|
@ -827,19 +839,45 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
* This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
|
* This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
|
||||||
* every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
|
* every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
|
||||||
* should not be used.
|
* should not be used.
|
||||||
|
* <p>
|
||||||
|
* Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
|
||||||
|
* commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e.
|
||||||
|
* {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In
|
||||||
|
* this case, the error is either passed to the callback (if provided) or thrown to the caller.
|
||||||
|
*
|
||||||
|
* @param commitType Whether or not the commit should block until it is acknowledged.
|
||||||
|
* @param callback Callback to invoke when the commit completes
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void commit(CommitType commitType, ConsumerCommitCallback callback) {
|
||||||
|
acquire();
|
||||||
|
try {
|
||||||
|
// need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance)
|
||||||
|
Map<TopicPartition, Long> allConsumed = new HashMap<TopicPartition, Long>(this.subscriptions.allConsumed());
|
||||||
|
commit(allConsumed, commitType, callback);
|
||||||
|
} finally {
|
||||||
|
release();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
|
||||||
|
* <p>
|
||||||
|
* This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
|
||||||
|
* every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
|
||||||
|
* should not be used.
|
||||||
|
* <p>
|
||||||
|
* Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous
|
||||||
|
* commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use
|
||||||
|
* {@link #commit(CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC})
|
||||||
|
* block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown
|
||||||
|
* to the caller).
|
||||||
*
|
*
|
||||||
* @param commitType Whether or not the commit should block until it is acknowledged.
|
* @param commitType Whether or not the commit should block until it is acknowledged.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void commit(CommitType commitType) {
|
public void commit(CommitType commitType) {
|
||||||
acquire();
|
commit(commitType, null);
|
||||||
try {
|
|
||||||
// Need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance)
|
|
||||||
Map<TopicPartition, Long> allConsumed = new HashMap<TopicPartition, Long>(this.subscriptions.allConsumed());
|
|
||||||
commit(allConsumed, commitType);
|
|
||||||
} finally {
|
|
||||||
release();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -866,8 +904,10 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
try {
|
try {
|
||||||
Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
|
Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
|
||||||
: Arrays.asList(partitions);
|
: Arrays.asList(partitions);
|
||||||
for (TopicPartition tp : parts)
|
for (TopicPartition tp : parts) {
|
||||||
|
log.debug("Seeking to beginning of partition {}", tp);
|
||||||
subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST);
|
subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST);
|
||||||
|
}
|
||||||
} finally {
|
} finally {
|
||||||
release();
|
release();
|
||||||
}
|
}
|
||||||
|
@ -881,8 +921,10 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
try {
|
try {
|
||||||
Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
|
Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
|
||||||
: Arrays.asList(partitions);
|
: Arrays.asList(partitions);
|
||||||
for (TopicPartition tp : parts)
|
for (TopicPartition tp : parts) {
|
||||||
|
log.debug("Seeking to end of partition {}", tp);
|
||||||
subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST);
|
subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST);
|
||||||
|
}
|
||||||
} finally {
|
} finally {
|
||||||
release();
|
release();
|
||||||
}
|
}
|
||||||
|
@ -929,19 +971,21 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
public long committed(TopicPartition partition) {
|
public long committed(TopicPartition partition) {
|
||||||
acquire();
|
acquire();
|
||||||
try {
|
try {
|
||||||
Set<TopicPartition> partitionsToFetch;
|
Long committed;
|
||||||
if (subscriptions.assignedPartitions().contains(partition)) {
|
if (subscriptions.assignedPartitions().contains(partition)) {
|
||||||
Long committed = this.subscriptions.committed(partition);
|
committed = this.subscriptions.committed(partition);
|
||||||
if (committed != null)
|
if (committed == null) {
|
||||||
return committed;
|
coordinator.refreshCommittedOffsetsIfNeeded();
|
||||||
partitionsToFetch = subscriptions.assignedPartitions();
|
committed = this.subscriptions.committed(partition);
|
||||||
} else {
|
|
||||||
partitionsToFetch = Collections.singleton(partition);
|
|
||||||
}
|
}
|
||||||
refreshCommittedOffsets(partitionsToFetch);
|
} else {
|
||||||
Long committed = this.subscriptions.committed(partition);
|
Map<TopicPartition, Long> offsets = coordinator.fetchCommittedOffsets(Collections.singleton(partition));
|
||||||
|
committed = offsets.get(partition);
|
||||||
|
}
|
||||||
|
|
||||||
if (committed == null)
|
if (committed == null)
|
||||||
throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition);
|
throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition);
|
||||||
|
|
||||||
return committed;
|
return committed;
|
||||||
} finally {
|
} finally {
|
||||||
release();
|
release();
|
||||||
|
@ -971,7 +1015,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
List<PartitionInfo> parts = cluster.partitionsForTopic(topic);
|
List<PartitionInfo> parts = cluster.partitionsForTopic(topic);
|
||||||
if (parts == null) {
|
if (parts == null) {
|
||||||
metadata.add(topic);
|
metadata.add(topic);
|
||||||
awaitMetadataUpdate();
|
client.awaitMetadataUpdate();
|
||||||
parts = metadata.fetch().partitionsForTopic(topic);
|
parts = metadata.fetch().partitionsForTopic(topic);
|
||||||
}
|
}
|
||||||
return parts;
|
return parts;
|
||||||
|
@ -997,7 +1041,6 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void wakeup() {
|
public void wakeup() {
|
||||||
this.wakeup.set(true);
|
|
||||||
this.client.wakeup();
|
this.client.wakeup();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1015,55 +1058,18 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Coordinator.RebalanceCallback wrapRebalanceCallback(final ConsumerRebalanceCallback callback) {
|
||||||
private boolean shouldAutoCommit(long now) {
|
return new Coordinator.RebalanceCallback() {
|
||||||
return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs;
|
@Override
|
||||||
|
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
|
||||||
|
callback.onPartitionsAssigned(KafkaConsumer.this, partitions);
|
||||||
}
|
}
|
||||||
|
|
||||||
private long timeToNextCommit(long now) {
|
@Override
|
||||||
if (!this.autoCommit)
|
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
|
||||||
return Long.MAX_VALUE;
|
callback.onPartitionsRevoked(KafkaConsumer.this, partitions);
|
||||||
long timeSinceLastCommit = now - this.lastCommitAttemptMs;
|
|
||||||
if (timeSinceLastCommit > this.autoCommitIntervalMs)
|
|
||||||
return 0;
|
|
||||||
return this.autoCommitIntervalMs - timeSinceLastCommit;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Request a metadata update and wait until it has occurred
|
|
||||||
*/
|
|
||||||
private void awaitMetadataUpdate() {
|
|
||||||
int version = this.metadata.requestUpdate();
|
|
||||||
do {
|
|
||||||
long now = time.milliseconds();
|
|
||||||
this.pollClient(this.retryBackoffMs, now);
|
|
||||||
} while (this.metadata.version() == version);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get partition assignment
|
|
||||||
*/
|
|
||||||
private void reassignPartitions(long now) {
|
|
||||||
// execute the user's callback before rebalance
|
|
||||||
log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions());
|
|
||||||
try {
|
|
||||||
this.rebalanceCallback.onPartitionsRevoked(this, this.subscriptions.assignedPartitions());
|
|
||||||
} catch (Exception e) {
|
|
||||||
log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
|
|
||||||
+ " failed on partition revocation: ", e);
|
|
||||||
}
|
|
||||||
|
|
||||||
// get new assigned partitions from the coordinator
|
|
||||||
assignPartitions();
|
|
||||||
|
|
||||||
// execute the user's callback after rebalance
|
|
||||||
log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions());
|
|
||||||
try {
|
|
||||||
this.rebalanceCallback.onPartitionsAssigned(this, this.subscriptions.assignedPartitions());
|
|
||||||
} catch (Exception e) {
|
|
||||||
log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
|
|
||||||
+ " failed on partition assignment: ", e);
|
|
||||||
}
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1075,267 +1081,11 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
* defined
|
* defined
|
||||||
*/
|
*/
|
||||||
private void updateFetchPositions(Set<TopicPartition> partitions) {
|
private void updateFetchPositions(Set<TopicPartition> partitions) {
|
||||||
// first refresh the committed positions in case they are not up-to-date
|
// refresh commits for all assigned partitions
|
||||||
refreshCommittedOffsets(partitions);
|
coordinator.refreshCommittedOffsetsIfNeeded();
|
||||||
|
|
||||||
// reset the fetch position to the committed position
|
// then do any offset lookups in case some positions are not known
|
||||||
for (TopicPartition tp : partitions) {
|
fetcher.updateFetchPositions(partitions);
|
||||||
// Skip if we already have a fetch position
|
|
||||||
if (subscriptions.fetched(tp) != null)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
// TODO: If there are several offsets to reset, we could submit offset requests in parallel
|
|
||||||
if (subscriptions.isOffsetResetNeeded(tp)) {
|
|
||||||
resetOffset(tp);
|
|
||||||
} else if (subscriptions.committed(tp) == null) {
|
|
||||||
// There's no committed position, so we need to reset with the default strategy
|
|
||||||
subscriptions.needOffsetReset(tp);
|
|
||||||
resetOffset(tp);
|
|
||||||
} else {
|
|
||||||
log.debug("Resetting offset for partition {} to the committed offset {}",
|
|
||||||
tp, subscriptions.committed(tp));
|
|
||||||
subscriptions.seek(tp, subscriptions.committed(tp));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Reset offsets for the given partition using the offset reset strategy.
|
|
||||||
*
|
|
||||||
* @param partition The given partition that needs reset offset
|
|
||||||
* @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined
|
|
||||||
*/
|
|
||||||
private void resetOffset(TopicPartition partition) {
|
|
||||||
OffsetResetStrategy strategy = subscriptions.resetStrategy(partition);
|
|
||||||
final long timestamp;
|
|
||||||
if (strategy == OffsetResetStrategy.EARLIEST)
|
|
||||||
timestamp = EARLIEST_OFFSET_TIMESTAMP;
|
|
||||||
else if (strategy == OffsetResetStrategy.LATEST)
|
|
||||||
timestamp = LATEST_OFFSET_TIMESTAMP;
|
|
||||||
else
|
|
||||||
throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined");
|
|
||||||
|
|
||||||
log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase());
|
|
||||||
long offset = listOffset(partition, timestamp);
|
|
||||||
this.subscriptions.seek(partition, offset);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Fetch a single offset before the given timestamp for the partition.
|
|
||||||
*
|
|
||||||
* @param partition The partition that needs fetching offset.
|
|
||||||
* @param timestamp The timestamp for fetching offset.
|
|
||||||
* @return The offset of the message that is published before the given timestamp
|
|
||||||
*/
|
|
||||||
private long listOffset(TopicPartition partition, long timestamp) {
|
|
||||||
while (true) {
|
|
||||||
RequestFuture<Long> future = fetcher.listOffset(partition, timestamp);
|
|
||||||
|
|
||||||
if (!future.isDone())
|
|
||||||
pollFuture(future, requestTimeoutMs);
|
|
||||||
|
|
||||||
if (future.isDone()) {
|
|
||||||
if (future.succeeded())
|
|
||||||
return future.value();
|
|
||||||
handleRequestFailure(future);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Refresh the committed offsets for given set of partitions and update the cache
|
|
||||||
*/
|
|
||||||
private void refreshCommittedOffsets(Set<TopicPartition> 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<TopicPartition, Long> offsets = fetchCommittedOffsets(partitions);
|
|
||||||
|
|
||||||
// update the position with the offsets
|
|
||||||
for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet()) {
|
|
||||||
TopicPartition tp = entry.getKey();
|
|
||||||
this.subscriptions.committed(tp, entry.getValue());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Block until we have received a partition assignment from the coordinator.
|
|
||||||
*/
|
|
||||||
private void assignPartitions() {
|
|
||||||
// Ensure that there are no pending requests to the coordinator. This is important
|
|
||||||
// in particular to avoid resending a pending JoinGroup request.
|
|
||||||
awaitCoordinatorInFlightRequests();
|
|
||||||
|
|
||||||
while (subscriptions.partitionAssignmentNeeded()) {
|
|
||||||
RequestFuture<Void> future = coordinator.assignPartitions(time.milliseconds());
|
|
||||||
|
|
||||||
// Block indefinitely for the join group request (which can take as long as a session timeout)
|
|
||||||
if (!future.isDone())
|
|
||||||
pollFuture(future);
|
|
||||||
|
|
||||||
if (future.failed())
|
|
||||||
handleRequestFailure(future);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Block until the coordinator for this group is known.
|
|
||||||
*/
|
|
||||||
private void ensureCoordinatorKnown() {
|
|
||||||
while (coordinator.coordinatorUnknown()) {
|
|
||||||
RequestFuture<Void> future = coordinator.discoverConsumerCoordinator();
|
|
||||||
|
|
||||||
if (!future.isDone())
|
|
||||||
pollFuture(future, requestTimeoutMs);
|
|
||||||
|
|
||||||
if (future.failed())
|
|
||||||
handleRequestFailure(future);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Block until any pending requests to the coordinator have been handled.
|
|
||||||
*/
|
|
||||||
public void awaitCoordinatorInFlightRequests() {
|
|
||||||
while (coordinator.hasInFlightRequests()) {
|
|
||||||
long now = time.milliseconds();
|
|
||||||
pollClient(-1, now);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Lookup the committed offsets for a set of partitions. This will block until the coordinator has
|
|
||||||
* responded to the offset fetch request.
|
|
||||||
* @param partitions List of partitions to get offsets for
|
|
||||||
* @return Map from partition to its respective offset
|
|
||||||
*/
|
|
||||||
private Map<TopicPartition, Long> fetchCommittedOffsets(Set<TopicPartition> partitions) {
|
|
||||||
while (true) {
|
|
||||||
long now = time.milliseconds();
|
|
||||||
RequestFuture<Map<TopicPartition, Long>> future = coordinator.fetchOffsets(partitions, now);
|
|
||||||
|
|
||||||
if (!future.isDone())
|
|
||||||
pollFuture(future, requestTimeoutMs);
|
|
||||||
|
|
||||||
if (future.isDone()) {
|
|
||||||
if (future.succeeded())
|
|
||||||
return future.value();
|
|
||||||
handleRequestFailure(future);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Commit offsets. This call blocks (regardless of commitType) until the coordinator
|
|
||||||
* can receive the commit request. Once the request has been made, however, only the
|
|
||||||
* synchronous commits will wait for a successful response from the coordinator.
|
|
||||||
* @param offsets Offsets to commit.
|
|
||||||
* @param commitType Commit policy
|
|
||||||
*/
|
|
||||||
private void commitOffsets(Map<TopicPartition, Long> offsets, CommitType commitType) {
|
|
||||||
if (commitType == CommitType.ASYNC) {
|
|
||||||
commitOffsetsAsync(offsets);
|
|
||||||
} else {
|
|
||||||
commitOffsetsSync(offsets);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void commitOffsetsAsync(Map<TopicPartition, Long> offsets) {
|
|
||||||
while (true) {
|
|
||||||
long now = time.milliseconds();
|
|
||||||
RequestFuture<Void> future = coordinator.commitOffsets(offsets, now);
|
|
||||||
|
|
||||||
if (!future.isDone() || future.succeeded())
|
|
||||||
return;
|
|
||||||
|
|
||||||
handleRequestFailure(future);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void commitOffsetsSync(Map<TopicPartition, Long> offsets) {
|
|
||||||
while (true) {
|
|
||||||
long now = time.milliseconds();
|
|
||||||
RequestFuture<Void> future = coordinator.commitOffsets(offsets, now);
|
|
||||||
|
|
||||||
if (!future.isDone())
|
|
||||||
pollFuture(future, requestTimeoutMs);
|
|
||||||
|
|
||||||
if (future.isDone()) {
|
|
||||||
if (future.succeeded())
|
|
||||||
return;
|
|
||||||
else
|
|
||||||
handleRequestFailure(future);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void handleRequestFailure(RequestFuture<?> future) {
|
|
||||||
if (future.hasException())
|
|
||||||
throw future.exception();
|
|
||||||
|
|
||||||
switch (future.retryAction()) {
|
|
||||||
case BACKOFF:
|
|
||||||
Utils.sleep(retryBackoffMs);
|
|
||||||
break;
|
|
||||||
case POLL:
|
|
||||||
pollClient(retryBackoffMs, time.milliseconds());
|
|
||||||
break;
|
|
||||||
case FIND_COORDINATOR:
|
|
||||||
ensureCoordinatorKnown();
|
|
||||||
break;
|
|
||||||
case REFRESH_METADATA:
|
|
||||||
awaitMetadataUpdate();
|
|
||||||
break;
|
|
||||||
case NOOP:
|
|
||||||
// Do nothing (retry now)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Poll until a result is ready or timeout expires
|
|
||||||
* @param future The future to poll for
|
|
||||||
* @param timeout The time in milliseconds to wait for the result
|
|
||||||
*/
|
|
||||||
private void pollFuture(RequestFuture<?> future, long timeout) {
|
|
||||||
// TODO: Update this code for KAFKA-2120, which adds request timeout to NetworkClient
|
|
||||||
// In particular, we must ensure that "timed out" requests will not have their callbacks
|
|
||||||
// invoked at a later time.
|
|
||||||
long remaining = timeout;
|
|
||||||
while (!future.isDone() && remaining >= 0) {
|
|
||||||
long start = time.milliseconds();
|
|
||||||
pollClient(remaining, start);
|
|
||||||
if (future.isDone()) return;
|
|
||||||
remaining -= time.milliseconds() - start;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Poll indefinitely until the result is ready.
|
|
||||||
* @param future The future to poll for.
|
|
||||||
*/
|
|
||||||
private void pollFuture(RequestFuture<?> future) {
|
|
||||||
while (!future.isDone()) {
|
|
||||||
long now = time.milliseconds();
|
|
||||||
pollClient(-1, now);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Poll for IO.
|
|
||||||
* @param timeout The maximum time to wait for IO to become available
|
|
||||||
* @param now The current time in milliseconds
|
|
||||||
* @throws ConsumerWakeupException if {@link #wakeup()} is invoked while the poll is active
|
|
||||||
*/
|
|
||||||
private void pollClient(long timeout, long now) {
|
|
||||||
this.client.poll(timeout, now);
|
|
||||||
|
|
||||||
if (wakeup.get()) {
|
|
||||||
wakeup.set(false);
|
|
||||||
throw new ConsumerWakeupException();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -1355,8 +1105,8 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
*/
|
*/
|
||||||
private void acquire() {
|
private void acquire() {
|
||||||
ensureNotClosed();
|
ensureNotClosed();
|
||||||
Long threadId = Thread.currentThread().getId();
|
long threadId = Thread.currentThread().getId();
|
||||||
if (!threadId.equals(currentThread.get()) && !currentThread.compareAndSet(null, threadId))
|
if (threadId != currentThread.get() && !currentThread.compareAndSet(NO_CURRENT_THREAD, threadId))
|
||||||
throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access");
|
throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access");
|
||||||
refcount.incrementAndGet();
|
refcount.incrementAndGet();
|
||||||
}
|
}
|
||||||
|
@ -1366,6 +1116,6 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
||||||
*/
|
*/
|
||||||
private void release() {
|
private void release() {
|
||||||
if (refcount.decrementAndGet() == 0)
|
if (refcount.decrementAndGet() == 0)
|
||||||
currentThread.set(null);
|
currentThread.set(NO_CURRENT_THREAD);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -106,16 +106,29 @@ public class MockConsumer<K, V> implements Consumer<K, V> {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType) {
|
public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
|
||||||
ensureNotClosed();
|
ensureNotClosed();
|
||||||
for (Entry<TopicPartition, Long> entry : offsets.entrySet())
|
for (Entry<TopicPartition, Long> entry : offsets.entrySet())
|
||||||
subscriptions.committed(entry.getKey(), entry.getValue());
|
subscriptions.committed(entry.getKey(), entry.getValue());
|
||||||
|
if (callback != null) {
|
||||||
|
callback.onComplete(offsets, null);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType) {
|
||||||
|
commit(offsets, commitType, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void commit(CommitType commitType, ConsumerCommitCallback callback) {
|
||||||
|
ensureNotClosed();
|
||||||
|
commit(this.subscriptions.allConsumed(), commitType, callback);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void commit(CommitType commitType) {
|
public synchronized void commit(CommitType commitType) {
|
||||||
ensureNotClosed();
|
commit(commitType, null);
|
||||||
commit(this.subscriptions.allConsumed(), commitType);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,296 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.ClientRequest;
|
||||||
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
|
import org.apache.kafka.clients.KafkaClient;
|
||||||
|
import org.apache.kafka.clients.Metadata;
|
||||||
|
import org.apache.kafka.clients.RequestCompletionHandler;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerWakeupException;
|
||||||
|
import org.apache.kafka.common.Node;
|
||||||
|
import org.apache.kafka.common.protocol.ApiKeys;
|
||||||
|
import org.apache.kafka.common.requests.AbstractRequest;
|
||||||
|
import org.apache.kafka.common.requests.RequestHeader;
|
||||||
|
import org.apache.kafka.common.requests.RequestSend;
|
||||||
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Higher level consumer access to the network layer with basic support for futures and
|
||||||
|
* task scheduling. NOT thread-safe!
|
||||||
|
*
|
||||||
|
* TODO: The current implementation is simplistic in that it provides a facility for queueing requests
|
||||||
|
* prior to delivery, but it makes no effort to retry requests which cannot be sent at the time
|
||||||
|
* {@link #poll(long)} is called. This makes the behavior of the queue predictable and easy to
|
||||||
|
* understand, but there are opportunities to provide timeout or retry capabilities in the future.
|
||||||
|
* How we do this may depend on KAFKA-2120, so for now, we retain the simplistic behavior.
|
||||||
|
*/
|
||||||
|
public class ConsumerNetworkClient implements Closeable {
|
||||||
|
private final KafkaClient client;
|
||||||
|
private final AtomicBoolean wakeup = new AtomicBoolean(false);
|
||||||
|
private final DelayedTaskQueue delayedTasks = new DelayedTaskQueue();
|
||||||
|
private final Map<Node, List<ClientRequest>> unsent = new HashMap<Node, List<ClientRequest>>();
|
||||||
|
private final Metadata metadata;
|
||||||
|
private final Time time;
|
||||||
|
private final long retryBackoffMs;
|
||||||
|
|
||||||
|
public ConsumerNetworkClient(KafkaClient client,
|
||||||
|
Metadata metadata,
|
||||||
|
Time time,
|
||||||
|
long retryBackoffMs) {
|
||||||
|
this.client = client;
|
||||||
|
this.metadata = metadata;
|
||||||
|
this.time = time;
|
||||||
|
this.retryBackoffMs = retryBackoffMs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Schedule a new task to be executed at the given time. This is "best-effort" scheduling and
|
||||||
|
* should only be used for coarse synchronization.
|
||||||
|
* @param task The task to be scheduled
|
||||||
|
* @param at The time it should run
|
||||||
|
*/
|
||||||
|
public void schedule(DelayedTask task, long at) {
|
||||||
|
delayedTasks.add(task, at);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unschedule a task. This will remove all instances of the task from the task queue.
|
||||||
|
* This is a no-op if the task is not scheduled.
|
||||||
|
* @param task The task to be unscheduled.
|
||||||
|
*/
|
||||||
|
public void unschedule(DelayedTask task) {
|
||||||
|
delayedTasks.remove(task);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send a new request. Note that the request is not actually transmitted on the
|
||||||
|
* network until one of the {@link #poll(long)} variants is invoked. At this
|
||||||
|
* point the request will either be transmitted successfully or will fail.
|
||||||
|
* Use the returned future to obtain the result of the send.
|
||||||
|
* @param node The destination of the request
|
||||||
|
* @param api The Kafka API call
|
||||||
|
* @param request The request payload
|
||||||
|
* @return A future which indicates the result of the send.
|
||||||
|
*/
|
||||||
|
public RequestFuture<ClientResponse> send(Node node,
|
||||||
|
ApiKeys api,
|
||||||
|
AbstractRequest request) {
|
||||||
|
long now = time.milliseconds();
|
||||||
|
RequestFutureCompletionHandler future = new RequestFutureCompletionHandler();
|
||||||
|
RequestHeader header = client.nextRequestHeader(api);
|
||||||
|
RequestSend send = new RequestSend(node.idString(), header, request.toStruct());
|
||||||
|
put(node, new ClientRequest(now, true, send, future));
|
||||||
|
return future;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void put(Node node, ClientRequest request) {
|
||||||
|
List<ClientRequest> nodeUnsent = unsent.get(node);
|
||||||
|
if (nodeUnsent == null) {
|
||||||
|
nodeUnsent = new ArrayList<ClientRequest>();
|
||||||
|
unsent.put(node, nodeUnsent);
|
||||||
|
}
|
||||||
|
nodeUnsent.add(request);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Node leastLoadedNode() {
|
||||||
|
return client.leastLoadedNode(time.milliseconds());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Block until the metadata has been refreshed.
|
||||||
|
*/
|
||||||
|
public void awaitMetadataUpdate() {
|
||||||
|
int version = this.metadata.requestUpdate();
|
||||||
|
do {
|
||||||
|
poll(Long.MAX_VALUE);
|
||||||
|
} while (this.metadata.version() == version);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wakeup an active poll. This will cause the polling thread to throw an exception either
|
||||||
|
* on the current poll if one is active, or the next poll.
|
||||||
|
*/
|
||||||
|
public void wakeup() {
|
||||||
|
this.wakeup.set(true);
|
||||||
|
this.client.wakeup();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Block indefinitely until the given request future has finished.
|
||||||
|
* @param future The request future to await.
|
||||||
|
* @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread
|
||||||
|
*/
|
||||||
|
public void poll(RequestFuture<?> future) {
|
||||||
|
while (!future.isDone())
|
||||||
|
poll(Long.MAX_VALUE);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Block until the provided request future request has finished or the timeout has expired.
|
||||||
|
* @param future The request future to wait for
|
||||||
|
* @param timeout The maximum duration (in ms) to wait for the request
|
||||||
|
* @return true if the future is done, false otherwise
|
||||||
|
* @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread
|
||||||
|
*/
|
||||||
|
public boolean poll(RequestFuture<?> future, long timeout) {
|
||||||
|
long now = time.milliseconds();
|
||||||
|
long deadline = now + timeout;
|
||||||
|
while (!future.isDone() && now < deadline) {
|
||||||
|
poll(deadline - now, now);
|
||||||
|
now = time.milliseconds();
|
||||||
|
}
|
||||||
|
return future.isDone();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Poll for any network IO. All send requests will either be transmitted on the network
|
||||||
|
* or failed when this call completes.
|
||||||
|
* @param timeout The maximum time to wait for an IO event.
|
||||||
|
* @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread
|
||||||
|
*/
|
||||||
|
public void poll(long timeout) {
|
||||||
|
poll(timeout, time.milliseconds());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void poll(long timeout, long now) {
|
||||||
|
// send all the requests we can send now
|
||||||
|
pollUnsentRequests(now);
|
||||||
|
|
||||||
|
// ensure we don't poll any longer than the deadline for
|
||||||
|
// the next scheduled task
|
||||||
|
timeout = Math.min(timeout, delayedTasks.nextTimeout(now));
|
||||||
|
clientPoll(timeout, now);
|
||||||
|
|
||||||
|
// execute scheduled tasks
|
||||||
|
now = time.milliseconds();
|
||||||
|
delayedTasks.poll(now);
|
||||||
|
|
||||||
|
// try again to send requests since buffer space may have been
|
||||||
|
// cleared or a connect finished in the poll
|
||||||
|
pollUnsentRequests(now);
|
||||||
|
|
||||||
|
// fail all requests that couldn't be sent
|
||||||
|
clearUnsentRequests(now);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Block until all pending requests from the given node have finished.
|
||||||
|
* @param node The node to await requests from
|
||||||
|
*/
|
||||||
|
public void awaitPendingRequests(Node node) {
|
||||||
|
while (pendingRequestCount(node) > 0)
|
||||||
|
poll(retryBackoffMs);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the count of pending requests to the given node. This includes both request that
|
||||||
|
* have been transmitted (i.e. in-flight requests) and those which are awaiting transmission.
|
||||||
|
* @param node The node in question
|
||||||
|
* @return The number of pending requests
|
||||||
|
*/
|
||||||
|
public int pendingRequestCount(Node node) {
|
||||||
|
List<ClientRequest> pending = unsent.get(node);
|
||||||
|
int unsentCount = pending == null ? 0 : pending.size();
|
||||||
|
return unsentCount + client.inFlightRequestCount(node.idString());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the total count of pending requests from all nodes. This includes both requests that
|
||||||
|
* have been transmitted (i.e. in-flight requests) and those which are awaiting transmission.
|
||||||
|
* @return The total count of pending requests
|
||||||
|
*/
|
||||||
|
public int pendingRequestCount() {
|
||||||
|
int total = 0;
|
||||||
|
for (List<ClientRequest> requests: unsent.values())
|
||||||
|
total += requests.size();
|
||||||
|
return total + client.inFlightRequestCount();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void pollUnsentRequests(long now) {
|
||||||
|
while (trySend(now))
|
||||||
|
clientPoll(0, now);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void clearUnsentRequests(long now) {
|
||||||
|
// clear all unsent requests and fail their corresponding futures
|
||||||
|
for (Map.Entry<Node, List<ClientRequest>> requestEntry: unsent.entrySet()) {
|
||||||
|
Iterator<ClientRequest> iterator = requestEntry.getValue().iterator();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
ClientRequest request = iterator.next();
|
||||||
|
RequestFutureCompletionHandler handler =
|
||||||
|
(RequestFutureCompletionHandler) request.callback();
|
||||||
|
handler.raise(SendFailedException.INSTANCE);
|
||||||
|
iterator.remove();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
unsent.clear();
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean trySend(long now) {
|
||||||
|
// send any requests that can be sent now
|
||||||
|
boolean requestsSent = false;
|
||||||
|
for (Map.Entry<Node, List<ClientRequest>> requestEntry: unsent.entrySet()) {
|
||||||
|
Node node = requestEntry.getKey();
|
||||||
|
Iterator<ClientRequest> iterator = requestEntry.getValue().iterator();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
ClientRequest request = iterator.next();
|
||||||
|
if (client.ready(node, now)) {
|
||||||
|
client.send(request);
|
||||||
|
iterator.remove();
|
||||||
|
requestsSent = true;
|
||||||
|
} else if (client.connectionFailed(node)) {
|
||||||
|
RequestFutureCompletionHandler handler =
|
||||||
|
(RequestFutureCompletionHandler) request.callback();
|
||||||
|
handler.onComplete(new ClientResponse(request, now, true, null));
|
||||||
|
iterator.remove();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return requestsSent;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void clientPoll(long timeout, long now) {
|
||||||
|
client.poll(timeout, now);
|
||||||
|
if (wakeup.get()) {
|
||||||
|
clearUnsentRequests(now);
|
||||||
|
wakeup.set(false);
|
||||||
|
throw new ConsumerWakeupException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
client.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RequestFutureCompletionHandler
|
||||||
|
extends RequestFuture<ClientResponse>
|
||||||
|
implements RequestCompletionHandler {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onComplete(ClientResponse response) {
|
||||||
|
complete(response);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -12,14 +12,14 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.clients.ClientRequest;
|
|
||||||
import org.apache.kafka.clients.ClientResponse;
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
import org.apache.kafka.clients.KafkaClient;
|
import org.apache.kafka.clients.consumer.CommitType;
|
||||||
import org.apache.kafka.clients.RequestCompletionHandler;
|
import org.apache.kafka.clients.consumer.ConsumerCommitCallback;
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.KafkaException;
|
||||||
import org.apache.kafka.common.MetricName;
|
import org.apache.kafka.common.MetricName;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.DisconnectException;
|
||||||
import org.apache.kafka.common.metrics.Measurable;
|
import org.apache.kafka.common.metrics.Measurable;
|
||||||
import org.apache.kafka.common.metrics.MetricConfig;
|
import org.apache.kafka.common.metrics.MetricConfig;
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
|
@ -30,7 +30,6 @@ import org.apache.kafka.common.metrics.stats.Max;
|
||||||
import org.apache.kafka.common.metrics.stats.Rate;
|
import org.apache.kafka.common.metrics.stats.Rate;
|
||||||
import org.apache.kafka.common.protocol.ApiKeys;
|
import org.apache.kafka.common.protocol.ApiKeys;
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
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.ConsumerMetadataRequest;
|
||||||
import org.apache.kafka.common.requests.ConsumerMetadataResponse;
|
import org.apache.kafka.common.requests.ConsumerMetadataResponse;
|
||||||
import org.apache.kafka.common.requests.HeartbeatRequest;
|
import org.apache.kafka.common.requests.HeartbeatRequest;
|
||||||
|
@ -41,15 +40,15 @@ import org.apache.kafka.common.requests.OffsetCommitRequest;
|
||||||
import org.apache.kafka.common.requests.OffsetCommitResponse;
|
import org.apache.kafka.common.requests.OffsetCommitResponse;
|
||||||
import org.apache.kafka.common.requests.OffsetFetchRequest;
|
import org.apache.kafka.common.requests.OffsetFetchRequest;
|
||||||
import org.apache.kafka.common.requests.OffsetFetchResponse;
|
import org.apache.kafka.common.requests.OffsetFetchResponse;
|
||||||
import org.apache.kafka.common.requests.RequestHeader;
|
|
||||||
import org.apache.kafka.common.requests.RequestSend;
|
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -62,23 +61,27 @@ public final class Coordinator {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(Coordinator.class);
|
private static final Logger log = LoggerFactory.getLogger(Coordinator.class);
|
||||||
|
|
||||||
private final KafkaClient client;
|
private final ConsumerNetworkClient client;
|
||||||
|
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final String groupId;
|
private final String groupId;
|
||||||
private final Heartbeat heartbeat;
|
private final Heartbeat heartbeat;
|
||||||
|
private final HeartbeatTask heartbeatTask;
|
||||||
private final int sessionTimeoutMs;
|
private final int sessionTimeoutMs;
|
||||||
private final String assignmentStrategy;
|
private final String assignmentStrategy;
|
||||||
private final SubscriptionState subscriptions;
|
private final SubscriptionState subscriptions;
|
||||||
private final CoordinatorMetrics sensors;
|
private final CoordinatorMetrics sensors;
|
||||||
|
private final long requestTimeoutMs;
|
||||||
|
private final long retryBackoffMs;
|
||||||
|
private final RebalanceCallback rebalanceCallback;
|
||||||
private Node consumerCoordinator;
|
private Node consumerCoordinator;
|
||||||
private String consumerId;
|
private String consumerId;
|
||||||
private int generation;
|
private int generation;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize the coordination manager.
|
* Initialize the coordination manager.
|
||||||
*/
|
*/
|
||||||
public Coordinator(KafkaClient client,
|
public Coordinator(ConsumerNetworkClient client,
|
||||||
String groupId,
|
String groupId,
|
||||||
int sessionTimeoutMs,
|
int sessionTimeoutMs,
|
||||||
String assignmentStrategy,
|
String assignmentStrategy,
|
||||||
|
@ -86,10 +89,13 @@ public final class Coordinator {
|
||||||
Metrics metrics,
|
Metrics metrics,
|
||||||
String metricGrpPrefix,
|
String metricGrpPrefix,
|
||||||
Map<String, String> metricTags,
|
Map<String, String> metricTags,
|
||||||
Time time) {
|
Time time,
|
||||||
|
long requestTimeoutMs,
|
||||||
|
long retryBackoffMs,
|
||||||
|
RebalanceCallback rebalanceCallback) {
|
||||||
|
|
||||||
this.time = time;
|
|
||||||
this.client = client;
|
this.client = client;
|
||||||
|
this.time = time;
|
||||||
this.generation = -1;
|
this.generation = -1;
|
||||||
this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
|
this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
|
||||||
this.groupId = groupId;
|
this.groupId = groupId;
|
||||||
|
@ -98,19 +104,190 @@ public final class Coordinator {
|
||||||
this.sessionTimeoutMs = sessionTimeoutMs;
|
this.sessionTimeoutMs = sessionTimeoutMs;
|
||||||
this.assignmentStrategy = assignmentStrategy;
|
this.assignmentStrategy = assignmentStrategy;
|
||||||
this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds());
|
this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds());
|
||||||
|
this.heartbeatTask = new HeartbeatTask();
|
||||||
this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
|
this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
|
||||||
|
this.requestTimeoutMs = requestTimeoutMs;
|
||||||
|
this.retryBackoffMs = retryBackoffMs;
|
||||||
|
this.rebalanceCallback = rebalanceCallback;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Refresh the committed offsets for provided partitions.
|
||||||
|
*/
|
||||||
|
public void refreshCommittedOffsetsIfNeeded() {
|
||||||
|
if (subscriptions.refreshCommitsNeeded()) {
|
||||||
|
Map<TopicPartition, Long> offsets = fetchCommittedOffsets(subscriptions.assignedPartitions());
|
||||||
|
for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet()) {
|
||||||
|
TopicPartition tp = entry.getKey();
|
||||||
|
this.subscriptions.committed(tp, entry.getValue());
|
||||||
|
}
|
||||||
|
this.subscriptions.commitsRefreshed();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Fetch the current committed offsets from the coordinator for a set of partitions.
|
||||||
|
* @param partitions The partitions to fetch offsets for
|
||||||
|
* @return A map from partition to the committed offset
|
||||||
|
*/
|
||||||
|
public Map<TopicPartition, Long> fetchCommittedOffsets(Set<TopicPartition> partitions) {
|
||||||
|
while (true) {
|
||||||
|
ensureCoordinatorKnown();
|
||||||
|
ensurePartitionAssignment();
|
||||||
|
|
||||||
|
// contact coordinator to fetch committed offsets
|
||||||
|
RequestFuture<Map<TopicPartition, Long>> future = sendOffsetFetchRequest(partitions);
|
||||||
|
client.poll(future);
|
||||||
|
|
||||||
|
if (future.succeeded())
|
||||||
|
return future.value();
|
||||||
|
|
||||||
|
if (!future.isRetriable())
|
||||||
|
throw future.exception();
|
||||||
|
|
||||||
|
Utils.sleep(retryBackoffMs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Ensure that we have a valid partition assignment from the coordinator.
|
||||||
|
*/
|
||||||
|
public void ensurePartitionAssignment() {
|
||||||
|
if (!subscriptions.partitionAssignmentNeeded())
|
||||||
|
return;
|
||||||
|
|
||||||
|
// execute the user's callback before rebalance
|
||||||
|
log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions());
|
||||||
|
try {
|
||||||
|
Set<TopicPartition> revoked = new HashSet<TopicPartition>(subscriptions.assignedPartitions());
|
||||||
|
rebalanceCallback.onPartitionsRevoked(revoked);
|
||||||
|
} catch (Exception e) {
|
||||||
|
log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
|
||||||
|
+ " failed on partition revocation: ", e);
|
||||||
|
}
|
||||||
|
|
||||||
|
reassignPartitions();
|
||||||
|
|
||||||
|
// execute the user's callback after rebalance
|
||||||
|
log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions());
|
||||||
|
try {
|
||||||
|
Set<TopicPartition> assigned = new HashSet<TopicPartition>(subscriptions.assignedPartitions());
|
||||||
|
rebalanceCallback.onPartitionsAssigned(assigned);
|
||||||
|
} catch (Exception e) {
|
||||||
|
log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
|
||||||
|
+ " failed on partition assignment: ", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void reassignPartitions() {
|
||||||
|
while (subscriptions.partitionAssignmentNeeded()) {
|
||||||
|
ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// ensure that there are no pending requests to the coordinator. This is important
|
||||||
|
// in particular to avoid resending a pending JoinGroup request.
|
||||||
|
if (client.pendingRequestCount(this.consumerCoordinator) > 0) {
|
||||||
|
client.awaitPendingRequests(this.consumerCoordinator);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
RequestFuture<Void> future = sendJoinGroupRequest();
|
||||||
|
client.poll(future);
|
||||||
|
|
||||||
|
if (future.failed()) {
|
||||||
|
if (!future.isRetriable())
|
||||||
|
throw future.exception();
|
||||||
|
Utils.sleep(retryBackoffMs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Block until the coordinator for this group is known.
|
||||||
|
*/
|
||||||
|
public void ensureCoordinatorKnown() {
|
||||||
|
while (coordinatorUnknown()) {
|
||||||
|
RequestFuture<Void> future = sendConsumerMetadataRequest();
|
||||||
|
client.poll(future, requestTimeoutMs);
|
||||||
|
|
||||||
|
if (future.failed())
|
||||||
|
client.awaitMetadataUpdate();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Commit offsets. This call blocks (regardless of commitType) until the coordinator
|
||||||
|
* can receive the commit request. Once the request has been made, however, only the
|
||||||
|
* synchronous commits will wait for a successful response from the coordinator.
|
||||||
|
* @param offsets Offsets to commit.
|
||||||
|
* @param commitType Commit policy
|
||||||
|
* @param callback Callback to be executed when the commit request finishes
|
||||||
|
*/
|
||||||
|
public void commitOffsets(Map<TopicPartition, Long> offsets, CommitType commitType, ConsumerCommitCallback callback) {
|
||||||
|
if (commitType == CommitType.ASYNC)
|
||||||
|
commitOffsetsAsync(offsets, callback);
|
||||||
|
else
|
||||||
|
commitOffsetsSync(offsets, callback);
|
||||||
|
}
|
||||||
|
|
||||||
|
private class HeartbeatTask implements DelayedTask {
|
||||||
|
|
||||||
|
public void reset() {
|
||||||
|
// start or restart the heartbeat task to be executed at the next chance
|
||||||
|
long now = time.milliseconds();
|
||||||
|
heartbeat.resetSessionTimeout(now);
|
||||||
|
client.unschedule(this);
|
||||||
|
client.schedule(this, now);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run(final long now) {
|
||||||
|
if (!subscriptions.partitionsAutoAssigned() ||
|
||||||
|
subscriptions.partitionAssignmentNeeded() ||
|
||||||
|
coordinatorUnknown())
|
||||||
|
// no need to send if we're not using auto-assignment or if we are
|
||||||
|
// awaiting a rebalance
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (heartbeat.sessionTimeoutExpired(now)) {
|
||||||
|
// we haven't received a successful heartbeat in one session interval
|
||||||
|
// so mark the coordinator dead
|
||||||
|
coordinatorDead();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!heartbeat.shouldHeartbeat(now)) {
|
||||||
|
// we don't need to heartbeat now, so reschedule for when we do
|
||||||
|
client.schedule(this, now + heartbeat.timeToNextHeartbeat(now));
|
||||||
|
} else {
|
||||||
|
heartbeat.sentHeartbeat(now);
|
||||||
|
RequestFuture<Void> future = sendHeartbeatRequest();
|
||||||
|
future.addListener(new RequestFutureListener<Void>() {
|
||||||
|
@Override
|
||||||
|
public void onSuccess(Void value) {
|
||||||
|
long now = time.milliseconds();
|
||||||
|
heartbeat.receiveHeartbeat(now);
|
||||||
|
long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now);
|
||||||
|
client.schedule(HeartbeatTask.this, nextHeartbeatTime);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(RuntimeException e) {
|
||||||
|
client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Send a request to get a new partition assignment. This is a non-blocking call which sends
|
* Send a request to get a new partition assignment. This is a non-blocking call which sends
|
||||||
* a JoinGroup request to the coordinator (if it is available). The returned future must
|
* a JoinGroup request to the coordinator (if it is available). The returned future must
|
||||||
* be polled to see if the request completed successfully.
|
* be polled to see if the request completed successfully.
|
||||||
* @param now The current time in milliseconds
|
|
||||||
* @return A request future whose completion indicates the result of the JoinGroup request.
|
* @return A request future whose completion indicates the result of the JoinGroup request.
|
||||||
*/
|
*/
|
||||||
public RequestFuture<Void> assignPartitions(final long now) {
|
private RequestFuture<Void> sendJoinGroupRequest() {
|
||||||
final RequestFuture<Void> future = newCoordinatorRequestFuture(now);
|
if (coordinatorUnknown())
|
||||||
if (future.isDone()) return future;
|
return RequestFuture.coordinatorNotAvailable();
|
||||||
|
|
||||||
// send a join group request to the coordinator
|
// send a join group request to the coordinator
|
||||||
List<String> subscribedTopics = new ArrayList<String>(subscriptions.subscribedTopics());
|
List<String> subscribedTopics = new ArrayList<String>(subscriptions.subscribedTopics());
|
||||||
|
@ -124,25 +301,20 @@ public final class Coordinator {
|
||||||
|
|
||||||
// create the request for the coordinator
|
// create the request for the coordinator
|
||||||
log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id());
|
log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id());
|
||||||
|
return client.send(consumerCoordinator, ApiKeys.JOIN_GROUP, request)
|
||||||
|
.compose(new JoinGroupResponseHandler());
|
||||||
|
}
|
||||||
|
|
||||||
|
private class JoinGroupResponseHandler extends CoordinatorResponseHandler<JoinGroupResponse, Void> {
|
||||||
|
|
||||||
RequestCompletionHandler completionHandler = new RequestCompletionHandler() {
|
|
||||||
@Override
|
@Override
|
||||||
public void onComplete(ClientResponse resp) {
|
public JoinGroupResponse parse(ClientResponse response) {
|
||||||
handleJoinResponse(resp, future);
|
return new JoinGroupResponse(response.responseBody());
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
sendCoordinator(ApiKeys.JOIN_GROUP, request.toStruct(), completionHandler, now);
|
|
||||||
return future;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void handleJoinResponse(ClientResponse response, RequestFuture<Void> future) {
|
@Override
|
||||||
if (response.wasDisconnected()) {
|
public void handle(JoinGroupResponse joinResponse, RequestFuture<Void> future) {
|
||||||
handleCoordinatorDisconnect(response);
|
|
||||||
future.retryWithNewCoordinator();
|
|
||||||
} else {
|
|
||||||
// process the response
|
// process the response
|
||||||
JoinGroupResponse joinResponse = new JoinGroupResponse(response.responseBody());
|
|
||||||
short errorCode = joinResponse.errorCode();
|
short errorCode = joinResponse.errorCode();
|
||||||
|
|
||||||
if (errorCode == Errors.NONE.code()) {
|
if (errorCode == Errors.NONE.code()) {
|
||||||
|
@ -152,36 +324,36 @@ public final class Coordinator {
|
||||||
// set the flag to refresh last committed offsets
|
// set the flag to refresh last committed offsets
|
||||||
subscriptions.needRefreshCommits();
|
subscriptions.needRefreshCommits();
|
||||||
|
|
||||||
log.debug("Joined group: {}", response);
|
log.debug("Joined group: {}", joinResponse.toStruct());
|
||||||
|
|
||||||
// record re-assignment time
|
// record re-assignment time
|
||||||
this.sensors.partitionReassignments.record(response.requestLatencyMs());
|
sensors.partitionReassignments.record(response.requestLatencyMs());
|
||||||
|
|
||||||
// update partition assignment
|
// update partition assignment
|
||||||
subscriptions.changePartitionAssignment(joinResponse.assignedPartitions());
|
subscriptions.changePartitionAssignment(joinResponse.assignedPartitions());
|
||||||
|
heartbeatTask.reset();
|
||||||
future.complete(null);
|
future.complete(null);
|
||||||
} else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) {
|
} else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) {
|
||||||
// reset the consumer id and retry immediately
|
// reset the consumer id and retry immediately
|
||||||
Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
|
Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
|
||||||
log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.",
|
log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.",
|
||||||
groupId);
|
groupId);
|
||||||
|
future.raise(Errors.UNKNOWN_CONSUMER_ID);
|
||||||
future.retryNow();
|
|
||||||
} else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|
} else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|
||||||
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|
||||||
// re-discover the coordinator and retry with backoff
|
// re-discover the coordinator and retry with backoff
|
||||||
coordinatorDead();
|
coordinatorDead();
|
||||||
log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.",
|
log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.",
|
||||||
groupId);
|
groupId);
|
||||||
future.retryWithNewCoordinator();
|
future.raise(Errors.forCode(errorCode));
|
||||||
} else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()
|
} else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()
|
||||||
|| errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code()
|
|| errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code()
|
||||||
|| errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) {
|
|| errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) {
|
||||||
// log the error and re-throw the exception
|
// log the error and re-throw the exception
|
||||||
KafkaException e = Errors.forCode(errorCode).exception();
|
Errors error = Errors.forCode(errorCode);
|
||||||
log.error("Attempt to join group {} failed due to: {}",
|
log.error("Attempt to join group {} failed due to: {}",
|
||||||
groupId, e.getMessage());
|
groupId, error.exception().getMessage());
|
||||||
future.raise(e);
|
future.raise(error);
|
||||||
} else {
|
} else {
|
||||||
// unexpected error, throw the exception
|
// unexpected error, throw the exception
|
||||||
future.raise(new KafkaException("Unexpected error in join group response: "
|
future.raise(new KafkaException("Unexpected error in join group response: "
|
||||||
|
@ -190,22 +362,65 @@ public final class Coordinator {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void commitOffsetsAsync(final Map<TopicPartition, Long> offsets, final ConsumerCommitCallback callback) {
|
||||||
|
this.subscriptions.needRefreshCommits();
|
||||||
|
RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
|
||||||
|
if (callback != null) {
|
||||||
|
future.addListener(new RequestFutureListener<Void>() {
|
||||||
|
@Override
|
||||||
|
public void onSuccess(Void value) {
|
||||||
|
callback.onComplete(offsets, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(RuntimeException e) {
|
||||||
|
callback.onComplete(offsets, e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void commitOffsetsSync(Map<TopicPartition, Long> offsets, ConsumerCommitCallback callback) {
|
||||||
|
while (true) {
|
||||||
|
ensureCoordinatorKnown();
|
||||||
|
ensurePartitionAssignment();
|
||||||
|
|
||||||
|
RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
|
||||||
|
client.poll(future);
|
||||||
|
|
||||||
|
if (future.succeeded()) {
|
||||||
|
if (callback != null)
|
||||||
|
callback.onComplete(offsets, null);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!future.isRetriable()) {
|
||||||
|
if (callback == null)
|
||||||
|
throw future.exception();
|
||||||
|
else
|
||||||
|
callback.onComplete(offsets, future.exception());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
Utils.sleep(retryBackoffMs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Commit offsets for the specified list of topics and partitions. This is a non-blocking call
|
* Commit offsets for the specified list of topics and partitions. This is a non-blocking call
|
||||||
* which returns a request future that can be polled in the case of a synchronous commit or ignored in the
|
* which returns a request future that can be polled in the case of a synchronous commit or ignored in the
|
||||||
* asynchronous case.
|
* asynchronous case.
|
||||||
*
|
*
|
||||||
* @param offsets The list of offsets per partition that should be committed.
|
* @param offsets The list of offsets per partition that should be committed.
|
||||||
* @param now The current time
|
|
||||||
* @return A request future whose value indicates whether the commit was successful or not
|
* @return A request future whose value indicates whether the commit was successful or not
|
||||||
*/
|
*/
|
||||||
public RequestFuture<Void> commitOffsets(final Map<TopicPartition, Long> offsets, long now) {
|
private RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, Long> offsets) {
|
||||||
final RequestFuture<Void> future = newCoordinatorRequestFuture(now);
|
if (coordinatorUnknown())
|
||||||
if (future.isDone()) return future;
|
return RequestFuture.coordinatorNotAvailable();
|
||||||
|
|
||||||
|
if (offsets.isEmpty())
|
||||||
|
return RequestFuture.voidSuccess();
|
||||||
|
|
||||||
if (offsets.isEmpty()) {
|
|
||||||
future.complete(null);
|
|
||||||
} else {
|
|
||||||
// create the offset commit request
|
// create the offset commit request
|
||||||
Map<TopicPartition, OffsetCommitRequest.PartitionData> offsetData;
|
Map<TopicPartition, OffsetCommitRequest.PartitionData> offsetData;
|
||||||
offsetData = new HashMap<TopicPartition, OffsetCommitRequest.PartitionData>(offsets.size());
|
offsetData = new HashMap<TopicPartition, OffsetCommitRequest.PartitionData>(offsets.size());
|
||||||
|
@ -217,28 +432,64 @@ public final class Coordinator {
|
||||||
OffsetCommitRequest.DEFAULT_RETENTION_TIME,
|
OffsetCommitRequest.DEFAULT_RETENTION_TIME,
|
||||||
offsetData);
|
offsetData);
|
||||||
|
|
||||||
RequestCompletionHandler handler = new OffsetCommitCompletionHandler(offsets, future);
|
return client.send(consumerCoordinator, ApiKeys.OFFSET_COMMIT, req)
|
||||||
sendCoordinator(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now);
|
.compose(new OffsetCommitResponseHandler(offsets));
|
||||||
}
|
}
|
||||||
|
|
||||||
return future;
|
|
||||||
|
private class OffsetCommitResponseHandler extends CoordinatorResponseHandler<OffsetCommitResponse, Void> {
|
||||||
|
|
||||||
|
private final Map<TopicPartition, Long> offsets;
|
||||||
|
|
||||||
|
public OffsetCommitResponseHandler(Map<TopicPartition, Long> offsets) {
|
||||||
|
this.offsets = offsets;
|
||||||
}
|
}
|
||||||
|
|
||||||
private <T> RequestFuture<T> newCoordinatorRequestFuture(long now) {
|
@Override
|
||||||
if (coordinatorUnknown())
|
public OffsetCommitResponse parse(ClientResponse response) {
|
||||||
return RequestFuture.newCoordinatorNeeded();
|
return new OffsetCommitResponse(response.responseBody());
|
||||||
|
}
|
||||||
|
|
||||||
if (client.ready(this.consumerCoordinator, now))
|
@Override
|
||||||
// We have an open connection and we're ready to send
|
public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> future) {
|
||||||
return new RequestFuture<T>();
|
sensors.commitLatency.record(response.requestLatencyMs());
|
||||||
|
for (Map.Entry<TopicPartition, Short> entry : commitResponse.responseData().entrySet()) {
|
||||||
if (this.client.connectionFailed(this.consumerCoordinator)) {
|
TopicPartition tp = entry.getKey();
|
||||||
|
long offset = this.offsets.get(tp);
|
||||||
|
short errorCode = entry.getValue();
|
||||||
|
if (errorCode == Errors.NONE.code()) {
|
||||||
|
log.debug("Committed offset {} for partition {}", offset, tp);
|
||||||
|
subscriptions.committed(tp, offset);
|
||||||
|
} else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|
||||||
|
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|
||||||
coordinatorDead();
|
coordinatorDead();
|
||||||
return RequestFuture.newCoordinatorNeeded();
|
future.raise(Errors.forCode(errorCode));
|
||||||
|
return;
|
||||||
|
} else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code()
|
||||||
|
|| errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) {
|
||||||
|
// do not need to throw the exception but just log the error
|
||||||
|
log.error("Error committing partition {} at offset {}: {}",
|
||||||
|
tp,
|
||||||
|
offset,
|
||||||
|
Errors.forCode(errorCode).exception().getMessage());
|
||||||
|
} else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
|
||||||
|
|| errorCode == Errors.ILLEGAL_GENERATION.code()) {
|
||||||
|
// need to re-join group
|
||||||
|
subscriptions.needReassignment();
|
||||||
|
future.raise(Errors.forCode(errorCode));
|
||||||
|
return;
|
||||||
|
} else {
|
||||||
|
// do not need to throw the exception but just log the error
|
||||||
|
future.raise(Errors.forCode(errorCode));
|
||||||
|
log.error("Error committing partition {} at offset {}: {}",
|
||||||
|
tp,
|
||||||
|
offset,
|
||||||
|
Errors.forCode(errorCode).exception().getMessage());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// The connection has been initiated, so we need to poll to finish it
|
future.complete(null);
|
||||||
return RequestFuture.pollNeeded();
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -246,35 +497,30 @@ public final class Coordinator {
|
||||||
* returned future can be polled to get the actual offsets returned from the broker.
|
* returned future can be polled to get the actual offsets returned from the broker.
|
||||||
*
|
*
|
||||||
* @param partitions The set of partitions to get offsets for.
|
* @param partitions The set of partitions to get offsets for.
|
||||||
* @param now The current time in milliseconds
|
|
||||||
* @return A request future containing the committed offsets.
|
* @return A request future containing the committed offsets.
|
||||||
*/
|
*/
|
||||||
public RequestFuture<Map<TopicPartition, Long>> fetchOffsets(Set<TopicPartition> partitions, long now) {
|
private RequestFuture<Map<TopicPartition, Long>> sendOffsetFetchRequest(Set<TopicPartition> partitions) {
|
||||||
final RequestFuture<Map<TopicPartition, Long>> future = newCoordinatorRequestFuture(now);
|
if (coordinatorUnknown())
|
||||||
if (future.isDone()) return future;
|
return RequestFuture.coordinatorNotAvailable();
|
||||||
|
|
||||||
log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", "));
|
log.debug("Fetching committed offsets for partitions: {}", Utils.join(partitions, ", "));
|
||||||
// construct the request
|
// construct the request
|
||||||
OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList<TopicPartition>(partitions));
|
OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList<TopicPartition>(partitions));
|
||||||
|
|
||||||
// send the request with a callback
|
// send the request with a callback
|
||||||
RequestCompletionHandler completionHandler = new RequestCompletionHandler() {
|
return client.send(consumerCoordinator, ApiKeys.OFFSET_FETCH, request)
|
||||||
@Override
|
.compose(new OffsetFetchResponseHandler());
|
||||||
public void onComplete(ClientResponse resp) {
|
|
||||||
handleOffsetFetchResponse(resp, future);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
sendCoordinator(ApiKeys.OFFSET_FETCH, request.toStruct(), completionHandler, now);
|
|
||||||
return future;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void handleOffsetFetchResponse(ClientResponse resp, RequestFuture<Map<TopicPartition, Long>> future) {
|
private class OffsetFetchResponseHandler extends CoordinatorResponseHandler<OffsetFetchResponse, Map<TopicPartition, Long>> {
|
||||||
if (resp.wasDisconnected()) {
|
|
||||||
handleCoordinatorDisconnect(resp);
|
@Override
|
||||||
future.retryWithNewCoordinator();
|
public OffsetFetchResponse parse(ClientResponse response) {
|
||||||
} else {
|
return new OffsetFetchResponse(response.responseBody());
|
||||||
// parse the response to get the offsets
|
}
|
||||||
OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody());
|
|
||||||
|
@Override
|
||||||
|
public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, Long>> future) {
|
||||||
Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>(response.responseData().size());
|
Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>(response.responseData().size());
|
||||||
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
|
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
|
||||||
TopicPartition tp = entry.getKey();
|
TopicPartition tp = entry.getKey();
|
||||||
|
@ -285,19 +531,21 @@ public final class Coordinator {
|
||||||
.getMessage());
|
.getMessage());
|
||||||
if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) {
|
if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) {
|
||||||
// just retry
|
// just retry
|
||||||
future.retryAfterBackoff();
|
future.raise(Errors.OFFSET_LOAD_IN_PROGRESS);
|
||||||
} else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|
} else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|
||||||
// re-discover the coordinator and retry
|
// re-discover the coordinator and retry
|
||||||
coordinatorDead();
|
coordinatorDead();
|
||||||
future.retryWithNewCoordinator();
|
future.raise(Errors.NOT_COORDINATOR_FOR_CONSUMER);
|
||||||
} else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
|
} else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
|
||||||
|| data.errorCode == Errors.ILLEGAL_GENERATION.code()) {
|
|| data.errorCode == Errors.ILLEGAL_GENERATION.code()) {
|
||||||
// need to re-join group
|
// need to re-join group
|
||||||
subscriptions.needReassignment();
|
subscriptions.needReassignment();
|
||||||
|
future.raise(Errors.forCode(data.errorCode));
|
||||||
} else {
|
} else {
|
||||||
future.raise(new KafkaException("Unexpected error in fetch offset response: "
|
future.raise(new KafkaException("Unexpected error in fetch offset response: "
|
||||||
+ Errors.forCode(data.errorCode).exception().getMessage()));
|
+ Errors.forCode(data.errorCode).exception().getMessage()));
|
||||||
}
|
}
|
||||||
|
return;
|
||||||
} else if (data.offset >= 0) {
|
} else if (data.offset >= 0) {
|
||||||
// record the position with the offset (-1 indicates no committed offset to fetch)
|
// record the position with the offset (-1 indicates no committed offset to fetch)
|
||||||
offsets.put(tp, data.offset);
|
offsets.put(tp, data.offset);
|
||||||
|
@ -306,82 +554,47 @@ public final class Coordinator {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!future.isDone())
|
|
||||||
future.complete(offsets);
|
future.complete(offsets);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Attempt to heartbeat the consumer coordinator if necessary, and check if the coordinator is still alive.
|
* Send a heartbeat request now (visible only for testing).
|
||||||
*
|
|
||||||
* @param now The current time
|
|
||||||
*/
|
*/
|
||||||
public void maybeHeartbeat(long now) {
|
public RequestFuture<Void> sendHeartbeatRequest() {
|
||||||
if (heartbeat.shouldHeartbeat(now) && coordinatorReady(now)) {
|
|
||||||
HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId);
|
HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId);
|
||||||
sendCoordinator(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now);
|
return client.send(consumerCoordinator, ApiKeys.HEARTBEAT, req)
|
||||||
this.heartbeat.sentHeartbeat(now);
|
.compose(new HeartbeatCompletionHandler());
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the time until the next heartbeat is needed.
|
|
||||||
* @param now The current time
|
|
||||||
* @return The duration in milliseconds before the next heartbeat will be needed.
|
|
||||||
*/
|
|
||||||
public long timeToNextHeartbeat(long now) {
|
|
||||||
return heartbeat.timeToNextHeartbeat(now);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Check whether the coordinator has any in-flight requests.
|
|
||||||
* @return true if the coordinator has pending requests.
|
|
||||||
*/
|
|
||||||
public boolean hasInFlightRequests() {
|
|
||||||
return !coordinatorUnknown() && client.inFlightRequestCount(consumerCoordinator.idString()) > 0;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean coordinatorUnknown() {
|
public boolean coordinatorUnknown() {
|
||||||
return this.consumerCoordinator == null;
|
return this.consumerCoordinator == null;
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean coordinatorReady(long now) {
|
|
||||||
return !coordinatorUnknown() && this.client.ready(this.consumerCoordinator, now);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to
|
* Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to
|
||||||
* one of the brokers. The returned future should be polled to get the result of the request.
|
* one of the brokers. The returned future should be polled to get the result of the request.
|
||||||
* @return A request future which indicates the completion of the metadata request
|
* @return A request future which indicates the completion of the metadata request
|
||||||
*/
|
*/
|
||||||
public RequestFuture<Void> discoverConsumerCoordinator() {
|
private RequestFuture<Void> sendConsumerMetadataRequest() {
|
||||||
// initiate the consumer metadata request
|
// initiate the consumer metadata request
|
||||||
// find a node to ask about the coordinator
|
// find a node to ask about the coordinator
|
||||||
long now = time.milliseconds();
|
Node node = this.client.leastLoadedNode();
|
||||||
Node node = this.client.leastLoadedNode(now);
|
|
||||||
|
|
||||||
if (node == null) {
|
if (node == null) {
|
||||||
return RequestFuture.metadataRefreshNeeded();
|
// TODO: If there are no brokers left, perhaps we should use the bootstrap set
|
||||||
} else if (!this.client.ready(node, now)) {
|
// from configuration?
|
||||||
if (this.client.connectionFailed(node)) {
|
return RequestFuture.noBrokersAvailable();
|
||||||
return RequestFuture.metadataRefreshNeeded();
|
|
||||||
} else {
|
} else {
|
||||||
return RequestFuture.pollNeeded();
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
final RequestFuture<Void> future = new RequestFuture<Void>();
|
|
||||||
|
|
||||||
// create a consumer metadata request
|
// create a consumer metadata request
|
||||||
log.debug("Issuing consumer metadata request to broker {}", node.id());
|
log.debug("Issuing consumer metadata request to broker {}", node.id());
|
||||||
ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId);
|
ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId);
|
||||||
RequestCompletionHandler completionHandler = new RequestCompletionHandler() {
|
return client.send(node, ApiKeys.CONSUMER_METADATA, metadataRequest)
|
||||||
|
.compose(new RequestFutureAdapter<ClientResponse, Void>() {
|
||||||
@Override
|
@Override
|
||||||
public void onComplete(ClientResponse resp) {
|
public void onSuccess(ClientResponse response, RequestFuture<Void> future) {
|
||||||
handleConsumerMetadataResponse(resp, future);
|
handleConsumerMetadataResponse(response, future);
|
||||||
}
|
}
|
||||||
};
|
});
|
||||||
send(node, ApiKeys.CONSUMER_METADATA, metadataRequest.toStruct(), completionHandler, now);
|
|
||||||
return future;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -391,7 +604,10 @@ public final class Coordinator {
|
||||||
// parse the response to get the coordinator info if it is not disconnected,
|
// parse the response to get the coordinator info if it is not disconnected,
|
||||||
// otherwise we need to request metadata update
|
// otherwise we need to request metadata update
|
||||||
if (resp.wasDisconnected()) {
|
if (resp.wasDisconnected()) {
|
||||||
future.retryAfterMetadataRefresh();
|
future.raise(new DisconnectException());
|
||||||
|
} else if (!coordinatorUnknown()) {
|
||||||
|
// We already found the coordinator, so ignore the request
|
||||||
|
future.complete(null);
|
||||||
} else {
|
} else {
|
||||||
ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody());
|
ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody());
|
||||||
// use MAX_VALUE - node.id as the coordinator id to mimic separate connections
|
// use MAX_VALUE - node.id as the coordinator id to mimic separate connections
|
||||||
|
@ -401,9 +617,10 @@ public final class Coordinator {
|
||||||
this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(),
|
this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(),
|
||||||
consumerMetadataResponse.node().host(),
|
consumerMetadataResponse.node().host(),
|
||||||
consumerMetadataResponse.node().port());
|
consumerMetadataResponse.node().port());
|
||||||
|
heartbeatTask.reset();
|
||||||
future.complete(null);
|
future.complete(null);
|
||||||
} else {
|
} else {
|
||||||
future.retryAfterBackoff();
|
future.raise(Errors.forCode(consumerMetadataResponse.errorCode()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -418,10 +635,53 @@ public final class Coordinator {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
private class HeartbeatCompletionHandler extends CoordinatorResponseHandler<HeartbeatResponse, Void> {
|
||||||
* Handle the case when the request gets cancelled due to coordinator disconnection.
|
@Override
|
||||||
*/
|
public HeartbeatResponse parse(ClientResponse response) {
|
||||||
private void handleCoordinatorDisconnect(ClientResponse response) {
|
return new HeartbeatResponse(response.responseBody());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void handle(HeartbeatResponse heartbeatResponse, RequestFuture<Void> future) {
|
||||||
|
sensors.heartbeatLatency.record(response.requestLatencyMs());
|
||||||
|
short error = heartbeatResponse.errorCode();
|
||||||
|
if (error == Errors.NONE.code()) {
|
||||||
|
log.debug("Received successful heartbeat response.");
|
||||||
|
future.complete(null);
|
||||||
|
} else if (error == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|
||||||
|
|| error == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|
||||||
|
log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead.");
|
||||||
|
coordinatorDead();
|
||||||
|
future.raise(Errors.forCode(error));
|
||||||
|
} else if (error == Errors.ILLEGAL_GENERATION.code()) {
|
||||||
|
log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group.");
|
||||||
|
subscriptions.needReassignment();
|
||||||
|
future.raise(Errors.ILLEGAL_GENERATION);
|
||||||
|
} else if (error == Errors.UNKNOWN_CONSUMER_ID.code()) {
|
||||||
|
log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group.");
|
||||||
|
consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
|
||||||
|
subscriptions.needReassignment();
|
||||||
|
future.raise(Errors.UNKNOWN_CONSUMER_ID);
|
||||||
|
} else {
|
||||||
|
future.raise(new KafkaException("Unexpected error in heartbeat response: "
|
||||||
|
+ Errors.forCode(error).exception().getMessage()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private abstract class CoordinatorResponseHandler<R, T>
|
||||||
|
extends RequestFutureAdapter<ClientResponse, T> {
|
||||||
|
protected ClientResponse response;
|
||||||
|
|
||||||
|
public abstract R parse(ClientResponse response);
|
||||||
|
|
||||||
|
public abstract void handle(R response, RequestFuture<T> future);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onSuccess(ClientResponse clientResponse, RequestFuture<T> future) {
|
||||||
|
this.response = clientResponse;
|
||||||
|
|
||||||
|
if (clientResponse.wasDisconnected()) {
|
||||||
int correlation = response.request().request().header().correlationId();
|
int correlation = response.request().request().header().correlationId();
|
||||||
log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected",
|
log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected",
|
||||||
response.request(),
|
response.request(),
|
||||||
|
@ -430,101 +690,27 @@ public final class Coordinator {
|
||||||
|
|
||||||
// mark the coordinator as dead
|
// mark the coordinator as dead
|
||||||
coordinatorDead();
|
coordinatorDead();
|
||||||
|
future.raise(new DisconnectException());
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
R response = parse(clientResponse);
|
||||||
private void sendCoordinator(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) {
|
handle(response, future);
|
||||||
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.idString(), header, request);
|
|
||||||
this.client.send(new ClientRequest(now, true, send, handler));
|
|
||||||
}
|
|
||||||
|
|
||||||
private class HeartbeatCompletionHandler implements RequestCompletionHandler {
|
|
||||||
@Override
|
|
||||||
public void onComplete(ClientResponse resp) {
|
|
||||||
if (resp.wasDisconnected()) {
|
|
||||||
handleCoordinatorDisconnect(resp);
|
|
||||||
} else {
|
|
||||||
HeartbeatResponse response = new HeartbeatResponse(resp.responseBody());
|
|
||||||
if (response.errorCode() == Errors.NONE.code()) {
|
|
||||||
log.debug("Received successful heartbeat response.");
|
|
||||||
} else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|
|
||||||
|| response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|
|
||||||
log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead.");
|
|
||||||
coordinatorDead();
|
|
||||||
} else if (response.errorCode() == Errors.ILLEGAL_GENERATION.code()) {
|
|
||||||
log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group.");
|
|
||||||
subscriptions.needReassignment();
|
|
||||||
} else if (response.errorCode() == Errors.UNKNOWN_CONSUMER_ID.code()) {
|
|
||||||
log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group.");
|
|
||||||
consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
|
|
||||||
subscriptions.needReassignment();
|
|
||||||
} else {
|
|
||||||
throw new KafkaException("Unexpected error in heartbeat response: "
|
|
||||||
+ Errors.forCode(response.errorCode()).exception().getMessage());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
sensors.heartbeatLatency.record(resp.requestLatencyMs());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private class OffsetCommitCompletionHandler implements RequestCompletionHandler {
|
|
||||||
|
|
||||||
private final Map<TopicPartition, Long> offsets;
|
|
||||||
private final RequestFuture<Void> future;
|
|
||||||
|
|
||||||
public OffsetCommitCompletionHandler(Map<TopicPartition, Long> offsets, RequestFuture<Void> future) {
|
|
||||||
this.offsets = offsets;
|
|
||||||
this.future = future;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onComplete(ClientResponse resp) {
|
public void onFailure(RuntimeException e, RequestFuture<T> future) {
|
||||||
if (resp.wasDisconnected()) {
|
if (e instanceof DisconnectException) {
|
||||||
handleCoordinatorDisconnect(resp);
|
log.debug("Coordinator request failed", e);
|
||||||
future.retryWithNewCoordinator();
|
|
||||||
} else {
|
|
||||||
OffsetCommitResponse commitResponse = new OffsetCommitResponse(resp.responseBody());
|
|
||||||
for (Map.Entry<TopicPartition, Short> entry : commitResponse.responseData().entrySet()) {
|
|
||||||
TopicPartition tp = entry.getKey();
|
|
||||||
short errorCode = entry.getValue();
|
|
||||||
long offset = this.offsets.get(tp);
|
|
||||||
if (errorCode == Errors.NONE.code()) {
|
|
||||||
log.debug("Committed offset {} for partition {}", offset, tp);
|
|
||||||
subscriptions.committed(tp, offset);
|
|
||||||
} else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|
|
||||||
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|
|
||||||
coordinatorDead();
|
coordinatorDead();
|
||||||
future.retryWithNewCoordinator();
|
}
|
||||||
} else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code()
|
future.raise(e);
|
||||||
|| errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) {
|
|
||||||
// do not need to throw the exception but just log the error
|
|
||||||
log.error("Error committing partition {} at offset {}: {}",
|
|
||||||
tp,
|
|
||||||
offset,
|
|
||||||
Errors.forCode(errorCode).exception().getMessage());
|
|
||||||
} else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
|
|
||||||
|| errorCode == Errors.ILLEGAL_GENERATION.code()) {
|
|
||||||
// need to re-join group
|
|
||||||
subscriptions.needReassignment();
|
|
||||||
} else {
|
|
||||||
// re-throw the exception as these should not happen
|
|
||||||
log.error("Error committing partition {} at offset {}: {}",
|
|
||||||
tp,
|
|
||||||
offset,
|
|
||||||
Errors.forCode(errorCode).exception().getMessage());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!future.isDone())
|
public interface RebalanceCallback {
|
||||||
future.complete(null);
|
void onPartitionsAssigned(Collection<TopicPartition> partitions);
|
||||||
}
|
void onPartitionsRevoked(Collection<TopicPartition> partitions);
|
||||||
sensors.commitLatency.record(resp.requestLatencyMs());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private class CoordinatorMetrics {
|
private class CoordinatorMetrics {
|
||||||
|
|
|
@ -0,0 +1,24 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
|
||||||
|
public interface DelayedTask {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Execute the task.
|
||||||
|
* @param now current time in milliseconds
|
||||||
|
*/
|
||||||
|
void run(long now);
|
||||||
|
}
|
|
@ -0,0 +1,96 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.PriorityQueue;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tracks a set of tasks to be executed after a delay.
|
||||||
|
*/
|
||||||
|
public class DelayedTaskQueue {
|
||||||
|
|
||||||
|
private PriorityQueue<Entry> tasks;
|
||||||
|
|
||||||
|
public DelayedTaskQueue() {
|
||||||
|
tasks = new PriorityQueue<Entry>();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Schedule a task for execution in the future.
|
||||||
|
*
|
||||||
|
* @param task the task to execute
|
||||||
|
* @param at the time at which to
|
||||||
|
*/
|
||||||
|
public void add(DelayedTask task, long at) {
|
||||||
|
tasks.add(new Entry(task, at));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove a task from the queue if it is present
|
||||||
|
* @param task the task to be removed
|
||||||
|
* @returns true if a task was removed as a result of this call
|
||||||
|
*/
|
||||||
|
public boolean remove(DelayedTask task) {
|
||||||
|
boolean wasRemoved = false;
|
||||||
|
Iterator<Entry> iterator = tasks.iterator();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
Entry entry = iterator.next();
|
||||||
|
if (entry.task.equals(task)) {
|
||||||
|
iterator.remove();
|
||||||
|
wasRemoved = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return wasRemoved;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get amount of time in milliseconds until the next event. Returns Long.MAX_VALUE if no tasks are scheduled.
|
||||||
|
*
|
||||||
|
* @return the remaining time in milliseconds
|
||||||
|
*/
|
||||||
|
public long nextTimeout(long now) {
|
||||||
|
if (tasks.isEmpty())
|
||||||
|
return Long.MAX_VALUE;
|
||||||
|
else
|
||||||
|
return Math.max(tasks.peek().timeout - now, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run any ready tasks.
|
||||||
|
*
|
||||||
|
* @param now the current time
|
||||||
|
*/
|
||||||
|
public void poll(long now) {
|
||||||
|
while (!tasks.isEmpty() && tasks.peek().timeout <= now) {
|
||||||
|
Entry entry = tasks.poll();
|
||||||
|
entry.task.run(now);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class Entry implements Comparable<Entry> {
|
||||||
|
DelayedTask task;
|
||||||
|
long timeout;
|
||||||
|
|
||||||
|
public Entry(DelayedTask task, long timeout) {
|
||||||
|
this.task = task;
|
||||||
|
this.timeout = timeout;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compareTo(Entry entry) {
|
||||||
|
return Long.compare(timeout, entry.timeout);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -13,17 +13,18 @@
|
||||||
|
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import org.apache.kafka.clients.ClientRequest;
|
|
||||||
import org.apache.kafka.clients.ClientResponse;
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
import org.apache.kafka.clients.KafkaClient;
|
|
||||||
import org.apache.kafka.clients.Metadata;
|
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.ConsumerRecord;
|
||||||
|
import org.apache.kafka.clients.consumer.NoOffsetForPartitionException;
|
||||||
|
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||||
import org.apache.kafka.common.Cluster;
|
import org.apache.kafka.common.Cluster;
|
||||||
import org.apache.kafka.common.MetricName;
|
import org.apache.kafka.common.MetricName;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.PartitionInfo;
|
import org.apache.kafka.common.PartitionInfo;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.DisconnectException;
|
||||||
|
import org.apache.kafka.common.errors.InvalidMetadataException;
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
import org.apache.kafka.common.metrics.Sensor;
|
import org.apache.kafka.common.metrics.Sensor;
|
||||||
import org.apache.kafka.common.metrics.stats.Avg;
|
import org.apache.kafka.common.metrics.stats.Avg;
|
||||||
|
@ -38,7 +39,6 @@ import org.apache.kafka.common.requests.FetchRequest;
|
||||||
import org.apache.kafka.common.requests.FetchResponse;
|
import org.apache.kafka.common.requests.FetchResponse;
|
||||||
import org.apache.kafka.common.requests.ListOffsetRequest;
|
import org.apache.kafka.common.requests.ListOffsetRequest;
|
||||||
import org.apache.kafka.common.requests.ListOffsetResponse;
|
import org.apache.kafka.common.requests.ListOffsetResponse;
|
||||||
import org.apache.kafka.common.requests.RequestSend;
|
|
||||||
import org.apache.kafka.common.serialization.Deserializer;
|
import org.apache.kafka.common.serialization.Deserializer;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
@ -52,21 +52,24 @@ import java.util.HashMap;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class manage the fetching process with the brokers.
|
* This class manage the fetching process with the brokers.
|
||||||
*/
|
*/
|
||||||
public class Fetcher<K, V> {
|
public class Fetcher<K, V> {
|
||||||
|
private static final long EARLIEST_OFFSET_TIMESTAMP = -2L;
|
||||||
|
private static final long LATEST_OFFSET_TIMESTAMP = -1L;
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(Fetcher.class);
|
private static final Logger log = LoggerFactory.getLogger(Fetcher.class);
|
||||||
|
|
||||||
private final KafkaClient client;
|
private final ConsumerNetworkClient client;
|
||||||
|
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final int minBytes;
|
private final int minBytes;
|
||||||
private final int maxWaitMs;
|
private final int maxWaitMs;
|
||||||
private final int fetchSize;
|
private final int fetchSize;
|
||||||
|
private final long retryBackoffMs;
|
||||||
private final boolean checkCrcs;
|
private final boolean checkCrcs;
|
||||||
private final Metadata metadata;
|
private final Metadata metadata;
|
||||||
private final FetchManagerMetrics sensors;
|
private final FetchManagerMetrics sensors;
|
||||||
|
@ -75,8 +78,7 @@ public class Fetcher<K, V> {
|
||||||
private final Deserializer<K> keyDeserializer;
|
private final Deserializer<K> keyDeserializer;
|
||||||
private final Deserializer<V> valueDeserializer;
|
private final Deserializer<V> valueDeserializer;
|
||||||
|
|
||||||
|
public Fetcher(ConsumerNetworkClient client,
|
||||||
public Fetcher(KafkaClient client,
|
|
||||||
int minBytes,
|
int minBytes,
|
||||||
int maxWaitMs,
|
int maxWaitMs,
|
||||||
int fetchSize,
|
int fetchSize,
|
||||||
|
@ -88,7 +90,8 @@ public class Fetcher<K, V> {
|
||||||
Metrics metrics,
|
Metrics metrics,
|
||||||
String metricGrpPrefix,
|
String metricGrpPrefix,
|
||||||
Map<String, String> metricTags,
|
Map<String, String> metricTags,
|
||||||
Time time) {
|
Time time,
|
||||||
|
long retryBackoffMs) {
|
||||||
|
|
||||||
this.time = time;
|
this.time = time;
|
||||||
this.client = client;
|
this.client = client;
|
||||||
|
@ -105,21 +108,101 @@ public class Fetcher<K, V> {
|
||||||
this.records = new LinkedList<PartitionRecords<K, V>>();
|
this.records = new LinkedList<PartitionRecords<K, V>>();
|
||||||
|
|
||||||
this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags);
|
this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags);
|
||||||
|
this.retryBackoffMs = retryBackoffMs;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set-up a fetch request for any node that we have assigned partitions for which doesn't have one.
|
* Set-up a fetch request for any node that we have assigned partitions for which doesn't have one.
|
||||||
*
|
*
|
||||||
* @param cluster The current cluster metadata
|
* @param cluster The current cluster metadata
|
||||||
* @param now The current time
|
|
||||||
*/
|
*/
|
||||||
public void initFetches(Cluster cluster, long now) {
|
public void initFetches(Cluster cluster) {
|
||||||
for (ClientRequest request : createFetchRequests(cluster)) {
|
for (Map.Entry<Node, FetchRequest> fetchEntry: createFetchRequests(cluster).entrySet()) {
|
||||||
Node node = cluster.nodeById(Integer.parseInt(request.request().destination()));
|
final FetchRequest fetch = fetchEntry.getValue();
|
||||||
if (client.ready(node, now)) {
|
client.send(fetchEntry.getKey(), ApiKeys.FETCH, fetch)
|
||||||
log.trace("Initiating fetch to node {}: {}", node.id(), request);
|
.addListener(new RequestFutureListener<ClientResponse>() {
|
||||||
client.send(request);
|
@Override
|
||||||
|
public void onSuccess(ClientResponse response) {
|
||||||
|
handleFetchResponse(response, fetch);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(RuntimeException e) {
|
||||||
|
log.debug("Fetch failed", e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update the fetch positions for the provided partitions.
|
||||||
|
* @param partitions
|
||||||
|
*/
|
||||||
|
public void updateFetchPositions(Set<TopicPartition> partitions) {
|
||||||
|
// reset the fetch position to the committed position
|
||||||
|
for (TopicPartition tp : partitions) {
|
||||||
|
// skip if we already have a fetch position
|
||||||
|
if (subscriptions.fetched(tp) != null)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
// TODO: If there are several offsets to reset, we could submit offset requests in parallel
|
||||||
|
if (subscriptions.isOffsetResetNeeded(tp)) {
|
||||||
|
resetOffset(tp);
|
||||||
|
} else if (subscriptions.committed(tp) == null) {
|
||||||
|
// there's no committed position, so we need to reset with the default strategy
|
||||||
|
subscriptions.needOffsetReset(tp);
|
||||||
|
resetOffset(tp);
|
||||||
|
} else {
|
||||||
|
log.debug("Resetting offset for partition {} to the committed offset {}",
|
||||||
|
tp, subscriptions.committed(tp));
|
||||||
|
subscriptions.seek(tp, subscriptions.committed(tp));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reset offsets for the given partition using the offset reset strategy.
|
||||||
|
*
|
||||||
|
* @param partition The given partition that needs reset offset
|
||||||
|
* @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined
|
||||||
|
*/
|
||||||
|
private void resetOffset(TopicPartition partition) {
|
||||||
|
OffsetResetStrategy strategy = subscriptions.resetStrategy(partition);
|
||||||
|
final long timestamp;
|
||||||
|
if (strategy == OffsetResetStrategy.EARLIEST)
|
||||||
|
timestamp = EARLIEST_OFFSET_TIMESTAMP;
|
||||||
|
else if (strategy == OffsetResetStrategy.LATEST)
|
||||||
|
timestamp = LATEST_OFFSET_TIMESTAMP;
|
||||||
|
else
|
||||||
|
throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined");
|
||||||
|
|
||||||
|
log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase());
|
||||||
|
long offset = listOffset(partition, timestamp);
|
||||||
|
this.subscriptions.seek(partition, offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Fetch a single offset before the given timestamp for the partition.
|
||||||
|
*
|
||||||
|
* @param partition The partition that needs fetching offset.
|
||||||
|
* @param timestamp The timestamp for fetching offset.
|
||||||
|
* @return The offset of the message that is published before the given timestamp
|
||||||
|
*/
|
||||||
|
private long listOffset(TopicPartition partition, long timestamp) {
|
||||||
|
while (true) {
|
||||||
|
RequestFuture<Long> future = sendListOffsetRequest(partition, timestamp);
|
||||||
|
client.poll(future);
|
||||||
|
|
||||||
|
if (future.succeeded())
|
||||||
|
return future.value();
|
||||||
|
|
||||||
|
if (!future.isRetriable())
|
||||||
|
throw future.exception();
|
||||||
|
|
||||||
|
if (future.exception() instanceof InvalidMetadataException)
|
||||||
|
client.awaitMetadataUpdate();
|
||||||
|
else
|
||||||
|
Utils.sleep(retryBackoffMs);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -163,37 +246,27 @@ public class Fetcher<K, V> {
|
||||||
* @param timestamp The timestamp for fetching offset.
|
* @param timestamp The timestamp for fetching offset.
|
||||||
* @return A response which can be polled to obtain the corresponding offset.
|
* @return A response which can be polled to obtain the corresponding offset.
|
||||||
*/
|
*/
|
||||||
public RequestFuture<Long> listOffset(final TopicPartition topicPartition, long timestamp) {
|
private RequestFuture<Long> sendListOffsetRequest(final TopicPartition topicPartition, long timestamp) {
|
||||||
Map<TopicPartition, ListOffsetRequest.PartitionData> partitions = new HashMap<TopicPartition, ListOffsetRequest.PartitionData>(1);
|
Map<TopicPartition, ListOffsetRequest.PartitionData> partitions = new HashMap<TopicPartition, ListOffsetRequest.PartitionData>(1);
|
||||||
partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1));
|
partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1));
|
||||||
long now = time.milliseconds();
|
|
||||||
PartitionInfo info = metadata.fetch().partition(topicPartition);
|
PartitionInfo info = metadata.fetch().partition(topicPartition);
|
||||||
if (info == null) {
|
if (info == null) {
|
||||||
metadata.add(topicPartition.topic());
|
metadata.add(topicPartition.topic());
|
||||||
log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition);
|
log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition);
|
||||||
return RequestFuture.metadataRefreshNeeded();
|
return RequestFuture.staleMetadata();
|
||||||
} else if (info.leader() == null) {
|
} else if (info.leader() == null) {
|
||||||
log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition);
|
log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition);
|
||||||
return RequestFuture.metadataRefreshNeeded();
|
return RequestFuture.leaderNotAvailable();
|
||||||
} else if (this.client.ready(info.leader(), now)) {
|
} else {
|
||||||
final RequestFuture<Long> future = new RequestFuture<Long>();
|
|
||||||
Node node = info.leader();
|
Node node = info.leader();
|
||||||
ListOffsetRequest request = new ListOffsetRequest(-1, partitions);
|
ListOffsetRequest request = new ListOffsetRequest(-1, partitions);
|
||||||
RequestSend send = new RequestSend(node.idString(),
|
return client.send(node, ApiKeys.LIST_OFFSETS, request)
|
||||||
this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS),
|
.compose(new RequestFutureAdapter<ClientResponse, Long>() {
|
||||||
request.toStruct());
|
|
||||||
RequestCompletionHandler completionHandler = new RequestCompletionHandler() {
|
|
||||||
@Override
|
@Override
|
||||||
public void onComplete(ClientResponse resp) {
|
public void onSuccess(ClientResponse response, RequestFuture<Long> future) {
|
||||||
handleListOffsetResponse(topicPartition, resp, future);
|
handleListOffsetResponse(topicPartition, response, future);
|
||||||
}
|
}
|
||||||
};
|
});
|
||||||
ClientRequest clientRequest = new ClientRequest(now, true, send, completionHandler);
|
|
||||||
this.client.send(clientRequest);
|
|
||||||
return future;
|
|
||||||
} else {
|
|
||||||
// We initiated a connect to the leader, but we need to poll to finish it.
|
|
||||||
return RequestFuture.pollNeeded();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -206,7 +279,7 @@ public class Fetcher<K, V> {
|
||||||
ClientResponse clientResponse,
|
ClientResponse clientResponse,
|
||||||
RequestFuture<Long> future) {
|
RequestFuture<Long> future) {
|
||||||
if (clientResponse.wasDisconnected()) {
|
if (clientResponse.wasDisconnected()) {
|
||||||
future.retryAfterMetadataRefresh();
|
future.raise(new DisconnectException());
|
||||||
} else {
|
} else {
|
||||||
ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody());
|
ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody());
|
||||||
short errorCode = lor.responseData().get(topicPartition).errorCode;
|
short errorCode = lor.responseData().get(topicPartition).errorCode;
|
||||||
|
@ -222,11 +295,11 @@ public class Fetcher<K, V> {
|
||||||
|| errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
|
|| errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
|
||||||
log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.",
|
log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.",
|
||||||
topicPartition);
|
topicPartition);
|
||||||
future.retryAfterMetadataRefresh();
|
future.raise(Errors.forCode(errorCode));
|
||||||
} else {
|
} else {
|
||||||
log.error("Attempt to fetch offsets for partition {} failed due to: {}",
|
log.error("Attempt to fetch offsets for partition {} failed due to: {}",
|
||||||
topicPartition, Errors.forCode(errorCode).exception().getMessage());
|
topicPartition, Errors.forCode(errorCode).exception().getMessage());
|
||||||
future.retryAfterMetadataRefresh();
|
future.raise(new StaleMetadataException());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -235,37 +308,31 @@ public class Fetcher<K, V> {
|
||||||
* Create fetch requests for all nodes for which we have assigned partitions
|
* Create fetch requests for all nodes for which we have assigned partitions
|
||||||
* that have no existing requests in flight.
|
* that have no existing requests in flight.
|
||||||
*/
|
*/
|
||||||
private List<ClientRequest> createFetchRequests(Cluster cluster) {
|
private Map<Node, FetchRequest> createFetchRequests(Cluster cluster) {
|
||||||
// create the fetch info
|
// create the fetch info
|
||||||
Map<Integer, Map<TopicPartition, FetchRequest.PartitionData>> fetchable = new HashMap<Integer, Map<TopicPartition, FetchRequest.PartitionData>>();
|
Map<Node, Map<TopicPartition, FetchRequest.PartitionData>> fetchable = new HashMap<Node, Map<TopicPartition, FetchRequest.PartitionData>>();
|
||||||
for (TopicPartition partition : subscriptions.assignedPartitions()) {
|
for (TopicPartition partition : subscriptions.assignedPartitions()) {
|
||||||
Node node = cluster.leaderFor(partition);
|
Node node = cluster.leaderFor(partition);
|
||||||
if (node == null) {
|
if (node == null) {
|
||||||
metadata.requestUpdate();
|
metadata.requestUpdate();
|
||||||
} else if (this.client.inFlightRequestCount(node.idString()) == 0) {
|
} else if (this.client.pendingRequestCount(node) == 0) {
|
||||||
// if there is a leader and no in-flight requests, issue a new fetch
|
// if there is a leader and no in-flight requests, issue a new fetch
|
||||||
Map<TopicPartition, FetchRequest.PartitionData> fetch = fetchable.get(node.id());
|
Map<TopicPartition, FetchRequest.PartitionData> fetch = fetchable.get(node);
|
||||||
if (fetch == null) {
|
if (fetch == null) {
|
||||||
fetch = new HashMap<TopicPartition, FetchRequest.PartitionData>();
|
fetch = new HashMap<TopicPartition, FetchRequest.PartitionData>();
|
||||||
fetchable.put(node.id(), fetch);
|
fetchable.put(node, fetch);
|
||||||
}
|
}
|
||||||
long offset = this.subscriptions.fetched(partition);
|
long offset = this.subscriptions.fetched(partition);
|
||||||
fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize));
|
fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// create the requests
|
// create the fetches
|
||||||
List<ClientRequest> requests = new ArrayList<ClientRequest>(fetchable.size());
|
Map<Node, FetchRequest> requests = new HashMap<Node, FetchRequest>();
|
||||||
for (Map.Entry<Integer, Map<TopicPartition, FetchRequest.PartitionData>> entry : fetchable.entrySet()) {
|
for (Map.Entry<Node, Map<TopicPartition, FetchRequest.PartitionData>> entry : fetchable.entrySet()) {
|
||||||
int nodeId = entry.getKey();
|
Node node = entry.getKey();
|
||||||
final FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue());
|
FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue());
|
||||||
RequestSend send = new RequestSend(Integer.toString(nodeId), this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct());
|
requests.put(node, fetch);
|
||||||
RequestCompletionHandler handler = new RequestCompletionHandler() {
|
|
||||||
public void onComplete(ClientResponse response) {
|
|
||||||
handleFetchResponse(response, fetch);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
requests.add(new ClientRequest(time.milliseconds(), true, send, handler));
|
|
||||||
}
|
}
|
||||||
return requests;
|
return requests;
|
||||||
}
|
}
|
||||||
|
@ -353,7 +420,6 @@ public class Fetcher<K, V> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private class FetchManagerMetrics {
|
private class FetchManagerMetrics {
|
||||||
public final Metrics metrics;
|
public final Metrics metrics;
|
||||||
public final String metricGrpName;
|
public final String metricGrpName;
|
||||||
|
|
|
@ -13,7 +13,7 @@
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A helper class for managing the heartbeat to the co-ordinator
|
* A helper class for managing the heartbeat to the coordinator
|
||||||
*/
|
*/
|
||||||
public final class Heartbeat {
|
public final class Heartbeat {
|
||||||
|
|
||||||
|
@ -25,18 +25,24 @@ public final class Heartbeat {
|
||||||
|
|
||||||
private final long timeout;
|
private final long timeout;
|
||||||
private long lastHeartbeatSend;
|
private long lastHeartbeatSend;
|
||||||
|
private long lastHeartbeatReceive;
|
||||||
|
private long lastSessionReset;
|
||||||
|
|
||||||
public Heartbeat(long timeout, long now) {
|
public Heartbeat(long timeout, long now) {
|
||||||
this.timeout = timeout;
|
this.timeout = timeout;
|
||||||
this.lastHeartbeatSend = now;
|
this.lastSessionReset = now;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void sentHeartbeat(long now) {
|
public void sentHeartbeat(long now) {
|
||||||
this.lastHeartbeatSend = now;
|
this.lastHeartbeatSend = now;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void receiveHeartbeat(long now) {
|
||||||
|
this.lastHeartbeatReceive = now;
|
||||||
|
}
|
||||||
|
|
||||||
public boolean shouldHeartbeat(long now) {
|
public boolean shouldHeartbeat(long now) {
|
||||||
return now - lastHeartbeatSend > (1.0 / HEARTBEATS_PER_SESSION_INTERVAL) * this.timeout;
|
return timeToNextHeartbeat(now) == 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long lastHeartbeatSend() {
|
public long lastHeartbeatSend() {
|
||||||
|
@ -44,7 +50,7 @@ public final class Heartbeat {
|
||||||
}
|
}
|
||||||
|
|
||||||
public long timeToNextHeartbeat(long now) {
|
public long timeToNextHeartbeat(long now) {
|
||||||
long timeSinceLastHeartbeat = now - lastHeartbeatSend;
|
long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset);
|
||||||
|
|
||||||
long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL;
|
long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL;
|
||||||
if (timeSinceLastHeartbeat > hbInterval)
|
if (timeSinceLastHeartbeat > hbInterval)
|
||||||
|
@ -52,4 +58,17 @@ public final class Heartbeat {
|
||||||
else
|
else
|
||||||
return hbInterval - timeSinceLastHeartbeat;
|
return hbInterval - timeSinceLastHeartbeat;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean sessionTimeoutExpired(long now) {
|
||||||
|
return now - Math.max(lastSessionReset, lastHeartbeatReceive) > timeout;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long interval() {
|
||||||
|
return timeout / HEARTBEATS_PER_SESSION_INTERVAL;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void resetSessionTimeout(long now) {
|
||||||
|
this.lastSessionReset = now;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -0,0 +1,23 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.errors.InvalidMetadataException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* No brokers were available to complete a request.
|
||||||
|
*/
|
||||||
|
public class NoAvailableBrokersException extends InvalidMetadataException {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
}
|
|
@ -12,78 +12,49 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer.internals;
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.errors.RetriableException;
|
||||||
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Result of an asynchronous request through {@link org.apache.kafka.clients.KafkaClient}. To get the
|
* Result of an asynchronous request from {@link ConsumerNetworkClient}. Use {@link ConsumerNetworkClient#poll(long)}
|
||||||
* result of the request, you must use poll using {@link org.apache.kafka.clients.KafkaClient#poll(long, long)}
|
* (and variants) to finish a request future. Use {@link #isDone()} to check if the future is complete, and
|
||||||
* until {@link #isDone()} returns true. Typical usage might look like this:
|
* {@link #succeeded()} to check if the request completed successfully. Typical usage might look like this:
|
||||||
*
|
*
|
||||||
* <pre>
|
* <pre>
|
||||||
* RequestFuture future = sendRequest();
|
* RequestFuture<ClientResponse> future = client.send(api, request);
|
||||||
* while (!future.isDone()) {
|
* client.poll(future);
|
||||||
* client.poll(timeout, now);
|
|
||||||
* }
|
|
||||||
*
|
*
|
||||||
* switch (future.outcome()) {
|
* if (future.succeeded()) {
|
||||||
* case SUCCESS:
|
* ClientResponse response = future.value();
|
||||||
* // handle request success
|
* // Handle response
|
||||||
* break;
|
* } else {
|
||||||
* case NEED_RETRY:
|
* throw future.exception();
|
||||||
* // retry after taking possible retry action
|
|
||||||
* break;
|
|
||||||
* case EXCEPTION:
|
|
||||||
* // handle exception
|
|
||||||
* }
|
* }
|
||||||
* </pre>
|
* </pre>
|
||||||
*
|
*
|
||||||
* When {@link #isDone()} returns true, there are three possible outcomes (obtained through {@link #outcome()}):
|
|
||||||
*
|
|
||||||
* <ol>
|
|
||||||
* <li> {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#SUCCESS}: If the request was
|
|
||||||
* successful, then you can use {@link #value()} to obtain the result.</li>
|
|
||||||
* <li> {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#EXCEPTION}: If an unhandled exception
|
|
||||||
* was encountered, you can use {@link #exception()} to get it.</li>
|
|
||||||
* <li> {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#NEED_RETRY}: The request may
|
|
||||||
* not have been successful, but the failure may be ephemeral and the caller just needs to try the request again.
|
|
||||||
* In this case, use {@link #retryAction()} to determine what action should be taken (if any) before
|
|
||||||
* retrying.</li>
|
|
||||||
* </ol>
|
|
||||||
*
|
|
||||||
* @param <T> Return type of the result (Can be Void if there is no response)
|
* @param <T> Return type of the result (Can be Void if there is no response)
|
||||||
*/
|
*/
|
||||||
public class RequestFuture<T> {
|
public class RequestFuture<T> {
|
||||||
public static final RequestFuture<Object> NEED_NEW_COORDINATOR = newRetryFuture(RetryAction.FIND_COORDINATOR);
|
|
||||||
public static final RequestFuture<Object> NEED_POLL = newRetryFuture(RetryAction.POLL);
|
|
||||||
public static final RequestFuture<Object> NEED_METADATA_REFRESH = newRetryFuture(RetryAction.REFRESH_METADATA);
|
|
||||||
|
|
||||||
public enum RetryAction {
|
private boolean isDone = false;
|
||||||
NOOP, // Retry immediately.
|
|
||||||
POLL, // Retry after calling poll (e.g. to finish a connection)
|
|
||||||
BACKOFF, // Retry after a delay
|
|
||||||
FIND_COORDINATOR, // Find a new coordinator before retrying
|
|
||||||
REFRESH_METADATA // Refresh metadata before retrying
|
|
||||||
}
|
|
||||||
|
|
||||||
public enum Outcome {
|
|
||||||
SUCCESS,
|
|
||||||
NEED_RETRY,
|
|
||||||
EXCEPTION
|
|
||||||
}
|
|
||||||
|
|
||||||
private Outcome outcome;
|
|
||||||
private RetryAction retryAction;
|
|
||||||
private T value;
|
private T value;
|
||||||
private RuntimeException exception;
|
private RuntimeException exception;
|
||||||
|
private List<RequestFutureListener<T>> listeners = new ArrayList<RequestFutureListener<T>>();
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check whether the response is ready to be handled
|
* Check whether the response is ready to be handled
|
||||||
* @return true if the response is ready, false otherwise
|
* @return true if the response is ready, false otherwise
|
||||||
*/
|
*/
|
||||||
public boolean isDone() {
|
public boolean isDone() {
|
||||||
return outcome != null;
|
return isDone;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the value corresponding to this request (if it has one, as indicated by {@link #outcome()}).
|
* Get the value corresponding to this request (only available if the request succeeded)
|
||||||
* @return the value if it exists or null
|
* @return the value if it exists or null
|
||||||
*/
|
*/
|
||||||
public T value() {
|
public T value() {
|
||||||
|
@ -92,118 +63,140 @@ public class RequestFuture<T> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if the request succeeded;
|
* Check if the request succeeded;
|
||||||
* @return true if a value is available, false otherwise
|
* @return true if the request completed and was successful
|
||||||
*/
|
*/
|
||||||
public boolean succeeded() {
|
public boolean succeeded() {
|
||||||
return outcome == Outcome.SUCCESS;
|
return isDone && exception == null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if the request completed failed.
|
* Check if the request failed.
|
||||||
* @return true if the request failed (whether or not it can be retried)
|
* @return true if the request completed with a failure
|
||||||
*/
|
*/
|
||||||
public boolean failed() {
|
public boolean failed() {
|
||||||
return outcome != Outcome.SUCCESS;
|
return isDone && exception != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the error from this response (assuming {@link #succeeded()} has returned false. If the
|
* Check if the request is retriable (convenience method for checking if
|
||||||
* response is not ready or if there is no retryAction, null is returned.
|
* the exception is an instance of {@link RetriableException}.
|
||||||
* @return the error if it exists or null
|
* @return true if it is retriable, false otherwise
|
||||||
*/
|
*/
|
||||||
public RetryAction retryAction() {
|
public boolean isRetriable() {
|
||||||
return retryAction;
|
return exception instanceof RetriableException;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the exception from a failed result. You should check that there is an exception
|
* Get the exception from a failed result (only available if the request failed)
|
||||||
* with {@link #hasException()} before using this method.
|
|
||||||
* @return The exception if it exists or null
|
* @return The exception if it exists or null
|
||||||
*/
|
*/
|
||||||
public RuntimeException exception() {
|
public RuntimeException exception() {
|
||||||
return exception;
|
return exception;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Check whether there was an exception.
|
|
||||||
* @return true if this request failed with an exception
|
|
||||||
*/
|
|
||||||
public boolean hasException() {
|
|
||||||
return outcome == Outcome.EXCEPTION;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Check the outcome of the future if it is ready.
|
|
||||||
* @return the outcome or null if the future is not finished
|
|
||||||
*/
|
|
||||||
public Outcome outcome() {
|
|
||||||
return outcome;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The request failed, but should be retried using the provided retry action.
|
|
||||||
* @param retryAction The action that should be taken by the caller before retrying the request
|
|
||||||
*/
|
|
||||||
public void retry(RetryAction retryAction) {
|
|
||||||
this.outcome = Outcome.NEED_RETRY;
|
|
||||||
this.retryAction = retryAction;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void retryNow() {
|
|
||||||
retry(RetryAction.NOOP);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void retryAfterBackoff() {
|
|
||||||
retry(RetryAction.BACKOFF);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void retryWithNewCoordinator() {
|
|
||||||
retry(RetryAction.FIND_COORDINATOR);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void retryAfterMetadataRefresh() {
|
|
||||||
retry(RetryAction.REFRESH_METADATA);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Complete the request successfully. After this call, {@link #succeeded()} will return true
|
* Complete the request successfully. After this call, {@link #succeeded()} will return true
|
||||||
* and the value can be obtained through {@link #value()}.
|
* and the value can be obtained through {@link #value()}.
|
||||||
* @param value corresponding value (or null if there is none)
|
* @param value corresponding value (or null if there is none)
|
||||||
*/
|
*/
|
||||||
public void complete(T value) {
|
public void complete(T value) {
|
||||||
this.outcome = Outcome.SUCCESS;
|
|
||||||
this.value = value;
|
this.value = value;
|
||||||
|
this.isDone = true;
|
||||||
|
fireSuccess();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Raise an exception. The request will be marked as failed, and the caller can either
|
* Raise an exception. The request will be marked as failed, and the caller can either
|
||||||
* handle the exception or throw it.
|
* handle the exception or throw it.
|
||||||
* @param e The exception that
|
* @param e corresponding exception to be passed to caller
|
||||||
*/
|
*/
|
||||||
public void raise(RuntimeException e) {
|
public void raise(RuntimeException e) {
|
||||||
this.outcome = Outcome.EXCEPTION;
|
|
||||||
this.exception = e;
|
this.exception = e;
|
||||||
|
this.isDone = true;
|
||||||
|
fireFailure();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <T> RequestFuture<T> newRetryFuture(RetryAction retryAction) {
|
/**
|
||||||
RequestFuture<T> result = new RequestFuture<T>();
|
* Raise an error. The request will be marked as failed.
|
||||||
result.retry(retryAction);
|
* @param error corresponding error to be passed to caller
|
||||||
return result;
|
*/
|
||||||
|
public void raise(Errors error) {
|
||||||
|
raise(error.exception());
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
private void fireSuccess() {
|
||||||
public static <T> RequestFuture<T> pollNeeded() {
|
for (RequestFutureListener listener: listeners)
|
||||||
return (RequestFuture<T>) NEED_POLL;
|
listener.onSuccess(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
private void fireFailure() {
|
||||||
public static <T> RequestFuture<T> metadataRefreshNeeded() {
|
for (RequestFutureListener listener: listeners)
|
||||||
return (RequestFuture<T>) NEED_METADATA_REFRESH;
|
listener.onFailure(exception);
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
/**
|
||||||
public static <T> RequestFuture<T> newCoordinatorNeeded() {
|
* Add a listener which will be notified when the future completes
|
||||||
return (RequestFuture<T>) NEED_NEW_COORDINATOR;
|
* @param listener
|
||||||
|
*/
|
||||||
|
public void addListener(RequestFutureListener<T> listener) {
|
||||||
|
if (isDone) {
|
||||||
|
if (exception != null)
|
||||||
|
listener.onFailure(exception);
|
||||||
|
else
|
||||||
|
listener.onSuccess(value);
|
||||||
|
} else {
|
||||||
|
this.listeners.add(listener);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert from a request future of one type to another type
|
||||||
|
* @param adapter The adapter which does the conversion
|
||||||
|
* @param <S> The type of the future adapted to
|
||||||
|
* @return The new future
|
||||||
|
*/
|
||||||
|
public <S> RequestFuture<S> compose(final RequestFutureAdapter<T, S> adapter) {
|
||||||
|
final RequestFuture<S> adapted = new RequestFuture<S>();
|
||||||
|
addListener(new RequestFutureListener<T>() {
|
||||||
|
@Override
|
||||||
|
public void onSuccess(T value) {
|
||||||
|
adapter.onSuccess(value, adapted);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(RuntimeException e) {
|
||||||
|
adapter.onFailure(e, adapted);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
return adapted;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> RequestFuture<T> failure(RuntimeException e) {
|
||||||
|
RequestFuture<T> future = new RequestFuture<T>();
|
||||||
|
future.raise(e);
|
||||||
|
return future;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static RequestFuture<Void> voidSuccess() {
|
||||||
|
RequestFuture<Void> future = new RequestFuture<Void>();
|
||||||
|
future.complete(null);
|
||||||
|
return future;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> RequestFuture<T> coordinatorNotAvailable() {
|
||||||
|
return failure(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> RequestFuture<T> leaderNotAvailable() {
|
||||||
|
return failure(Errors.LEADER_NOT_AVAILABLE.exception());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> RequestFuture<T> noBrokersAvailable() {
|
||||||
|
return failure(new NoAvailableBrokersException());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> RequestFuture<T> staleMetadata() {
|
||||||
|
return failure(new StaleMetadataException());
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,28 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adapt from a request future of one type to another.
|
||||||
|
*
|
||||||
|
* @param <F> Type to adapt from
|
||||||
|
* @param <T> Type to adapt to
|
||||||
|
*/
|
||||||
|
public abstract class RequestFutureAdapter<F, T> {
|
||||||
|
|
||||||
|
public abstract void onSuccess(F value, RequestFuture<T> future);
|
||||||
|
|
||||||
|
public void onFailure(RuntimeException e, RequestFuture<T> future) {
|
||||||
|
future.raise(e);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,23 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Listener interface to hook into RequestFuture completion.
|
||||||
|
*/
|
||||||
|
public interface RequestFutureListener<T> {
|
||||||
|
|
||||||
|
void onSuccess(T value);
|
||||||
|
|
||||||
|
void onFailure(RuntimeException e);
|
||||||
|
}
|
|
@ -0,0 +1,27 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.errors.RetriableException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Exception used in {@link ConsumerNetworkClient} to indicate the failure
|
||||||
|
* to transmit a request to the networking layer. This could be either because
|
||||||
|
* the client is still connecting to the given host or its send buffer is full.
|
||||||
|
*/
|
||||||
|
public class SendFailedException extends RetriableException {
|
||||||
|
public static final SendFailedException INSTANCE = new SendFailedException();
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,22 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.errors.InvalidMetadataException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Thrown when metadata is old and needs to be refreshed.
|
||||||
|
*/
|
||||||
|
public class StaleMetadataException extends InvalidMetadataException {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
}
|
|
@ -138,7 +138,6 @@ public class SubscriptionState {
|
||||||
|
|
||||||
public void committed(TopicPartition tp, long offset) {
|
public void committed(TopicPartition tp, long offset) {
|
||||||
this.committed.put(tp, offset);
|
this.committed.put(tp, offset);
|
||||||
this.needsFetchCommittedOffsets = false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public Long committed(TopicPartition tp) {
|
public Long committed(TopicPartition tp) {
|
||||||
|
@ -153,6 +152,10 @@ public class SubscriptionState {
|
||||||
return this.needsFetchCommittedOffsets;
|
return this.needsFetchCommittedOffsets;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void commitsRefreshed() {
|
||||||
|
this.needsFetchCommittedOffsets = false;
|
||||||
|
}
|
||||||
|
|
||||||
public void seek(TopicPartition tp, long offset) {
|
public void seek(TopicPartition tp, long offset) {
|
||||||
fetched(tp, offset);
|
fetched(tp, offset);
|
||||||
consumed(tp, offset);
|
consumed(tp, offset);
|
||||||
|
|
|
@ -0,0 +1,40 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.kafka.common.errors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has
|
||||||
|
* not yet been created.
|
||||||
|
*/
|
||||||
|
public class ConsumerCoordinatorNotAvailableException extends RetriableException {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
public ConsumerCoordinatorNotAvailableException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public ConsumerCoordinatorNotAvailableException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ConsumerCoordinatorNotAvailableException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ConsumerCoordinatorNotAvailableException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,39 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.common.errors;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Server disconnected before a request could be completed.
|
||||||
|
*/
|
||||||
|
public class DisconnectException extends RetriableException {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
public DisconnectException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public DisconnectException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DisconnectException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DisconnectException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,33 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.common.errors;
|
||||||
|
|
||||||
|
public class IllegalGenerationException extends RetriableException {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
public IllegalGenerationException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public IllegalGenerationException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public IllegalGenerationException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public IllegalGenerationException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,40 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.kafka.common.errors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is
|
||||||
|
* not a coordinator for.
|
||||||
|
*/
|
||||||
|
public class NotCoordinatorForConsumerException extends RetriableException {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
public NotCoordinatorForConsumerException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public NotCoordinatorForConsumerException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public NotCoordinatorForConsumerException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public NotCoordinatorForConsumerException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,40 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.kafka.common.errors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change
|
||||||
|
* for that offsets topic partition).
|
||||||
|
*/
|
||||||
|
public class OffsetLoadInProgressException extends RetriableException {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
public OffsetLoadInProgressException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public OffsetLoadInProgressException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public OffsetLoadInProgressException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public OffsetLoadInProgressException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,33 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.common.errors;
|
||||||
|
|
||||||
|
public class UnknownConsumerIdException extends RetriableException {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
public UnknownConsumerIdException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public UnknownConsumerIdException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public UnknownConsumerIdException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public UnknownConsumerIdException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
}
|
|
@ -53,11 +53,11 @@ public enum Errors {
|
||||||
NETWORK_EXCEPTION(13,
|
NETWORK_EXCEPTION(13,
|
||||||
new NetworkException("The server disconnected before a response was received.")),
|
new NetworkException("The server disconnected before a response was received.")),
|
||||||
OFFSET_LOAD_IN_PROGRESS(14,
|
OFFSET_LOAD_IN_PROGRESS(14,
|
||||||
new ApiException("The coordinator is loading offsets and can't process requests.")),
|
new OffsetLoadInProgressException("The coordinator is loading offsets and can't process requests.")),
|
||||||
CONSUMER_COORDINATOR_NOT_AVAILABLE(15,
|
CONSUMER_COORDINATOR_NOT_AVAILABLE(15,
|
||||||
new ApiException("The coordinator is not available.")),
|
new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")),
|
||||||
NOT_COORDINATOR_FOR_CONSUMER(16,
|
NOT_COORDINATOR_FOR_CONSUMER(16,
|
||||||
new ApiException("This is not the correct co-ordinator for this consumer.")),
|
new NotCoordinatorForConsumerException("This is not the correct coordinator for this consumer.")),
|
||||||
INVALID_TOPIC_EXCEPTION(17,
|
INVALID_TOPIC_EXCEPTION(17,
|
||||||
new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")),
|
new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")),
|
||||||
RECORD_LIST_TOO_LARGE(18,
|
RECORD_LIST_TOO_LARGE(18,
|
||||||
|
@ -69,13 +69,13 @@ public enum Errors {
|
||||||
INVALID_REQUIRED_ACKS(21,
|
INVALID_REQUIRED_ACKS(21,
|
||||||
new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")),
|
new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")),
|
||||||
ILLEGAL_GENERATION(22,
|
ILLEGAL_GENERATION(22,
|
||||||
new ApiException("Specified consumer generation id is not valid.")),
|
new IllegalGenerationException("Specified consumer generation id is not valid.")),
|
||||||
INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23,
|
INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23,
|
||||||
new ApiException("The request partition assignment strategy does not match that of the group.")),
|
new ApiException("The request partition assignment strategy does not match that of the group.")),
|
||||||
UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24,
|
UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24,
|
||||||
new ApiException("The request partition assignment strategy is unknown to the broker.")),
|
new ApiException("The request partition assignment strategy is unknown to the broker.")),
|
||||||
UNKNOWN_CONSUMER_ID(25,
|
UNKNOWN_CONSUMER_ID(25,
|
||||||
new ApiException("The coordinator is not aware of this consumer.")),
|
new UnknownConsumerIdException("The coordinator is not aware of this consumer.")),
|
||||||
INVALID_SESSION_TIMEOUT(26,
|
INVALID_SESSION_TIMEOUT(26,
|
||||||
new ApiException("The session timeout is not within an acceptable range.")),
|
new ApiException("The session timeout is not within an acceptable range.")),
|
||||||
COMMITTING_PARTITIONS_NOT_ASSIGNED(27,
|
COMMITTING_PARTITIONS_NOT_ASSIGNED(27,
|
||||||
|
|
|
@ -0,0 +1,125 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.ClientResponse;
|
||||||
|
import org.apache.kafka.clients.Metadata;
|
||||||
|
import org.apache.kafka.clients.MockClient;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerWakeupException;
|
||||||
|
import org.apache.kafka.common.Cluster;
|
||||||
|
import org.apache.kafka.common.Node;
|
||||||
|
import org.apache.kafka.common.protocol.ApiKeys;
|
||||||
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
|
import org.apache.kafka.common.protocol.types.Struct;
|
||||||
|
import org.apache.kafka.common.requests.HeartbeatRequest;
|
||||||
|
import org.apache.kafka.common.requests.HeartbeatResponse;
|
||||||
|
import org.apache.kafka.common.utils.MockTime;
|
||||||
|
import org.apache.kafka.test.TestUtils;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
public class ConsumerNetworkClientTest {
|
||||||
|
|
||||||
|
private String topicName = "test";
|
||||||
|
private MockTime time = new MockTime();
|
||||||
|
private MockClient client = new MockClient(time);
|
||||||
|
private Cluster cluster = TestUtils.singletonCluster(topicName, 1);
|
||||||
|
private Node node = cluster.nodes().get(0);
|
||||||
|
private Metadata metadata = new Metadata(0, Long.MAX_VALUE);
|
||||||
|
private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100);
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void send() {
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.NONE.code()));
|
||||||
|
RequestFuture<ClientResponse> future = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest());
|
||||||
|
assertEquals(1, consumerClient.pendingRequestCount());
|
||||||
|
assertEquals(1, consumerClient.pendingRequestCount(node));
|
||||||
|
assertFalse(future.isDone());
|
||||||
|
|
||||||
|
consumerClient.poll(future);
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertTrue(future.succeeded());
|
||||||
|
|
||||||
|
ClientResponse clientResponse = future.value();
|
||||||
|
HeartbeatResponse response = new HeartbeatResponse(clientResponse.responseBody());
|
||||||
|
assertEquals(Errors.NONE.code(), response.errorCode());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void multiSend() {
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.NONE.code()));
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.NONE.code()));
|
||||||
|
RequestFuture<ClientResponse> future1 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest());
|
||||||
|
RequestFuture<ClientResponse> future2 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest());
|
||||||
|
assertEquals(2, consumerClient.pendingRequestCount());
|
||||||
|
assertEquals(2, consumerClient.pendingRequestCount(node));
|
||||||
|
|
||||||
|
consumerClient.awaitPendingRequests(node);
|
||||||
|
assertTrue(future1.succeeded());
|
||||||
|
assertTrue(future2.succeeded());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void schedule() {
|
||||||
|
TestDelayedTask task = new TestDelayedTask();
|
||||||
|
consumerClient.schedule(task, time.milliseconds());
|
||||||
|
consumerClient.poll(0);
|
||||||
|
assertEquals(1, task.executions);
|
||||||
|
|
||||||
|
consumerClient.schedule(task, time.milliseconds() + 100);
|
||||||
|
consumerClient.poll(0);
|
||||||
|
assertEquals(1, task.executions);
|
||||||
|
|
||||||
|
time.sleep(100);
|
||||||
|
consumerClient.poll(0);
|
||||||
|
assertEquals(2, task.executions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void wakeup() {
|
||||||
|
RequestFuture<ClientResponse> future = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest());
|
||||||
|
consumerClient.wakeup();
|
||||||
|
try {
|
||||||
|
consumerClient.poll(0);
|
||||||
|
fail();
|
||||||
|
} catch (ConsumerWakeupException e) {
|
||||||
|
}
|
||||||
|
|
||||||
|
client.respond(heartbeatResponse(Errors.NONE.code()));
|
||||||
|
consumerClient.poll(future);
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private HeartbeatRequest heartbeatRequest() {
|
||||||
|
return new HeartbeatRequest("group", 1, "consumerId");
|
||||||
|
}
|
||||||
|
|
||||||
|
private Struct heartbeatResponse(short error) {
|
||||||
|
HeartbeatResponse response = new HeartbeatResponse(error);
|
||||||
|
return response.toStruct();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class TestDelayedTask implements DelayedTask {
|
||||||
|
int executions = 0;
|
||||||
|
@Override
|
||||||
|
public void run(long now) {
|
||||||
|
executions++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -18,13 +18,19 @@ package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.Metadata;
|
||||||
import org.apache.kafka.clients.MockClient;
|
import org.apache.kafka.clients.MockClient;
|
||||||
|
import org.apache.kafka.clients.consumer.CommitType;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerCommitCallback;
|
||||||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||||
import org.apache.kafka.common.Cluster;
|
import org.apache.kafka.common.Cluster;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.ApiException;
|
||||||
|
import org.apache.kafka.common.errors.DisconnectException;
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
import org.apache.kafka.common.protocol.types.Struct;
|
import org.apache.kafka.common.protocol.types.Struct;
|
||||||
|
@ -36,10 +42,12 @@ import org.apache.kafka.common.requests.OffsetFetchResponse;
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
import org.apache.kafka.common.utils.MockTime;
|
||||||
import org.apache.kafka.test.TestUtils;
|
import org.apache.kafka.test.TestUtils;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -51,16 +59,34 @@ public class CoordinatorTest {
|
||||||
private String groupId = "test-group";
|
private String groupId = "test-group";
|
||||||
private TopicPartition tp = new TopicPartition(topicName, 0);
|
private TopicPartition tp = new TopicPartition(topicName, 0);
|
||||||
private int sessionTimeoutMs = 10;
|
private int sessionTimeoutMs = 10;
|
||||||
|
private long retryBackoffMs = 100;
|
||||||
|
private long requestTimeoutMs = 5000;
|
||||||
private String rebalanceStrategy = "not-matter";
|
private String rebalanceStrategy = "not-matter";
|
||||||
private MockTime time = new MockTime();
|
private MockTime time;
|
||||||
private MockClient client = new MockClient(time);
|
private MockClient client;
|
||||||
private Cluster cluster = TestUtils.singletonCluster(topicName, 1);
|
private Cluster cluster = TestUtils.singletonCluster(topicName, 1);
|
||||||
private Node node = cluster.nodes().get(0);
|
private Node node = cluster.nodes().get(0);
|
||||||
private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST);
|
private SubscriptionState subscriptions;
|
||||||
private Metrics metrics = new Metrics(time);
|
private Metadata metadata;
|
||||||
|
private Metrics metrics;
|
||||||
private Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
private Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||||
|
private ConsumerNetworkClient consumerClient;
|
||||||
|
private MockRebalanceCallback rebalanceCallback;
|
||||||
|
private Coordinator coordinator;
|
||||||
|
|
||||||
private Coordinator coordinator = new Coordinator(client,
|
@Before
|
||||||
|
public void setup() {
|
||||||
|
this.time = new MockTime();
|
||||||
|
this.client = new MockClient(time);
|
||||||
|
this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST);
|
||||||
|
this.metadata = new Metadata(0, Long.MAX_VALUE);
|
||||||
|
this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100);
|
||||||
|
this.metrics = new Metrics(time);
|
||||||
|
this.rebalanceCallback = new MockRebalanceCallback();
|
||||||
|
|
||||||
|
client.setNode(node);
|
||||||
|
|
||||||
|
this.coordinator = new Coordinator(consumerClient,
|
||||||
groupId,
|
groupId,
|
||||||
sessionTimeoutMs,
|
sessionTimeoutMs,
|
||||||
rebalanceStrategy,
|
rebalanceStrategy,
|
||||||
|
@ -68,91 +94,138 @@ public class CoordinatorTest {
|
||||||
metrics,
|
metrics,
|
||||||
"consumer" + groupId,
|
"consumer" + groupId,
|
||||||
metricTags,
|
metricTags,
|
||||||
time);
|
time,
|
||||||
|
requestTimeoutMs,
|
||||||
@Before
|
retryBackoffMs,
|
||||||
public void setup() {
|
rebalanceCallback);
|
||||||
client.setNode(node);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNormalHeartbeat() {
|
public void testNormalHeartbeat() {
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// normal heartbeat
|
// normal heartbeat
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat
|
RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
|
||||||
assertEquals(1, client.inFlightRequestCount());
|
assertEquals(1, consumerClient.pendingRequestCount());
|
||||||
client.respond(heartbeatResponse(Errors.NONE.code()));
|
assertFalse(future.isDone());
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.NONE.code()));
|
||||||
|
consumerClient.poll(0);
|
||||||
|
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertTrue(future.succeeded());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCoordinatorNotAvailable() {
|
public void testCoordinatorNotAvailable() {
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// consumer_coordinator_not_available will mark coordinator as unknown
|
// consumer_coordinator_not_available will mark coordinator as unknown
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat
|
RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
|
||||||
assertEquals(1, client.inFlightRequestCount());
|
assertEquals(1, consumerClient.pendingRequestCount());
|
||||||
client.respond(heartbeatResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()));
|
assertFalse(future.isDone());
|
||||||
|
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()));
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
consumerClient.poll(0);
|
||||||
|
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertTrue(future.failed());
|
||||||
|
assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), future.exception());
|
||||||
assertTrue(coordinator.coordinatorUnknown());
|
assertTrue(coordinator.coordinatorUnknown());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNotCoordinator() {
|
public void testNotCoordinator() {
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// not_coordinator will mark coordinator as unknown
|
// not_coordinator will mark coordinator as unknown
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat
|
RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
|
||||||
assertEquals(1, client.inFlightRequestCount());
|
assertEquals(1, consumerClient.pendingRequestCount());
|
||||||
client.respond(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_CONSUMER.code()));
|
assertFalse(future.isDone());
|
||||||
|
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_CONSUMER.code()));
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
consumerClient.poll(0);
|
||||||
|
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertTrue(future.failed());
|
||||||
|
assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), future.exception());
|
||||||
assertTrue(coordinator.coordinatorUnknown());
|
assertTrue(coordinator.coordinatorUnknown());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIllegalGeneration() {
|
public void testIllegalGeneration() {
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// illegal_generation will cause re-partition
|
// illegal_generation will cause re-partition
|
||||||
subscriptions.subscribe(topicName);
|
subscriptions.subscribe(topicName);
|
||||||
subscriptions.changePartitionAssignment(Collections.singletonList(tp));
|
subscriptions.changePartitionAssignment(Collections.singletonList(tp));
|
||||||
|
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat
|
RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
|
||||||
assertEquals(1, client.inFlightRequestCount());
|
assertEquals(1, consumerClient.pendingRequestCount());
|
||||||
client.respond(heartbeatResponse(Errors.ILLEGAL_GENERATION.code()));
|
assertFalse(future.isDone());
|
||||||
|
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.ILLEGAL_GENERATION.code()));
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
consumerClient.poll(0);
|
||||||
|
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertTrue(future.failed());
|
||||||
|
assertEquals(Errors.ILLEGAL_GENERATION.exception(), future.exception());
|
||||||
|
assertTrue(subscriptions.partitionAssignmentNeeded());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUnknownConsumerId() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// illegal_generation will cause re-partition
|
||||||
|
subscriptions.subscribe(topicName);
|
||||||
|
subscriptions.changePartitionAssignment(Collections.singletonList(tp));
|
||||||
|
|
||||||
|
time.sleep(sessionTimeoutMs);
|
||||||
|
RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
|
||||||
|
assertEquals(1, consumerClient.pendingRequestCount());
|
||||||
|
assertFalse(future.isDone());
|
||||||
|
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.UNKNOWN_CONSUMER_ID.code()));
|
||||||
|
time.sleep(sessionTimeoutMs);
|
||||||
|
consumerClient.poll(0);
|
||||||
|
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertTrue(future.failed());
|
||||||
|
assertEquals(Errors.UNKNOWN_CONSUMER_ID.exception(), future.exception());
|
||||||
assertTrue(subscriptions.partitionAssignmentNeeded());
|
assertTrue(subscriptions.partitionAssignmentNeeded());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCoordinatorDisconnect() {
|
public void testCoordinatorDisconnect() {
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// coordinator disconnect will mark coordinator as unknown
|
// coordinator disconnect will mark coordinator as unknown
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat
|
RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
|
||||||
assertEquals(1, client.inFlightRequestCount());
|
assertEquals(1, consumerClient.pendingRequestCount());
|
||||||
client.respond(heartbeatResponse(Errors.NONE.code()), true); // return disconnected
|
assertFalse(future.isDone());
|
||||||
|
|
||||||
|
client.prepareResponse(heartbeatResponse(Errors.NONE.code()), true); // return disconnected
|
||||||
time.sleep(sessionTimeoutMs);
|
time.sleep(sessionTimeoutMs);
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
consumerClient.poll(0);
|
||||||
|
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertTrue(future.failed());
|
||||||
|
assertTrue(future.exception() instanceof DisconnectException);
|
||||||
assertTrue(coordinator.coordinatorUnknown());
|
assertTrue(coordinator.coordinatorUnknown());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -162,16 +235,18 @@ public class CoordinatorTest {
|
||||||
subscriptions.needReassignment();
|
subscriptions.needReassignment();
|
||||||
|
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// normal join group
|
// normal join group
|
||||||
client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()));
|
client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()));
|
||||||
coordinator.assignPartitions(time.milliseconds());
|
coordinator.ensurePartitionAssignment();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
assertFalse(subscriptions.partitionAssignmentNeeded());
|
assertFalse(subscriptions.partitionAssignmentNeeded());
|
||||||
assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
|
assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
|
||||||
|
assertEquals(1, rebalanceCallback.revokedCount);
|
||||||
|
assertEquals(Collections.emptySet(), rebalanceCallback.revoked);
|
||||||
|
assertEquals(1, rebalanceCallback.assignedCount);
|
||||||
|
assertEquals(Collections.singleton(tp), rebalanceCallback.assigned);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -180,165 +255,228 @@ public class CoordinatorTest {
|
||||||
subscriptions.needReassignment();
|
subscriptions.needReassignment();
|
||||||
|
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
assertTrue(subscriptions.partitionAssignmentNeeded());
|
|
||||||
|
|
||||||
// diconnected from original coordinator will cause re-discover and join again
|
// disconnected from original coordinator will cause re-discover and join again
|
||||||
client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()), true);
|
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()));
|
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()));
|
client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()));
|
||||||
coordinator.assignPartitions(time.milliseconds());
|
coordinator.ensurePartitionAssignment();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
assertFalse(subscriptions.partitionAssignmentNeeded());
|
assertFalse(subscriptions.partitionAssignmentNeeded());
|
||||||
assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
|
assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
|
||||||
|
assertEquals(1, rebalanceCallback.revokedCount);
|
||||||
|
assertEquals(Collections.emptySet(), rebalanceCallback.revoked);
|
||||||
|
assertEquals(1, rebalanceCallback.assignedCount);
|
||||||
|
assertEquals(Collections.singleton(tp), rebalanceCallback.assigned);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test(expected = ApiException.class)
|
||||||
|
public void testUnknownPartitionAssignmentStrategy() {
|
||||||
|
subscriptions.subscribe(topicName);
|
||||||
|
subscriptions.needReassignment();
|
||||||
|
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// coordinator doesn't like our assignment strategy
|
||||||
|
client.prepareResponse(joinGroupResponse(0, "consumer", Collections.<TopicPartition>emptyList(), Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()));
|
||||||
|
coordinator.ensurePartitionAssignment();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = ApiException.class)
|
||||||
|
public void testInvalidSessionTimeout() {
|
||||||
|
subscriptions.subscribe(topicName);
|
||||||
|
subscriptions.needReassignment();
|
||||||
|
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// coordinator doesn't like our assignment strategy
|
||||||
|
client.prepareResponse(joinGroupResponse(0, "consumer", Collections.<TopicPartition>emptyList(), Errors.INVALID_SESSION_TIMEOUT.code()));
|
||||||
|
coordinator.ensurePartitionAssignment();
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCommitOffsetNormal() {
|
public void testCommitOffsetNormal() {
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// With success flag
|
|
||||||
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
|
||||||
RequestFuture<Void> result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds());
|
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
|
||||||
assertTrue(result.isDone());
|
|
||||||
assertTrue(result.succeeded());
|
|
||||||
|
|
||||||
// Without success flag
|
AtomicBoolean success = new AtomicBoolean(false);
|
||||||
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds());
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, callback(success));
|
||||||
client.respond(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
|
consumerClient.poll(0);
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
assertTrue(success.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCommitOffsetError() {
|
public void testCommitOffsetAsyncCoordinatorNotAvailable() {
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// async commit with coordinator not available
|
// async commit with coordinator not available
|
||||||
|
MockCommitCallback cb = new MockCommitCallback();
|
||||||
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())));
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())));
|
||||||
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds());
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb);
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
consumerClient.poll(0);
|
||||||
|
|
||||||
assertTrue(coordinator.coordinatorUnknown());
|
assertTrue(coordinator.coordinatorUnknown());
|
||||||
// resume
|
assertEquals(1, cb.invoked);
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), cb.exception);
|
||||||
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), time.milliseconds());
|
|
||||||
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())));
|
|
||||||
RequestFuture<Void> result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds());
|
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
|
||||||
assertTrue(result.isDone());
|
|
||||||
assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction());
|
|
||||||
|
|
||||||
// sync commit with coordinator disconnected
|
|
||||||
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true);
|
|
||||||
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
|
|
||||||
result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds());
|
|
||||||
|
|
||||||
assertEquals(0, client.poll(0, time.milliseconds()).size());
|
|
||||||
assertTrue(result.isDone());
|
|
||||||
assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction());
|
|
||||||
|
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
|
||||||
coordinator.discoverConsumerCoordinator();
|
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds());
|
|
||||||
assertEquals(1, client.poll(0, time.milliseconds()).size());
|
|
||||||
assertTrue(result.isDone());
|
|
||||||
assertTrue(result.succeeded());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitOffsetAsyncNotCoordinator() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// async commit with not coordinator
|
||||||
|
MockCommitCallback cb = new MockCommitCallback();
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code())));
|
||||||
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb);
|
||||||
|
consumerClient.poll(0);
|
||||||
|
|
||||||
|
assertTrue(coordinator.coordinatorUnknown());
|
||||||
|
assertEquals(1, cb.invoked);
|
||||||
|
assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), cb.exception);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFetchOffset() {
|
public void testCommitOffsetAsyncDisconnected() {
|
||||||
|
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
coordinator.discoverConsumerCoordinator();
|
coordinator.ensureCoordinatorKnown();
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
|
|
||||||
// normal fetch
|
// async commit with coordinator disconnected
|
||||||
|
MockCommitCallback cb = new MockCommitCallback();
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true);
|
||||||
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb);
|
||||||
|
consumerClient.poll(0);
|
||||||
|
|
||||||
|
assertTrue(coordinator.coordinatorUnknown());
|
||||||
|
assertEquals(1, cb.invoked);
|
||||||
|
assertTrue(cb.exception instanceof DisconnectException);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitOffsetSyncNotCoordinator() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
|
||||||
|
MockCommitCallback cb = new MockCommitCallback();
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code())));
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
|
||||||
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb);
|
||||||
|
assertEquals(1, cb.invoked);
|
||||||
|
assertNull(cb.exception);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitOffsetSyncCoordinatorNotAvailable() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
|
||||||
|
MockCommitCallback cb = new MockCommitCallback();
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())));
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
|
||||||
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb);
|
||||||
|
assertEquals(1, cb.invoked);
|
||||||
|
assertNull(cb.exception);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitOffsetSyncCoordinatorDisconnected() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
|
||||||
|
MockCommitCallback cb = new MockCommitCallback();
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true);
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
|
||||||
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb);
|
||||||
|
assertEquals(1, cb.invoked);
|
||||||
|
assertNull(cb.exception);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = ApiException.class)
|
||||||
|
public void testCommitOffsetSyncThrowsNonRetriableException() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// sync commit with invalid partitions should throw if we have no callback
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code())), false);
|
||||||
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitOffsetSyncCallbackHandlesNonRetriableException() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
// sync commit with invalid partitions should throw if we have no callback
|
||||||
|
MockCommitCallback cb = new MockCommitCallback();
|
||||||
|
client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code())), false);
|
||||||
|
coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb);
|
||||||
|
assertTrue(cb.exception instanceof ApiException);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRefreshOffset() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
subscriptions.subscribe(tp);
|
||||||
|
subscriptions.needRefreshCommits();
|
||||||
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
|
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
|
||||||
RequestFuture<Map<TopicPartition, Long>> result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds());
|
coordinator.refreshCommittedOffsetsIfNeeded();
|
||||||
client.poll(0, time.milliseconds());
|
assertFalse(subscriptions.refreshCommitsNeeded());
|
||||||
assertTrue(result.isDone());
|
assertEquals(100L, (long) subscriptions.committed(tp));
|
||||||
assertEquals(100L, (long) result.value().get(tp));
|
}
|
||||||
|
|
||||||
// fetch with loading in progress
|
@Test
|
||||||
|
public void testRefreshOffsetLoadInProgress() {
|
||||||
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
|
coordinator.ensureCoordinatorKnown();
|
||||||
|
|
||||||
|
subscriptions.subscribe(tp);
|
||||||
|
subscriptions.needRefreshCommits();
|
||||||
client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L));
|
client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L));
|
||||||
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
|
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
|
||||||
|
coordinator.refreshCommittedOffsetsIfNeeded();
|
||||||
|
assertFalse(subscriptions.refreshCommitsNeeded());
|
||||||
|
assertEquals(100L, (long) subscriptions.committed(tp));
|
||||||
|
}
|
||||||
|
|
||||||
result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds());
|
@Test
|
||||||
client.poll(0, time.milliseconds());
|
public void testRefreshOffsetNotCoordinatorForConsumer() {
|
||||||
assertTrue(result.isDone());
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
assertTrue(result.failed());
|
coordinator.ensureCoordinatorKnown();
|
||||||
assertEquals(RequestFuture.RetryAction.BACKOFF, result.retryAction());
|
|
||||||
|
|
||||||
result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds());
|
subscriptions.subscribe(tp);
|
||||||
client.poll(0, time.milliseconds());
|
subscriptions.needRefreshCommits();
|
||||||
assertTrue(result.isDone());
|
|
||||||
assertEquals(100L, (long) result.value().get(tp));
|
|
||||||
|
|
||||||
// fetch with not coordinator
|
|
||||||
client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L));
|
client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L));
|
||||||
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
|
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
|
||||||
|
coordinator.refreshCommittedOffsetsIfNeeded();
|
||||||
|
assertFalse(subscriptions.refreshCommitsNeeded());
|
||||||
|
assertEquals(100L, (long) subscriptions.committed(tp));
|
||||||
|
}
|
||||||
|
|
||||||
result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds());
|
@Test
|
||||||
client.poll(0, time.milliseconds());
|
public void testRefreshOffsetWithNoFetchableOffsets() {
|
||||||
assertTrue(result.isDone());
|
client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
|
||||||
assertTrue(result.failed());
|
coordinator.ensureCoordinatorKnown();
|
||||||
assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction());
|
|
||||||
|
|
||||||
coordinator.discoverConsumerCoordinator();
|
subscriptions.subscribe(tp);
|
||||||
client.poll(0, time.milliseconds());
|
subscriptions.needRefreshCommits();
|
||||||
|
|
||||||
result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds());
|
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
assertTrue(result.isDone());
|
|
||||||
assertEquals(100L, (long) result.value().get(tp));
|
|
||||||
|
|
||||||
// fetch with no fetchable offsets
|
|
||||||
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L));
|
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L));
|
||||||
result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds());
|
coordinator.refreshCommittedOffsetsIfNeeded();
|
||||||
client.poll(0, time.milliseconds());
|
assertFalse(subscriptions.refreshCommitsNeeded());
|
||||||
assertTrue(result.isDone());
|
assertEquals(null, subscriptions.committed(tp));
|
||||||
assertTrue(result.value().isEmpty());
|
|
||||||
|
|
||||||
// fetch with offset -1
|
|
||||||
client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L));
|
|
||||||
result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds());
|
|
||||||
client.poll(0, time.milliseconds());
|
|
||||||
assertTrue(result.isDone());
|
|
||||||
assertTrue(result.value().isEmpty());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private Struct consumerMetadataResponse(Node node, short error) {
|
private Struct consumerMetadataResponse(Node node, short error) {
|
||||||
|
@ -366,4 +504,45 @@ public class CoordinatorTest {
|
||||||
OffsetFetchResponse response = new OffsetFetchResponse(Collections.singletonMap(tp, data));
|
OffsetFetchResponse response = new OffsetFetchResponse(Collections.singletonMap(tp, data));
|
||||||
return response.toStruct();
|
return response.toStruct();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private ConsumerCommitCallback callback(final AtomicBoolean success) {
|
||||||
|
return new ConsumerCommitCallback() {
|
||||||
|
@Override
|
||||||
|
public void onComplete(Map<TopicPartition, Long> offsets, Exception exception) {
|
||||||
|
if (exception == null)
|
||||||
|
success.set(true);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class MockCommitCallback implements ConsumerCommitCallback {
|
||||||
|
public int invoked = 0;
|
||||||
|
public Exception exception = null;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onComplete(Map<TopicPartition, Long> offsets, Exception exception) {
|
||||||
|
invoked++;
|
||||||
|
this.exception = exception;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class MockRebalanceCallback implements Coordinator.RebalanceCallback {
|
||||||
|
public Collection<TopicPartition> revoked;
|
||||||
|
public Collection<TopicPartition> assigned;
|
||||||
|
public int revokedCount = 0;
|
||||||
|
public int assignedCount = 0;
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
|
||||||
|
this.assigned = partitions;
|
||||||
|
assignedCount++;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
|
||||||
|
this.revoked = partitions;
|
||||||
|
revokedCount++;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,89 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
public class DelayedTaskQueueTest {
|
||||||
|
private DelayedTaskQueue scheduler = new DelayedTaskQueue();
|
||||||
|
private ArrayList<DelayedTask> executed = new ArrayList<DelayedTask>();
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testScheduling() {
|
||||||
|
// Empty scheduler
|
||||||
|
assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(0));
|
||||||
|
scheduler.poll(0);
|
||||||
|
assertEquals(Collections.emptyList(), executed);
|
||||||
|
|
||||||
|
TestTask task1 = new TestTask();
|
||||||
|
TestTask task2 = new TestTask();
|
||||||
|
TestTask task3 = new TestTask();
|
||||||
|
scheduler.add(task1, 20);
|
||||||
|
assertEquals(20, scheduler.nextTimeout(0));
|
||||||
|
scheduler.add(task2, 10);
|
||||||
|
assertEquals(10, scheduler.nextTimeout(0));
|
||||||
|
scheduler.add(task3, 30);
|
||||||
|
assertEquals(10, scheduler.nextTimeout(0));
|
||||||
|
|
||||||
|
scheduler.poll(5);
|
||||||
|
assertEquals(Collections.emptyList(), executed);
|
||||||
|
assertEquals(5, scheduler.nextTimeout(5));
|
||||||
|
|
||||||
|
scheduler.poll(10);
|
||||||
|
assertEquals(Arrays.asList(task2), executed);
|
||||||
|
assertEquals(10, scheduler.nextTimeout(10));
|
||||||
|
|
||||||
|
scheduler.poll(20);
|
||||||
|
assertEquals(Arrays.asList(task2, task1), executed);
|
||||||
|
assertEquals(20, scheduler.nextTimeout(10));
|
||||||
|
|
||||||
|
scheduler.poll(30);
|
||||||
|
assertEquals(Arrays.asList(task2, task1, task3), executed);
|
||||||
|
assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(30));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRemove() {
|
||||||
|
TestTask task1 = new TestTask();
|
||||||
|
TestTask task2 = new TestTask();
|
||||||
|
TestTask task3 = new TestTask();
|
||||||
|
scheduler.add(task1, 20);
|
||||||
|
scheduler.add(task2, 10);
|
||||||
|
scheduler.add(task3, 30);
|
||||||
|
scheduler.add(task1, 40);
|
||||||
|
assertEquals(10, scheduler.nextTimeout(0));
|
||||||
|
|
||||||
|
scheduler.remove(task2);
|
||||||
|
assertEquals(20, scheduler.nextTimeout(0));
|
||||||
|
|
||||||
|
scheduler.remove(task1);
|
||||||
|
assertEquals(30, scheduler.nextTimeout(0));
|
||||||
|
|
||||||
|
scheduler.remove(task3);
|
||||||
|
assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
private class TestTask implements DelayedTask {
|
||||||
|
@Override
|
||||||
|
public void run(long now) {
|
||||||
|
executed.add(this);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -52,6 +52,7 @@ public class FetcherTest {
|
||||||
private int minBytes = 1;
|
private int minBytes = 1;
|
||||||
private int maxWaitMs = 0;
|
private int maxWaitMs = 0;
|
||||||
private int fetchSize = 1000;
|
private int fetchSize = 1000;
|
||||||
|
private long retryBackoffMs = 100;
|
||||||
private MockTime time = new MockTime();
|
private MockTime time = new MockTime();
|
||||||
private MockClient client = new MockClient(time);
|
private MockClient client = new MockClient(time);
|
||||||
private Metadata metadata = new Metadata(0, Long.MAX_VALUE);
|
private Metadata metadata = new Metadata(0, Long.MAX_VALUE);
|
||||||
|
@ -60,10 +61,11 @@ public class FetcherTest {
|
||||||
private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST);
|
private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST);
|
||||||
private Metrics metrics = new Metrics(time);
|
private Metrics metrics = new Metrics(time);
|
||||||
private Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
private Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||||
|
private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100);
|
||||||
|
|
||||||
private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
|
private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
|
||||||
|
|
||||||
private Fetcher<byte[], byte[]> fetcher = new Fetcher<byte[], byte[]>(client,
|
private Fetcher<byte[], byte[]> fetcher = new Fetcher<byte[], byte[]>(consumerClient,
|
||||||
minBytes,
|
minBytes,
|
||||||
maxWaitMs,
|
maxWaitMs,
|
||||||
fetchSize,
|
fetchSize,
|
||||||
|
@ -75,7 +77,8 @@ public class FetcherTest {
|
||||||
metrics,
|
metrics,
|
||||||
"consumer" + groupId,
|
"consumer" + groupId,
|
||||||
metricTags,
|
metricTags,
|
||||||
time);
|
time,
|
||||||
|
retryBackoffMs);
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
|
@ -97,9 +100,9 @@ public class FetcherTest {
|
||||||
subscriptions.consumed(tp, 0);
|
subscriptions.consumed(tp, 0);
|
||||||
|
|
||||||
// normal fetch
|
// normal fetch
|
||||||
fetcher.initFetches(cluster, time.milliseconds());
|
fetcher.initFetches(cluster);
|
||||||
client.respond(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L));
|
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L));
|
||||||
client.poll(0, time.milliseconds());
|
consumerClient.poll(0);
|
||||||
records = fetcher.fetchedRecords().get(tp);
|
records = fetcher.fetchedRecords().get(tp);
|
||||||
assertEquals(3, records.size());
|
assertEquals(3, records.size());
|
||||||
assertEquals(4L, (long) subscriptions.fetched(tp)); // this is the next fetching position
|
assertEquals(4L, (long) subscriptions.fetched(tp)); // this is the next fetching position
|
||||||
|
@ -119,24 +122,24 @@ public class FetcherTest {
|
||||||
subscriptions.consumed(tp, 0);
|
subscriptions.consumed(tp, 0);
|
||||||
|
|
||||||
// fetch with not leader
|
// fetch with not leader
|
||||||
fetcher.initFetches(cluster, time.milliseconds());
|
fetcher.initFetches(cluster);
|
||||||
client.respond(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L));
|
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L));
|
||||||
client.poll(0, time.milliseconds());
|
consumerClient.poll(0);
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
|
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
|
||||||
|
|
||||||
// fetch with unknown topic partition
|
// fetch with unknown topic partition
|
||||||
fetcher.initFetches(cluster, time.milliseconds());
|
fetcher.initFetches(cluster);
|
||||||
client.respond(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L));
|
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L));
|
||||||
client.poll(0, time.milliseconds());
|
consumerClient.poll(0);
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
|
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
|
||||||
|
|
||||||
// fetch with out of range
|
// fetch with out of range
|
||||||
subscriptions.fetched(tp, 5);
|
subscriptions.fetched(tp, 5);
|
||||||
fetcher.initFetches(cluster, time.milliseconds());
|
fetcher.initFetches(cluster);
|
||||||
client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L));
|
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L));
|
||||||
client.poll(0, time.milliseconds());
|
consumerClient.poll(0);
|
||||||
assertTrue(subscriptions.isOffsetResetNeeded(tp));
|
assertTrue(subscriptions.isOffsetResetNeeded(tp));
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
assertEquals(null, subscriptions.fetched(tp));
|
assertEquals(null, subscriptions.fetched(tp));
|
||||||
|
@ -151,9 +154,9 @@ public class FetcherTest {
|
||||||
subscriptions.consumed(tp, 5);
|
subscriptions.consumed(tp, 5);
|
||||||
|
|
||||||
// fetch with out of range
|
// fetch with out of range
|
||||||
fetcher.initFetches(cluster, time.milliseconds());
|
fetcher.initFetches(cluster);
|
||||||
client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L));
|
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L));
|
||||||
client.poll(0, time.milliseconds());
|
consumerClient.poll(0);
|
||||||
assertTrue(subscriptions.isOffsetResetNeeded(tp));
|
assertTrue(subscriptions.isOffsetResetNeeded(tp));
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
assertEquals(null, subscriptions.fetched(tp));
|
assertEquals(null, subscriptions.fetched(tp));
|
||||||
|
|
|
@ -51,4 +51,19 @@ public class HeartbeatTest {
|
||||||
assertEquals(0, heartbeat.timeToNextHeartbeat(100));
|
assertEquals(0, heartbeat.timeToNextHeartbeat(100));
|
||||||
assertEquals(0, heartbeat.timeToNextHeartbeat(200));
|
assertEquals(0, heartbeat.timeToNextHeartbeat(200));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSessionTimeoutExpired() {
|
||||||
|
heartbeat.sentHeartbeat(time.milliseconds());
|
||||||
|
time.sleep(305);
|
||||||
|
assertTrue(heartbeat.sessionTimeoutExpired(time.milliseconds()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testResetSession() {
|
||||||
|
heartbeat.sentHeartbeat(time.milliseconds());
|
||||||
|
time.sleep(305);
|
||||||
|
heartbeat.resetSessionTimeout(time.milliseconds());
|
||||||
|
assertFalse(heartbeat.sessionTimeoutExpired(time.milliseconds()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,57 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
|
||||||
|
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
|
||||||
|
* License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
|
||||||
|
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
public class RequestFutureTest {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testComposeSuccessCase() {
|
||||||
|
RequestFuture<String> future = new RequestFuture<String>();
|
||||||
|
RequestFuture<Integer> composed = future.compose(new RequestFutureAdapter<String, Integer>() {
|
||||||
|
@Override
|
||||||
|
public void onSuccess(String value, RequestFuture<Integer> future) {
|
||||||
|
future.complete(value.length());
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
future.complete("hello");
|
||||||
|
|
||||||
|
assertTrue(composed.isDone());
|
||||||
|
assertTrue(composed.succeeded());
|
||||||
|
assertEquals(5, (int) composed.value());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testComposeFailureCase() {
|
||||||
|
RequestFuture<String> future = new RequestFuture<String>();
|
||||||
|
RequestFuture<Integer> composed = future.compose(new RequestFutureAdapter<String, Integer>() {
|
||||||
|
@Override
|
||||||
|
public void onSuccess(String value, RequestFuture<Integer> future) {
|
||||||
|
future.complete(value.length());
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
RuntimeException e = new RuntimeException();
|
||||||
|
future.raise(e);
|
||||||
|
|
||||||
|
assertTrue(composed.isDone());
|
||||||
|
assertTrue(composed.failed());
|
||||||
|
assertEquals(e, composed.exception());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -163,7 +163,13 @@ object AdminUtils extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
def deleteTopic(zkClient: ZkClient, topic: String) {
|
def deleteTopic(zkClient: ZkClient, topic: String) {
|
||||||
|
try {
|
||||||
ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic))
|
ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic))
|
||||||
|
} catch {
|
||||||
|
case e1: ZkNodeExistsException => throw new TopicAlreadyMarkedForDeletionException(
|
||||||
|
"topic %s is already marked for deletion".format(topic))
|
||||||
|
case e2: Throwable => throw new AdminOperationException(e2.toString)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def isConsumerGroupActive(zkClient: ZkClient, group: String) = {
|
def isConsumerGroupActive(zkClient: ZkClient, group: String) = {
|
||||||
|
@ -244,8 +250,19 @@ object AdminUtils extends Logging {
|
||||||
require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.")
|
require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.")
|
||||||
|
|
||||||
val topicPath = ZkUtils.getTopicPath(topic)
|
val topicPath = ZkUtils.getTopicPath(topic)
|
||||||
if(!update && zkClient.exists(topicPath))
|
|
||||||
|
if (!update) {
|
||||||
|
if (zkClient.exists(topicPath))
|
||||||
throw new TopicExistsException("Topic \"%s\" already exists.".format(topic))
|
throw new TopicExistsException("Topic \"%s\" already exists.".format(topic))
|
||||||
|
else if (Topic.hasCollisionChars(topic)) {
|
||||||
|
val allTopics = ZkUtils.getAllTopics(zkClient)
|
||||||
|
val collidingTopics = allTopics.filter(t => Topic.hasCollision(topic, t))
|
||||||
|
if (collidingTopics.nonEmpty) {
|
||||||
|
throw new InvalidTopicException("Topic \"%s\" collides with existing topics: %s".format(topic, collidingTopics.mkString(", ")))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment))
|
partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment))
|
||||||
|
|
||||||
// write out the config if there is any, this isn't transactional with the partition assignments
|
// write out the config if there is any, this isn't transactional with the partition assignments
|
||||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.kafka.common.utils.Utils
|
||||||
import kafka.coordinator.ConsumerCoordinator
|
import kafka.coordinator.ConsumerCoordinator
|
||||||
|
|
||||||
|
|
||||||
object TopicCommand {
|
object TopicCommand extends Logging {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
|
|
||||||
|
@ -48,7 +48,7 @@ object TopicCommand {
|
||||||
opts.checkArgs()
|
opts.checkArgs()
|
||||||
|
|
||||||
val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000)
|
val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000)
|
||||||
|
var exitCode = 0
|
||||||
try {
|
try {
|
||||||
if(opts.options.has(opts.createOpt))
|
if(opts.options.has(opts.createOpt))
|
||||||
createTopic(zkClient, opts)
|
createTopic(zkClient, opts)
|
||||||
|
@ -62,11 +62,14 @@ object TopicCommand {
|
||||||
deleteTopic(zkClient, opts)
|
deleteTopic(zkClient, opts)
|
||||||
} catch {
|
} catch {
|
||||||
case e: Throwable =>
|
case e: Throwable =>
|
||||||
println("Error while executing topic command " + e.getMessage)
|
println("Error while executing topic command : " + e.getMessage)
|
||||||
println(Utils.stackTrace(e))
|
error(Utils.stackTrace(e))
|
||||||
|
exitCode = 1
|
||||||
} finally {
|
} finally {
|
||||||
zkClient.close()
|
zkClient.close()
|
||||||
|
System.exit(exitCode)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def getTopics(zkClient: ZkClient, opts: TopicCommandOptions): Seq[String] = {
|
private def getTopics(zkClient: ZkClient, opts: TopicCommandOptions): Seq[String] = {
|
||||||
|
@ -82,9 +85,11 @@ object TopicCommand {
|
||||||
def createTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
|
def createTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
|
||||||
val topic = opts.options.valueOf(opts.topicOpt)
|
val topic = opts.options.valueOf(opts.topicOpt)
|
||||||
val configs = parseTopicConfigsToBeAdded(opts)
|
val configs = parseTopicConfigsToBeAdded(opts)
|
||||||
|
if (Topic.hasCollisionChars(topic))
|
||||||
|
println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could collide. To avoid issues it is best to use either, but not both.")
|
||||||
if (opts.options.has(opts.replicaAssignmentOpt)) {
|
if (opts.options.has(opts.replicaAssignmentOpt)) {
|
||||||
val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt))
|
val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt))
|
||||||
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs)
|
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs, update = false)
|
||||||
} else {
|
} else {
|
||||||
CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt)
|
CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt)
|
||||||
val partitions = opts.options.valueOf(opts.partitionsOpt).intValue
|
val partitions = opts.options.valueOf(opts.partitionsOpt).intValue
|
||||||
|
@ -97,7 +102,8 @@ object TopicCommand {
|
||||||
def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
|
def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
|
||||||
val topics = getTopics(zkClient, opts)
|
val topics = getTopics(zkClient, opts)
|
||||||
if (topics.length == 0) {
|
if (topics.length == 0) {
|
||||||
println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt)))
|
throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt),
|
||||||
|
opts.options.valueOf(opts.zkConnectOpt)))
|
||||||
}
|
}
|
||||||
topics.foreach { topic =>
|
topics.foreach { topic =>
|
||||||
val configs = AdminUtils.fetchTopicConfig(zkClient, topic)
|
val configs = AdminUtils.fetchTopicConfig(zkClient, topic)
|
||||||
|
@ -138,7 +144,8 @@ object TopicCommand {
|
||||||
def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
|
def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
|
||||||
val topics = getTopics(zkClient, opts)
|
val topics = getTopics(zkClient, opts)
|
||||||
if (topics.length == 0) {
|
if (topics.length == 0) {
|
||||||
println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt)))
|
throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt),
|
||||||
|
opts.options.valueOf(opts.zkConnectOpt)))
|
||||||
}
|
}
|
||||||
topics.foreach { topic =>
|
topics.foreach { topic =>
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -34,7 +34,7 @@ object TopicMetadata {
|
||||||
val partitionsMetadata: Array[PartitionMetadata] = new Array[PartitionMetadata](numPartitions)
|
val partitionsMetadata: Array[PartitionMetadata] = new Array[PartitionMetadata](numPartitions)
|
||||||
for(i <- 0 until numPartitions) {
|
for(i <- 0 until numPartitions) {
|
||||||
val partitionMetadata = PartitionMetadata.readFrom(buffer, brokers)
|
val partitionMetadata = PartitionMetadata.readFrom(buffer, brokers)
|
||||||
partitionsMetadata(partitionMetadata.partitionId) = partitionMetadata
|
partitionsMetadata(i) = partitionMetadata
|
||||||
}
|
}
|
||||||
new TopicMetadata(topic, partitionsMetadata, errorCode)
|
new TopicMetadata(topic, partitionsMetadata, errorCode)
|
||||||
}
|
}
|
||||||
|
|
|
@ -43,4 +43,26 @@ object Topic {
|
||||||
case None => throw new InvalidTopicException("topic name " + topic + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'")
|
case None => throw new InvalidTopicException("topic name " + topic + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Due to limitations in metric names, topics with a period ('.') or underscore ('_') could collide.
|
||||||
|
*
|
||||||
|
* @param topic The topic to check for colliding character
|
||||||
|
* @return true if the topic has collision characters
|
||||||
|
*/
|
||||||
|
def hasCollisionChars(topic: String): Boolean = {
|
||||||
|
topic.contains("_") || topic.contains(".")
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if the topicNames collide due to a period ('.') or underscore ('_') in the same position.
|
||||||
|
*
|
||||||
|
* @param topicA A topic to check for collision
|
||||||
|
* @param topicB A topic to check for collision
|
||||||
|
* @return true if the topics collide
|
||||||
|
*/
|
||||||
|
def hasCollision(topicA: String, topicB: String): Boolean = {
|
||||||
|
topicA.replace('.', '_') == topicB.replace('.', '_')
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,21 @@
|
||||||
|
/**
|
||||||
|
* 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 kafka.common
|
||||||
|
|
||||||
|
class TopicAlreadyMarkedForDeletionException(message: String) extends RuntimeException(message) {
|
||||||
|
}
|
|
@ -58,6 +58,7 @@ object ConsumerConfig extends Config {
|
||||||
validateGroupId(config.groupId)
|
validateGroupId(config.groupId)
|
||||||
validateAutoOffsetReset(config.autoOffsetReset)
|
validateAutoOffsetReset(config.autoOffsetReset)
|
||||||
validateOffsetsStorage(config.offsetsStorage)
|
validateOffsetsStorage(config.offsetsStorage)
|
||||||
|
validatePartitionAssignmentStrategy(config.partitionAssignmentStrategy)
|
||||||
}
|
}
|
||||||
|
|
||||||
def validateClientId(clientId: String) {
|
def validateClientId(clientId: String) {
|
||||||
|
@ -85,6 +86,15 @@ object ConsumerConfig extends Config {
|
||||||
"Valid values are 'zookeeper' and 'kafka'")
|
"Valid values are 'zookeeper' and 'kafka'")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def validatePartitionAssignmentStrategy(strategy: String) {
|
||||||
|
strategy match {
|
||||||
|
case "range" =>
|
||||||
|
case "roundrobin" =>
|
||||||
|
case _ => throw new InvalidConfigException("Wrong value " + strategy + " of partition.assignment.strategy in consumer config; " +
|
||||||
|
"Valid values are 'range' and 'roundrobin'")
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) {
|
class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) {
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
package kafka.consumer
|
package kafka.consumer
|
||||||
|
|
||||||
|
|
||||||
import java.nio.channels.ClosedByInterruptException
|
import java.nio.channels.{AsynchronousCloseException, ClosedByInterruptException}
|
||||||
|
|
||||||
import kafka.api._
|
import kafka.api._
|
||||||
import kafka.network._
|
import kafka.network._
|
||||||
|
@ -59,6 +59,16 @@ class SimpleConsumer(val host: String,
|
||||||
connect()
|
connect()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unblock thread by closing channel and triggering AsynchronousCloseException if a read operation is in progress.
|
||||||
|
*
|
||||||
|
* This handles a bug found in Java 1.7 and below, where interrupting a thread can not correctly unblock
|
||||||
|
* the thread from waiting on ReadableByteChannel.read().
|
||||||
|
*/
|
||||||
|
def disconnectToHandleJavaIOBug() = {
|
||||||
|
disconnect()
|
||||||
|
}
|
||||||
|
|
||||||
def close() {
|
def close() {
|
||||||
lock synchronized {
|
lock synchronized {
|
||||||
disconnect()
|
disconnect()
|
||||||
|
@ -76,6 +86,9 @@ class SimpleConsumer(val host: String,
|
||||||
} catch {
|
} catch {
|
||||||
case e : ClosedByInterruptException =>
|
case e : ClosedByInterruptException =>
|
||||||
throw e
|
throw e
|
||||||
|
// Should not observe this exception when running Kafka with Java 1.8
|
||||||
|
case e: AsynchronousCloseException =>
|
||||||
|
throw e
|
||||||
case e : Throwable =>
|
case e : Throwable =>
|
||||||
info("Reconnect due to socket error: %s".format(e.toString))
|
info("Reconnect due to socket error: %s".format(e.toString))
|
||||||
// retry once
|
// retry once
|
||||||
|
|
|
@ -387,8 +387,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
|
||||||
/**
|
/**
|
||||||
* This callback is invoked by the replica state machine's broker change listener, with the list of newly started
|
* This callback is invoked by the replica state machine's broker change listener, with the list of newly started
|
||||||
* brokers as input. It does the following -
|
* brokers as input. It does the following -
|
||||||
* 1. Triggers the OnlinePartition state change for all new/offline partitions
|
* 1. Sends update metadata request to all live and shutting down brokers
|
||||||
* 2. It checks whether there are reassigned replicas assigned to any newly started brokers. If
|
* 2. Triggers the OnlinePartition state change for all new/offline partitions
|
||||||
|
* 3. It checks whether there are reassigned replicas assigned to any newly started brokers. If
|
||||||
* so, it performs the reassignment logic for each topic/partition.
|
* so, it performs the reassignment logic for each topic/partition.
|
||||||
*
|
*
|
||||||
* Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point for two reasons:
|
* Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point for two reasons:
|
||||||
|
@ -400,10 +401,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
|
||||||
def onBrokerStartup(newBrokers: Seq[Int]) {
|
def onBrokerStartup(newBrokers: Seq[Int]) {
|
||||||
info("New broker startup callback for %s".format(newBrokers.mkString(",")))
|
info("New broker startup callback for %s".format(newBrokers.mkString(",")))
|
||||||
val newBrokersSet = newBrokers.toSet
|
val newBrokersSet = newBrokers.toSet
|
||||||
// send update metadata request for all partitions to the newly restarted brokers. In cases of controlled shutdown
|
// send update metadata request to all live and shutting down brokers. Old brokers will get to know of the new
|
||||||
// leaders will not be elected when a new broker comes up. So at least in the common controlled shutdown case, the
|
// broker via this update.
|
||||||
// metadata will reach the new brokers faster
|
// In cases of controlled shutdown leaders will not be elected when a new broker comes up. So at least in the
|
||||||
sendUpdateMetadataRequest(newBrokers)
|
// common controlled shutdown case, the metadata will reach the new brokers faster
|
||||||
|
sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq)
|
||||||
// the very first thing to do when a new broker comes up is send it the entire list of partitions that it is
|
// the very first thing to do when a new broker comes up is send it the entire list of partitions that it is
|
||||||
// supposed to host. Based on that the broker starts the high watermark threads for the input list of partitions
|
// supposed to host. Based on that the broker starts the high watermark threads for the input list of partitions
|
||||||
val allReplicasOnNewBrokers = controllerContext.replicasOnBrokers(newBrokersSet)
|
val allReplicasOnNewBrokers = controllerContext.replicasOnBrokers(newBrokersSet)
|
||||||
|
@ -433,6 +435,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
|
||||||
* 1. Mark partitions with dead leaders as offline
|
* 1. Mark partitions with dead leaders as offline
|
||||||
* 2. Triggers the OnlinePartition state change for all new/offline partitions
|
* 2. Triggers the OnlinePartition state change for all new/offline partitions
|
||||||
* 3. Invokes the OfflineReplica state change on the input list of newly started brokers
|
* 3. Invokes the OfflineReplica state change on the input list of newly started brokers
|
||||||
|
* 4. If no partitions are effected then send UpdateMetadataRequest to live or shutting down brokers
|
||||||
*
|
*
|
||||||
* Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point. This is because
|
* Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point. This is because
|
||||||
* the partition state machine will refresh our cache for us when performing leader election for all new/offline
|
* the partition state machine will refresh our cache for us when performing leader election for all new/offline
|
||||||
|
@ -464,6 +467,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
|
||||||
// since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state
|
// since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state
|
||||||
deleteTopicManager.failReplicaDeletion(replicasForTopicsToBeDeleted)
|
deleteTopicManager.failReplicaDeletion(replicasForTopicsToBeDeleted)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// If broker failure did not require leader re-election, inform brokers of failed broker
|
||||||
|
// Note that during leader re-election, brokers update their metadata
|
||||||
|
if (partitionsWithoutLeader.isEmpty) {
|
||||||
|
sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -210,7 +210,7 @@ class ConsumerCoordinator(val brokerId: Int,
|
||||||
responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
|
responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
|
||||||
} else if (!group.has(consumerId)) {
|
} else if (!group.has(consumerId)) {
|
||||||
responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
|
responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
|
||||||
} else if (generationId != group.generationId) {
|
} else if (generationId != group.generationId || !group.is(Stable)) {
|
||||||
responseCallback(Errors.ILLEGAL_GENERATION.code)
|
responseCallback(Errors.ILLEGAL_GENERATION.code)
|
||||||
} else {
|
} else {
|
||||||
val consumer = group.get(consumerId)
|
val consumer = group.get(consumerId)
|
||||||
|
|
|
@ -67,7 +67,9 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke
|
||||||
def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition])
|
def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition])
|
||||||
|
|
||||||
override def shutdown(){
|
override def shutdown(){
|
||||||
initiateShutdown()
|
val justShutdown = initiateShutdown()
|
||||||
|
if (justShutdown && isInterruptible)
|
||||||
|
simpleConsumer.disconnectToHandleJavaIOBug()
|
||||||
inLock(partitionMapLock) {
|
inLock(partitionMapLock) {
|
||||||
partitionMapCond.signalAll()
|
partitionMapCond.signalAll()
|
||||||
}
|
}
|
||||||
|
|
|
@ -12,17 +12,13 @@
|
||||||
*/
|
*/
|
||||||
package kafka.api
|
package kafka.api
|
||||||
|
|
||||||
|
import java.{lang, util}
|
||||||
|
|
||||||
import org.apache.kafka.clients.producer.ProducerConfig
|
import org.apache.kafka.clients.producer.ProducerConfig
|
||||||
import org.apache.kafka.clients.producer.ProducerRecord
|
import org.apache.kafka.clients.producer.ProducerRecord
|
||||||
import org.apache.kafka.clients.consumer.Consumer
|
import org.apache.kafka.clients.consumer._
|
||||||
import org.apache.kafka.clients.consumer.KafkaConsumer
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerRecord
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerConfig
|
|
||||||
import org.apache.kafka.clients.consumer.CommitType
|
|
||||||
import org.apache.kafka.common.serialization.ByteArrayDeserializer
|
import org.apache.kafka.common.serialization.ByteArrayDeserializer
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.clients.consumer.NoOffsetForPartitionException
|
|
||||||
|
|
||||||
import kafka.utils.{TestUtils, Logging}
|
import kafka.utils.{TestUtils, Logging}
|
||||||
import kafka.server.KafkaConfig
|
import kafka.server.KafkaConfig
|
||||||
|
@ -46,6 +42,8 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val part = 0
|
val part = 0
|
||||||
val tp = new TopicPartition(topic, part)
|
val tp = new TopicPartition(topic, part)
|
||||||
|
val part2 = 1
|
||||||
|
val tp2 = new TopicPartition(topic, part2)
|
||||||
|
|
||||||
// configure the servers and clients
|
// configure the servers and clients
|
||||||
this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown
|
this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown
|
||||||
|
@ -56,12 +54,13 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test")
|
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test")
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString)
|
this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString)
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
|
this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
|
||||||
|
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
|
||||||
|
|
||||||
override def setUp() {
|
override def setUp() {
|
||||||
super.setUp()
|
super.setUp()
|
||||||
|
|
||||||
// create the test topic with all the brokers as replicas
|
// create the test topic with all the brokers as replicas
|
||||||
TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers)
|
TestUtils.createTopic(this.zkClient, topic, 2, serverCount, this.servers)
|
||||||
}
|
}
|
||||||
|
|
||||||
def testSimpleConsumption() {
|
def testSimpleConsumption() {
|
||||||
|
@ -74,6 +73,45 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
|
||||||
|
|
||||||
this.consumers(0).seek(tp, 0)
|
this.consumers(0).seek(tp, 0)
|
||||||
consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0)
|
consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0)
|
||||||
|
|
||||||
|
// check async commit callbacks
|
||||||
|
val commitCallback = new CountConsumerCommitCallback()
|
||||||
|
this.consumers(0).commit(CommitType.ASYNC, commitCallback)
|
||||||
|
|
||||||
|
// shouldn't make progress until poll is invoked
|
||||||
|
Thread.sleep(10)
|
||||||
|
assertEquals(0, commitCallback.count)
|
||||||
|
awaitCommitCallback(this.consumers(0), commitCallback)
|
||||||
|
}
|
||||||
|
|
||||||
|
def testCommitSpecifiedOffsets() {
|
||||||
|
sendRecords(5, tp)
|
||||||
|
sendRecords(7, tp2)
|
||||||
|
|
||||||
|
this.consumers(0).subscribe(tp)
|
||||||
|
this.consumers(0).subscribe(tp2)
|
||||||
|
|
||||||
|
// Need to poll to join the group
|
||||||
|
this.consumers(0).poll(50)
|
||||||
|
val pos1 = this.consumers(0).position(tp)
|
||||||
|
val pos2 = this.consumers(0).position(tp2)
|
||||||
|
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp, 3L)), CommitType.SYNC)
|
||||||
|
assertEquals(3, this.consumers(0).committed(tp))
|
||||||
|
intercept[NoOffsetForPartitionException] {
|
||||||
|
this.consumers(0).committed(tp2)
|
||||||
|
}
|
||||||
|
// positions should not change
|
||||||
|
assertEquals(pos1, this.consumers(0).position(tp))
|
||||||
|
assertEquals(pos2, this.consumers(0).position(tp2))
|
||||||
|
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 5L)), CommitType.SYNC)
|
||||||
|
assertEquals(3, this.consumers(0).committed(tp))
|
||||||
|
assertEquals(5, this.consumers(0).committed(tp2))
|
||||||
|
|
||||||
|
// Using async should pick up the committed changes after commit completes
|
||||||
|
val commitCallback = new CountConsumerCommitCallback()
|
||||||
|
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 7L)), CommitType.ASYNC, commitCallback)
|
||||||
|
awaitCommitCallback(this.consumers(0), commitCallback)
|
||||||
|
assertEquals(7, this.consumers(0).committed(tp2))
|
||||||
}
|
}
|
||||||
|
|
||||||
def testAutoOffsetReset() {
|
def testAutoOffsetReset() {
|
||||||
|
@ -150,7 +188,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
|
||||||
|
|
||||||
def testPartitionReassignmentCallback() {
|
def testPartitionReassignmentCallback() {
|
||||||
val callback = new TestConsumerReassignmentCallback()
|
val callback = new TestConsumerReassignmentCallback()
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test
|
this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test
|
||||||
val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer())
|
val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer())
|
||||||
consumer0.subscribe(topic)
|
consumer0.subscribe(topic)
|
||||||
|
|
||||||
|
@ -172,6 +210,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
|
||||||
// this should cause another callback execution
|
// this should cause another callback execution
|
||||||
while(callback.callsToAssigned < 2)
|
while(callback.callsToAssigned < 2)
|
||||||
consumer0.poll(50)
|
consumer0.poll(50)
|
||||||
|
|
||||||
assertEquals(2, callback.callsToAssigned)
|
assertEquals(2, callback.callsToAssigned)
|
||||||
assertEquals(2, callback.callsToRevoked)
|
assertEquals(2, callback.callsToRevoked)
|
||||||
|
|
||||||
|
@ -191,9 +230,13 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def sendRecords(numRecords: Int) {
|
private def sendRecords(numRecords: Int): Unit = {
|
||||||
|
sendRecords(numRecords, tp)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def sendRecords(numRecords: Int, tp: TopicPartition) {
|
||||||
val futures = (0 until numRecords).map { i =>
|
val futures = (0 until numRecords).map { i =>
|
||||||
this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes))
|
this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), i.toString.getBytes, i.toString.getBytes))
|
||||||
}
|
}
|
||||||
futures.map(_.get)
|
futures.map(_.get)
|
||||||
}
|
}
|
||||||
|
@ -218,4 +261,18 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private def awaitCommitCallback(consumer: Consumer[Array[Byte], Array[Byte]], commitCallback: CountConsumerCommitCallback): Unit = {
|
||||||
|
val startCount = commitCallback.count
|
||||||
|
val started = System.currentTimeMillis()
|
||||||
|
while (commitCallback.count == startCount && System.currentTimeMillis() - started < 10000)
|
||||||
|
this.consumers(0).poll(10000)
|
||||||
|
assertEquals(startCount + 1, commitCallback.count)
|
||||||
|
}
|
||||||
|
|
||||||
|
private class CountConsumerCommitCallback extends ConsumerCommitCallback {
|
||||||
|
var count = 0
|
||||||
|
|
||||||
|
override def onComplete(offsets: util.Map[TopicPartition, lang.Long], exception: Exception): Unit = count += 1
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -96,7 +96,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testIncrementPartitions",
|
val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testIncrementPartitions",
|
||||||
2000,0).topicsMetadata
|
2000,0).topicsMetadata
|
||||||
val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1))
|
val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1))
|
||||||
val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata
|
val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata.sortBy(_.partitionId)
|
||||||
assertEquals(partitionDataForTopic1.size, 3)
|
assertEquals(partitionDataForTopic1.size, 3)
|
||||||
assertEquals(partitionDataForTopic1(1).partitionId, 1)
|
assertEquals(partitionDataForTopic1(1).partitionId, 1)
|
||||||
assertEquals(partitionDataForTopic1(2).partitionId, 2)
|
assertEquals(partitionDataForTopic1(2).partitionId, 2)
|
||||||
|
@ -121,7 +121,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testManualAssignmentOfReplicas",
|
val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testManualAssignmentOfReplicas",
|
||||||
2000,0).topicsMetadata
|
2000,0).topicsMetadata
|
||||||
val metaDataForTopic2 = metadata.filter(p => p.topic.equals(topic2))
|
val metaDataForTopic2 = metadata.filter(p => p.topic.equals(topic2))
|
||||||
val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata
|
val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata.sortBy(_.partitionId)
|
||||||
assertEquals(partitionDataForTopic2.size, 3)
|
assertEquals(partitionDataForTopic2.size, 3)
|
||||||
assertEquals(partitionDataForTopic2(1).partitionId, 1)
|
assertEquals(partitionDataForTopic2(1).partitionId, 1)
|
||||||
assertEquals(partitionDataForTopic2(2).partitionId, 2)
|
assertEquals(partitionDataForTopic2(2).partitionId, 2)
|
||||||
|
@ -146,12 +146,13 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
2000,0).topicsMetadata
|
2000,0).topicsMetadata
|
||||||
|
|
||||||
val metaDataForTopic3 = metadata.filter(p => p.topic.equals(topic3)).head
|
val metaDataForTopic3 = metadata.filter(p => p.topic.equals(topic3)).head
|
||||||
val partition1DataForTopic3 = metaDataForTopic3.partitionsMetadata(1)
|
val partitionsMetadataForTopic3 = metaDataForTopic3.partitionsMetadata.sortBy(_.partitionId)
|
||||||
val partition2DataForTopic3 = metaDataForTopic3.partitionsMetadata(2)
|
val partition1DataForTopic3 = partitionsMetadataForTopic3(1)
|
||||||
val partition3DataForTopic3 = metaDataForTopic3.partitionsMetadata(3)
|
val partition2DataForTopic3 = partitionsMetadataForTopic3(2)
|
||||||
val partition4DataForTopic3 = metaDataForTopic3.partitionsMetadata(4)
|
val partition3DataForTopic3 = partitionsMetadataForTopic3(3)
|
||||||
val partition5DataForTopic3 = metaDataForTopic3.partitionsMetadata(5)
|
val partition4DataForTopic3 = partitionsMetadataForTopic3(4)
|
||||||
val partition6DataForTopic3 = metaDataForTopic3.partitionsMetadata(6)
|
val partition5DataForTopic3 = partitionsMetadataForTopic3(5)
|
||||||
|
val partition6DataForTopic3 = partitionsMetadataForTopic3(6)
|
||||||
|
|
||||||
assertEquals(partition1DataForTopic3.replicas.size, 4)
|
assertEquals(partition1DataForTopic3.replicas.size, 4)
|
||||||
assertEquals(partition1DataForTopic3.replicas(0).id, 3)
|
assertEquals(partition1DataForTopic3.replicas(0).id, 3)
|
||||||
|
|
|
@ -24,7 +24,7 @@ import kafka.utils._
|
||||||
import kafka.log._
|
import kafka.log._
|
||||||
import kafka.zk.ZooKeeperTestHarness
|
import kafka.zk.ZooKeeperTestHarness
|
||||||
import kafka.utils.{Logging, ZkUtils, TestUtils}
|
import kafka.utils.{Logging, ZkUtils, TestUtils}
|
||||||
import kafka.common.{TopicExistsException, TopicAndPartition}
|
import kafka.common.{InvalidTopicException, TopicExistsException, TopicAndPartition}
|
||||||
import kafka.server.{KafkaServer, KafkaConfig}
|
import kafka.server.{KafkaServer, KafkaConfig}
|
||||||
import java.io.File
|
import java.io.File
|
||||||
import TestUtils._
|
import TestUtils._
|
||||||
|
@ -134,6 +134,20 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testTopicCreationWithCollision() {
|
||||||
|
val topic = "test.topic"
|
||||||
|
val collidingTopic = "test_topic"
|
||||||
|
TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3, 4))
|
||||||
|
// create the topic
|
||||||
|
AdminUtils.createTopic(zkClient, topic, 3, 1)
|
||||||
|
|
||||||
|
intercept[InvalidTopicException] {
|
||||||
|
// shouldn't be able to create a topic that collides
|
||||||
|
AdminUtils.createTopic(zkClient, collidingTopic, 3, 1)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private def getBrokersWithPartitionDir(servers: Iterable[KafkaServer], topic: String, partitionId: Int): Set[Int] = {
|
private def getBrokersWithPartitionDir(servers: Iterable[KafkaServer], topic: String, partitionId: Int): Set[Int] = {
|
||||||
servers.filter(server => new File(server.config.logDirs.head, topic + "-" + partitionId).exists)
|
servers.filter(server => new File(server.config.logDirs.head, topic + "-" + partitionId).exists)
|
||||||
.map(_.config.brokerId)
|
.map(_.config.brokerId)
|
||||||
|
|
|
@ -57,4 +57,45 @@ class TopicTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testTopicHasCollisionChars() = {
|
||||||
|
val falseTopics = List("start", "end", "middle", "many")
|
||||||
|
val trueTopics = List(
|
||||||
|
".start", "end.", "mid.dle", ".ma.ny.",
|
||||||
|
"_start", "end_", "mid_dle", "_ma_ny."
|
||||||
|
)
|
||||||
|
|
||||||
|
falseTopics.foreach( t =>
|
||||||
|
assertFalse(Topic.hasCollisionChars(t))
|
||||||
|
)
|
||||||
|
|
||||||
|
trueTopics.foreach( t =>
|
||||||
|
assertTrue(Topic.hasCollisionChars(t))
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testTopicHasCollision() = {
|
||||||
|
val periodFirstMiddleLastNone = List(".topic", "to.pic", "topic.", "topic")
|
||||||
|
val underscoreFirstMiddleLastNone = List("_topic", "to_pic", "topic_", "topic")
|
||||||
|
|
||||||
|
// Self
|
||||||
|
periodFirstMiddleLastNone.foreach { t =>
|
||||||
|
assertTrue(Topic.hasCollision(t, t))
|
||||||
|
}
|
||||||
|
underscoreFirstMiddleLastNone.foreach { t =>
|
||||||
|
assertTrue(Topic.hasCollision(t, t))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Same Position
|
||||||
|
periodFirstMiddleLastNone.zip(underscoreFirstMiddleLastNone).foreach { case (t1, t2) =>
|
||||||
|
assertTrue(Topic.hasCollision(t1, t2))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Different Position
|
||||||
|
periodFirstMiddleLastNone.zip(underscoreFirstMiddleLastNone.reverse).foreach { case (t1, t2) =>
|
||||||
|
assertFalse(Topic.hasCollision(t1, t2))
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -43,7 +43,7 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite {
|
||||||
type HeartbeatCallback = Short => Unit
|
type HeartbeatCallback = Short => Unit
|
||||||
|
|
||||||
val ConsumerMinSessionTimeout = 10
|
val ConsumerMinSessionTimeout = 10
|
||||||
val ConsumerMaxSessionTimeout = 30
|
val ConsumerMaxSessionTimeout = 100
|
||||||
val DefaultSessionTimeout = 20
|
val DefaultSessionTimeout = 20
|
||||||
var consumerCoordinator: ConsumerCoordinator = null
|
var consumerCoordinator: ConsumerCoordinator = null
|
||||||
var offsetManager : OffsetManager = null
|
var offsetManager : OffsetManager = null
|
||||||
|
@ -231,6 +231,30 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite {
|
||||||
assertEquals(Errors.NONE.code, heartbeatResult)
|
assertEquals(Errors.NONE.code, heartbeatResult)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testHeartbeatDuringRebalanceCausesIllegalGeneration() {
|
||||||
|
val groupId = "groupId"
|
||||||
|
val partitionAssignmentStrategy = "range"
|
||||||
|
|
||||||
|
// First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts)
|
||||||
|
val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy,
|
||||||
|
100, isCoordinatorForGroup = true)
|
||||||
|
val assignedConsumerId = joinGroupResult._2
|
||||||
|
val initialGenerationId = joinGroupResult._3
|
||||||
|
val joinGroupErrorCode = joinGroupResult._4
|
||||||
|
assertEquals(Errors.NONE.code, joinGroupErrorCode)
|
||||||
|
|
||||||
|
// Then join with a new consumer to trigger a rebalance
|
||||||
|
EasyMock.reset(offsetManager)
|
||||||
|
sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy,
|
||||||
|
DefaultSessionTimeout, isCoordinatorForGroup = true)
|
||||||
|
|
||||||
|
// We should be in the middle of a rebalance, so the heartbeat should return illegal generation
|
||||||
|
EasyMock.reset(offsetManager)
|
||||||
|
val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true)
|
||||||
|
assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult)
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testGenerationIdIncrementsOnRebalance() {
|
def testGenerationIdIncrementsOnRebalance() {
|
||||||
val groupId = "groupId"
|
val groupId = "groupId"
|
||||||
|
@ -267,16 +291,25 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite {
|
||||||
(responseFuture, responseCallback)
|
(responseFuture, responseCallback)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def joinGroup(groupId: String,
|
private def sendJoinGroup(groupId: String,
|
||||||
consumerId: String,
|
consumerId: String,
|
||||||
partitionAssignmentStrategy: String,
|
partitionAssignmentStrategy: String,
|
||||||
sessionTimeout: Int,
|
sessionTimeout: Int,
|
||||||
isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = {
|
isCoordinatorForGroup: Boolean): Future[JoinGroupCallbackParams] = {
|
||||||
val (responseFuture, responseCallback) = setupJoinGroupCallback
|
val (responseFuture, responseCallback) = setupJoinGroupCallback
|
||||||
EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
|
EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
|
||||||
EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
|
EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
|
||||||
EasyMock.replay(offsetManager)
|
EasyMock.replay(offsetManager)
|
||||||
consumerCoordinator.handleJoinGroup(groupId, consumerId, Set.empty, sessionTimeout, partitionAssignmentStrategy, responseCallback)
|
consumerCoordinator.handleJoinGroup(groupId, consumerId, Set.empty, sessionTimeout, partitionAssignmentStrategy, responseCallback)
|
||||||
|
responseFuture
|
||||||
|
}
|
||||||
|
|
||||||
|
private def joinGroup(groupId: String,
|
||||||
|
consumerId: String,
|
||||||
|
partitionAssignmentStrategy: String,
|
||||||
|
sessionTimeout: Int,
|
||||||
|
isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = {
|
||||||
|
val responseFuture = sendJoinGroup(groupId, consumerId, partitionAssignmentStrategy, sessionTimeout, isCoordinatorForGroup)
|
||||||
Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
|
Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -25,7 +25,7 @@ import kafka.api.{TopicMetadataResponse, TopicMetadataRequest}
|
||||||
import kafka.client.ClientUtils
|
import kafka.client.ClientUtils
|
||||||
import kafka.cluster.{Broker, BrokerEndPoint}
|
import kafka.cluster.{Broker, BrokerEndPoint}
|
||||||
import kafka.common.ErrorMapping
|
import kafka.common.ErrorMapping
|
||||||
import kafka.server.{NotRunning, KafkaConfig, KafkaServer}
|
import kafka.server.{KafkaConfig, KafkaServer, NotRunning}
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import kafka.utils.TestUtils._
|
import kafka.utils.TestUtils._
|
||||||
import kafka.zk.ZooKeeperTestHarness
|
import kafka.zk.ZooKeeperTestHarness
|
||||||
|
@ -36,7 +36,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
private var server1: KafkaServer = null
|
private var server1: KafkaServer = null
|
||||||
var brokerEndPoints: Seq[BrokerEndPoint] = null
|
var brokerEndPoints: Seq[BrokerEndPoint] = null
|
||||||
var adHocConfigs: Seq[KafkaConfig] = null
|
var adHocConfigs: Seq[KafkaConfig] = null
|
||||||
val numConfigs: Int = 2
|
val numConfigs: Int = 4
|
||||||
|
|
||||||
override def setUp() {
|
override def setUp() {
|
||||||
super.setUp()
|
super.setUp()
|
||||||
|
@ -171,13 +171,15 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
|
|
||||||
|
|
||||||
def testIsrAfterBrokerShutDownAndJoinsBack {
|
def testIsrAfterBrokerShutDownAndJoinsBack {
|
||||||
|
val numBrokers = 2 //just 2 brokers are enough for the test
|
||||||
|
|
||||||
// start adHoc brokers
|
// start adHoc brokers
|
||||||
val adHocServers = adHocConfigs.map(p => createServer(p))
|
val adHocServers = adHocConfigs.take(numBrokers - 1).map(p => createServer(p))
|
||||||
val allServers: Seq[KafkaServer] = Seq(server1) ++ adHocServers
|
val allServers: Seq[KafkaServer] = Seq(server1) ++ adHocServers
|
||||||
|
|
||||||
// create topic
|
// create topic
|
||||||
val topic: String = "test"
|
val topic: String = "test"
|
||||||
AdminUtils.createTopic(zkClient, topic, 1, numConfigs)
|
AdminUtils.createTopic(zkClient, topic, 1, numBrokers)
|
||||||
|
|
||||||
// shutdown a broker
|
// shutdown a broker
|
||||||
adHocServers.last.shutdown()
|
adHocServers.last.shutdown()
|
||||||
|
@ -192,4 +194,60 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
// shutdown adHoc brokers
|
// shutdown adHoc brokers
|
||||||
adHocServers.map(p => p.shutdown())
|
adHocServers.map(p => p.shutdown())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private def checkMetadata(servers: Seq[KafkaServer], expectedBrokersCount: Int): Unit = {
|
||||||
|
var topicMetadata: TopicMetadataResponse = new TopicMetadataResponse(Seq(), Seq(), -1)
|
||||||
|
|
||||||
|
// Get topic metadata from old broker
|
||||||
|
// Wait for metadata to get updated by checking metadata from a new broker
|
||||||
|
waitUntilTrue(() => {
|
||||||
|
topicMetadata = ClientUtils.fetchTopicMetadata(
|
||||||
|
Set.empty, brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000, 0)
|
||||||
|
topicMetadata.brokers.size == expectedBrokersCount},
|
||||||
|
"Alive brokers list is not correctly propagated by coordinator to brokers"
|
||||||
|
)
|
||||||
|
|
||||||
|
// Assert that topic metadata at new brokers is updated correctly
|
||||||
|
servers.filter(x => x.brokerState.currentState != NotRunning.state).foreach(x =>
|
||||||
|
waitUntilTrue(() =>
|
||||||
|
topicMetadata == ClientUtils.fetchTopicMetadata(
|
||||||
|
Set.empty,
|
||||||
|
Seq(new Broker(x.config.brokerId,
|
||||||
|
x.config.hostName,
|
||||||
|
x.boundPort()).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)),
|
||||||
|
"TopicMetadataTest-testBasicTopicMetadata",
|
||||||
|
2000, 0), "Topic metadata is not correctly updated"))
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def testAliveBrokerListWithNoTopics {
|
||||||
|
checkMetadata(Seq(server1), 1)
|
||||||
|
}
|
||||||
|
|
||||||
|
def testAliveBrokersListWithNoTopicsAfterNewBrokerStartup {
|
||||||
|
var adHocServers = adHocConfigs.takeRight(adHocConfigs.size - 1).map(p => createServer(p))
|
||||||
|
|
||||||
|
checkMetadata(adHocServers, numConfigs - 1)
|
||||||
|
|
||||||
|
// Add a broker
|
||||||
|
adHocServers = adHocServers ++ Seq(createServer(adHocConfigs.head))
|
||||||
|
|
||||||
|
checkMetadata(adHocServers, numConfigs)
|
||||||
|
adHocServers.map(p => p.shutdown())
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def testAliveBrokersListWithNoTopicsAfterABrokerShutdown {
|
||||||
|
val adHocServers = adHocConfigs.map(p => createServer(p))
|
||||||
|
|
||||||
|
checkMetadata(adHocServers, numConfigs)
|
||||||
|
|
||||||
|
// Shutdown a broker
|
||||||
|
adHocServers.last.shutdown()
|
||||||
|
adHocServers.last.awaitShutdown()
|
||||||
|
|
||||||
|
checkMetadata(adHocServers, numConfigs - 1)
|
||||||
|
|
||||||
|
adHocServers.map(p => p.shutdown())
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -134,7 +134,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite {
|
||||||
Assert.assertEquals(expectedConfig.compressionType, actualConfig.compressionType)
|
Assert.assertEquals(expectedConfig.compressionType, actualConfig.compressionType)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def atLeastXIntProp(x: Int): String = (nextInt(Int.MaxValue - 1) + x).toString
|
private def atLeastXIntProp(x: Int): String = (nextInt(Int.MaxValue - x) + x).toString
|
||||||
|
|
||||||
private def atLeastOneIntProp: String = atLeastXIntProp(1)
|
private def atLeastOneIntProp: String = atLeastXIntProp(1)
|
||||||
|
|
||||||
|
@ -157,9 +157,9 @@ class KafkaConfigConfigDefTest extends JUnit3Suite {
|
||||||
case KafkaConfig.QueuedMaxRequestsProp => expected.setProperty(name, atLeastOneIntProp)
|
case KafkaConfig.QueuedMaxRequestsProp => expected.setProperty(name, atLeastOneIntProp)
|
||||||
|
|
||||||
case KafkaConfig.PortProp => expected.setProperty(name, "1234")
|
case KafkaConfig.PortProp => expected.setProperty(name, "1234")
|
||||||
case KafkaConfig.HostNameProp => expected.setProperty(name, nextString(10))
|
case KafkaConfig.HostNameProp => expected.setProperty(name, "hostname")
|
||||||
case KafkaConfig.ListenersProp => expected.setProperty(name, "PLAINTEXT://:9092")
|
case KafkaConfig.ListenersProp => expected.setProperty(name, "PLAINTEXT://:9092")
|
||||||
case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, nextString(10))
|
case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, "advertised.hostname")
|
||||||
case KafkaConfig.AdvertisedPortProp => expected.setProperty(name, "4321")
|
case KafkaConfig.AdvertisedPortProp => expected.setProperty(name, "4321")
|
||||||
case KafkaConfig.AdvertisedListenersProp => expected.setProperty(name, "PLAINTEXT://:2909")
|
case KafkaConfig.AdvertisedListenersProp => expected.setProperty(name, "PLAINTEXT://:2909")
|
||||||
case KafkaConfig.SocketRequestMaxBytesProp => expected.setProperty(name, atLeastOneIntProp)
|
case KafkaConfig.SocketRequestMaxBytesProp => expected.setProperty(name, atLeastOneIntProp)
|
||||||
|
@ -225,7 +225,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite {
|
||||||
case KafkaConfig.ReplicaLagTimeMaxMsProp => expected.setProperty(name, atLeastXIntProp(321))
|
case KafkaConfig.ReplicaLagTimeMaxMsProp => expected.setProperty(name, atLeastXIntProp(321))
|
||||||
//require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor)
|
//require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor)
|
||||||
case KafkaConfig.OffsetCommitRequiredAcksProp => expected.setProperty(name, "-1")
|
case KafkaConfig.OffsetCommitRequiredAcksProp => expected.setProperty(name, "-1")
|
||||||
case KafkaConfig.OffsetsTopicReplicationFactorProp => expected.setProperty(name, inRangeIntProp(-1, Short.MaxValue))
|
case KafkaConfig.OffsetsTopicReplicationFactorProp => expected.setProperty(name, inRangeIntProp(1, Short.MaxValue))
|
||||||
//BrokerCompressionCodec.isValid(compressionType)
|
//BrokerCompressionCodec.isValid(compressionType)
|
||||||
case KafkaConfig.CompressionTypeProp => expected.setProperty(name, randFrom(BrokerCompressionCodec.brokerCompressionOptions))
|
case KafkaConfig.CompressionTypeProp => expected.setProperty(name, randFrom(BrokerCompressionCodec.brokerCompressionOptions))
|
||||||
|
|
||||||
|
|
|
@ -158,6 +158,7 @@ object TestUtils extends Logging {
|
||||||
props.put("controlled.shutdown.enable", enableControlledShutdown.toString)
|
props.put("controlled.shutdown.enable", enableControlledShutdown.toString)
|
||||||
props.put("delete.topic.enable", enableDeleteTopic.toString)
|
props.put("delete.topic.enable", enableDeleteTopic.toString)
|
||||||
props.put("controlled.shutdown.retry.backoff.ms", "100")
|
props.put("controlled.shutdown.retry.backoff.ms", "100")
|
||||||
|
props.put("port", port.toString)
|
||||||
props
|
props
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -95,6 +95,7 @@ def continue_maybe(prompt):
|
||||||
fail("Okay, exiting")
|
fail("Okay, exiting")
|
||||||
|
|
||||||
def clean_up():
|
def clean_up():
|
||||||
|
if original_head != get_current_branch():
|
||||||
print "Restoring head pointer to %s" % original_head
|
print "Restoring head pointer to %s" % original_head
|
||||||
run_cmd("git checkout %s" % original_head)
|
run_cmd("git checkout %s" % original_head)
|
||||||
|
|
||||||
|
@ -104,6 +105,8 @@ def clean_up():
|
||||||
print "Deleting local branch %s" % branch
|
print "Deleting local branch %s" % branch
|
||||||
run_cmd("git branch -D %s" % branch)
|
run_cmd("git branch -D %s" % branch)
|
||||||
|
|
||||||
|
def get_current_branch():
|
||||||
|
return run_cmd("git rev-parse --abbrev-ref HEAD").replace("\n", "")
|
||||||
|
|
||||||
# merge the requested PR and return the merge hash
|
# merge the requested PR and return the merge hash
|
||||||
def merge_pr(pr_num, target_ref, title, body, pr_repo_desc):
|
def merge_pr(pr_num, target_ref, title, body, pr_repo_desc):
|
||||||
|
@ -350,7 +353,7 @@ def standardize_jira_ref(text):
|
||||||
def main():
|
def main():
|
||||||
global original_head
|
global original_head
|
||||||
|
|
||||||
original_head = run_cmd("git rev-parse HEAD")[:8]
|
original_head = get_current_branch()
|
||||||
|
|
||||||
branches = get_json("%s/branches" % GITHUB_API_BASE)
|
branches = get_json("%s/branches" % GITHUB_API_BASE)
|
||||||
branch_names = filter(lambda x: x.startswith(RELEASE_BRANCH_PREFIX), [x['name'] for x in branches])
|
branch_names = filter(lambda x: x.startswith(RELEASE_BRANCH_PREFIX), [x['name'] for x in branches])
|
||||||
|
|
Loading…
Reference in New Issue