diff --git a/automq-shell/src/main/java/com/automq/shell/util/CLIUtils.java b/automq-shell/src/main/java/com/automq/shell/util/CLIUtils.java
deleted file mode 100644
index f6609d31c3..0000000000
--- a/automq-shell/src/main/java/com/automq/shell/util/CLIUtils.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Copyright 2024, AutoMQ HK Limited.
- *
- * The use of this file is governed by the Business Source License,
- * as detailed in the file "/LICENSE.S3Stream" included in this repository.
- *
- * As of the Change Date specified in that file, in accordance with
- * the Business Source License, use of this software will be governed
- * by the Apache License, Version 2.0
- */
-
-package com.automq.shell.util;
-
-import java.util.Collections;
-import org.apache.kafka.clients.ApiVersions;
-import org.apache.kafka.clients.ClientUtils;
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.ManualMetadataUpdater;
-import org.apache.kafka.clients.NetworkClient;
-import org.apache.kafka.clients.admin.AdminClientConfig;
-import org.apache.kafka.common.metrics.Metrics;
-import org.apache.kafka.common.network.ChannelBuilder;
-import org.apache.kafka.common.network.NetworkReceive;
-import org.apache.kafka.common.network.Selector;
-import org.apache.kafka.common.utils.LogContext;
-import org.apache.kafka.common.utils.Time;
-
-public class CLIUtils {
- public static NetworkClient buildNetworkClient(
- String prefix,
- AdminClientConfig config,
- Metrics metrics,
- Time time,
- LogContext logContext) {
- ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext);
-
- String metricGroupPrefix = prefix + "-channel";
-
- Selector selector = new Selector(
- NetworkReceive.UNLIMITED,
- config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG),
- metrics,
- time,
- metricGroupPrefix,
- Collections.emptyMap(),
- false,
- channelBuilder,
- logContext
- );
-
- String clientId = prefix + "-network-client";
- return new NetworkClient(
- selector,
- new ManualMetadataUpdater(),
- clientId,
- 100,
- config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG),
- config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG),
- config.getInt(CommonClientConfigs.SEND_BUFFER_CONFIG),
- config.getInt(CommonClientConfigs.RECEIVE_BUFFER_CONFIG),
- config.getInt(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG),
- config.getLong(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG),
- config.getLong(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG),
- time,
- false,
- new ApiVersions(),
- logContext
- );
- }
-}
diff --git a/automq-shell/src/main/java/com/automq/shell/util/S3PropUtil.java b/automq-shell/src/main/java/com/automq/shell/util/S3PropUtil.java
deleted file mode 100644
index 6b277273c2..0000000000
--- a/automq-shell/src/main/java/com/automq/shell/util/S3PropUtil.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Copyright 2024, AutoMQ HK Limited.
- *
- * The use of this file is governed by the Business Source License,
- * as detailed in the file "/LICENSE.S3Stream" included in this repository.
- *
- * As of the Change Date specified in that file, in accordance with
- * the Business Source License, use of this software will be governed
- * by the Apache License, Version 2.0
- */
-package com.automq.shell.util;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.nio.charset.Charset;
-import java.util.Enumeration;
-import java.util.Properties;
-
-public class S3PropUtil {
- public static final String BROKER_PROPS_PATH = "template/broker.properties";
- public static final String CONTROLLER_PROPS_PATH = "template/controller.properties";
- public static final String SERVER_PROPS_PATH = "template/server.properties";
-
- public static void persist(Properties props, String fileName) throws IOException {
- File directory = new File("generated");
- if (!directory.exists() && !directory.mkdirs()) {
- throw new IOException("Can't create directory " + directory.getAbsolutePath());
- }
-
- String targetPath = "generated/" + fileName;
- File file = new File(targetPath);
- try (PrintWriter pw = new PrintWriter(file, Charset.forName("utf-8"))) {
- for (Enumeration e = props.propertyNames(); e.hasMoreElements(); ) {
- String key = (String) e.nextElement();
- pw.println(key + "=" + props.getProperty(key));
- }
- }
- }
-
- public static Properties loadTemplateProps(String propsPath) throws IOException {
- try (var in = S3PropUtil.class.getClassLoader().getResourceAsStream(propsPath)) {
- if (in != null) {
- Properties props = new Properties();
- props.load(in);
- return props;
- } else {
- throw new IOException(String.format("Can not find resource file under path: %s", propsPath));
- }
- }
- }
-}
diff --git a/build.gradle b/build.gradle
index 6248e4025e..1cb318afd2 100644
--- a/build.gradle
+++ b/build.gradle
@@ -937,6 +937,7 @@ project(':core') {
api libs.scalaLibrary
implementation project(':server-common')
+ implementation project(':group-coordinator:group-coordinator-api')
implementation project(':group-coordinator')
implementation project(':transaction-coordinator')
implementation project(':metadata')
@@ -1367,6 +1368,66 @@ project(':metadata') {
}
}
+project(':group-coordinator:group-coordinator-api') {
+ base {
+ archivesName = "kafka-group-coordinator-api"
+ }
+
+ dependencies {
+ implementation project(':clients')
+ }
+
+ task createVersionFile() {
+ def receiptFile = file("$buildDir/kafka/$buildVersionFileName")
+ inputs.property "commitId", commitId
+ inputs.property "version", version
+ outputs.file receiptFile
+
+ doLast {
+ def data = [
+ commitId: commitId,
+ version: version,
+ ]
+
+ receiptFile.parentFile.mkdirs()
+ def content = data.entrySet().collect { "$it.key=$it.value" }.sort().join("\n")
+ receiptFile.setText(content, "ISO-8859-1")
+ }
+ }
+
+ sourceSets {
+ main {
+ java {
+ srcDirs = ["src/main/java"]
+ }
+ }
+ test {
+ java {
+ srcDirs = ["src/test/java"]
+ }
+ }
+ }
+
+ jar {
+ dependsOn createVersionFile
+ from("$buildDir") {
+ include "kafka/$buildVersionFileName"
+ }
+ }
+
+ clean.doFirst {
+ delete "$buildDir/kafka/"
+ }
+
+ javadoc {
+ include "**/org/apache/kafka/coordinator/group/api/**"
+ }
+
+ checkstyle {
+ configProperties = checkstyleConfigProperties("import-control-group-coordinator.xml")
+ }
+}
+
project(':group-coordinator') {
base {
archivesName = "kafka-group-coordinator"
@@ -1380,6 +1441,7 @@ project(':group-coordinator') {
implementation project(':server-common')
implementation project(':clients')
implementation project(':metadata')
+ implementation project(':group-coordinator:group-coordinator-api')
implementation project(':storage')
implementation libs.jacksonDatabind
implementation libs.jacksonJDK8Datatypes
@@ -2597,10 +2659,6 @@ project(':streams:test-utils') {
testRuntimeOnly libs.slf4jlog4j
}
- javadoc {
- include "**/org/apache/kafka/streams/test/**"
- }
-
tasks.create(name: "copyDependantLibs", type: Copy) {
from (configurations.runtimeClasspath) {
exclude('kafka-streams*')
@@ -3023,6 +3081,7 @@ project(':jmh-benchmarks') {
implementation project(':raft')
implementation project(':clients')
implementation project(':group-coordinator')
+ implementation project(':group-coordinator:group-coordinator-api')
implementation project(':metadata')
implementation project(':storage')
implementation project(':streams')
diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml
index ed6c53a322..a30de55e41 100644
--- a/checkstyle/import-control-core.xml
+++ b/checkstyle/import-control-core.xml
@@ -38,6 +38,7 @@
+
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 71c8e0dd4b..a663195985 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -443,6 +443,7 @@
+
diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index ce25c63f28..fe1c805176 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -202,7 +202,7 @@
files="StreamThread.java"/>
+ files="(InternalTopologyBuilder|KafkaStreams|KStreamImpl|KTableImpl|StreamsPartitionAssignor).java"/>
@@ -211,7 +211,7 @@
files="StreamsMetricsImpl.java"/>
+ files="(KafkaStreams|StreamsPartitionAssignor|StreamThread|TaskManager|TaskAssignmentUtils|GlobalStateManagerImpl|KStreamImplJoin|TopologyConfig|KTableKTableOuterJoin).java"/>
diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
index 6b6b56059c..b08bc12950 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
@@ -245,7 +245,9 @@ public static NetworkClient createNetworkClient(AbstractConfig config,
throttleTimeSensor,
logContext,
hostResolver,
- clientTelemetrySender);
+ clientTelemetrySender,
+ MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG))
+ );
} catch (Throwable t) {
closeQuietly(selector, "Selector");
closeQuietly(channelBuilder, "ChannelBuilder");
diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
index 6f5be8a006..40b806c4e9 100644
--- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
+++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
@@ -220,6 +220,19 @@ public class CommonClientConfigs {
public static final String DEFAULT_API_TIMEOUT_MS_DOC = "Specifies the timeout (in milliseconds) for client APIs. " +
"This configuration is used as the default timeout for all client operations that do not specify a timeout
parameter.";
+ public static final String METADATA_RECOVERY_STRATEGY_CONFIG = "metadata.recovery.strategy";
+ public static final String METADATA_RECOVERY_STRATEGY_DOC = "Controls how the client recovers when none of the brokers known to it is available. " +
+ "If set to none
, the client fails. If set to rebootstrap
, " +
+ "the client repeats the bootstrap process using bootstrap.servers
. " +
+ "Rebootstrapping is useful when a client communicates with brokers so infrequently " +
+ "that the set of brokers may change entirely before the client refreshes metadata. " +
+ "Metadata recovery is triggered when all last-known brokers appear unavailable simultaneously. " +
+ "Brokers appear unavailable when disconnected and no current retry attempt is in-progress. " +
+ "Consider increasing reconnect.backoff.ms
and reconnect.backoff.max.ms
and " +
+ "decreasing socket.connection.setup.timeout.ms
and socket.connection.setup.timeout.max.ms
" +
+ "for the client.";
+ public static final String DEFAULT_METADATA_RECOVERY_STRATEGY = MetadataRecoveryStrategy.NONE.name;
+
/**
* Postprocess the configuration so that exponential backoff is disabled when reconnect backoff
* is explicitly configured but the maximum reconnect backoff is not explicitly configured.
diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java
index a03d57b40f..46b6498606 100644
--- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java
@@ -130,7 +130,7 @@ public interface KafkaClient extends Closeable {
* @param now The current time in ms
* @return The node with the fewest in-flight requests.
*/
- Node leastLoadedNode(long now);
+ LeastLoadedNode leastLoadedNode(long now);
/**
* The number of currently in-flight requests for which we have not yet returned a response
diff --git a/clients/src/main/java/org/apache/kafka/clients/LeastLoadedNode.java b/clients/src/main/java/org/apache/kafka/clients/LeastLoadedNode.java
new file mode 100644
index 0000000000..b2b93e6c94
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/LeastLoadedNode.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients;
+
+import org.apache.kafka.common.Node;
+
+public class LeastLoadedNode {
+ private final Node node;
+ private final boolean atLeastOneConnectionReady;
+
+ public LeastLoadedNode(Node node, boolean atLeastOneConnectionReady) {
+ this.node = node;
+ this.atLeastOneConnectionReady = atLeastOneConnectionReady;
+ }
+
+ public Node node() {
+ return node;
+ }
+
+ /**
+ * Indicates if the least loaded node is available or at least a ready connection exists.
+ *
+ *
There may be no node available while ready connections to live nodes exist. This may happen when
+ * the connections are overloaded with in-flight requests. This function takes this into account.
+ */
+ public boolean hasNodeAvailableOrConnectionReady() {
+ return node != null || atLeastOneConnectionReady;
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
index 30cad44a4b..9246da0100 100644
--- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
@@ -82,6 +82,8 @@ public class Metadata implements Closeable {
private final ClusterResourceListeners clusterResourceListeners;
private boolean isClosed;
private final Map lastSeenLeaderEpochs;
+ /** Addresses with which the metadata was originally bootstrapped. */
+ private List bootstrapAddresses;
/**
* Create a new Metadata instance
@@ -304,6 +306,12 @@ public synchronized void bootstrap(List addresses) {
this.needFullUpdate = true;
this.updateVersion += 1;
this.metadataSnapshot = MetadataSnapshot.bootstrap(addresses);
+ this.bootstrapAddresses = addresses;
+ }
+
+ public synchronized void rebootstrap() {
+ log.info("Rebootstrapping with {}", this.bootstrapAddresses);
+ this.bootstrap(this.bootstrapAddresses);
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataRecoveryStrategy.java b/clients/src/main/java/org/apache/kafka/clients/MetadataRecoveryStrategy.java
new file mode 100644
index 0000000000..a4e0340c24
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/MetadataRecoveryStrategy.java
@@ -0,0 +1,44 @@
+/*
+ * 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;
+
+import java.util.Locale;
+
+/**
+ * Defines the strategies which clients can follow to deal with the situation when none of the known nodes is available.
+ */
+public enum MetadataRecoveryStrategy {
+ NONE("none"),
+ REBOOTSTRAP("rebootstrap");
+
+ public final String name;
+
+ MetadataRecoveryStrategy(String name) {
+ this.name = name;
+ }
+
+ public static MetadataRecoveryStrategy forName(String name) {
+ if (name == null) {
+ throw new IllegalArgumentException("Illegal MetadataRecoveryStrategy: null");
+ }
+ try {
+ return MetadataRecoveryStrategy.valueOf(name.toUpperCase(Locale.ROOT));
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException("Illegal MetadataRecoveryStrategy: " + name);
+ }
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
index 3a7af6617e..8ac92acd88 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -114,6 +114,8 @@ private enum State {
/* time in ms to wait before retrying to create connection to a server */
private final long reconnectBackoffMs;
+ private final MetadataRecoveryStrategy metadataRecoveryStrategy;
+
private final Time time;
/**
@@ -147,7 +149,8 @@ public NetworkClient(Selectable selector,
Time time,
boolean discoverBrokerVersions,
ApiVersions apiVersions,
- LogContext logContext) {
+ LogContext logContext,
+ MetadataRecoveryStrategy metadataRecoveryStrategy) {
this(selector,
metadata,
clientId,
@@ -163,7 +166,8 @@ public NetworkClient(Selectable selector,
discoverBrokerVersions,
apiVersions,
null,
- logContext);
+ logContext,
+ metadataRecoveryStrategy);
}
public NetworkClient(Selectable selector,
@@ -181,7 +185,8 @@ public NetworkClient(Selectable selector,
boolean discoverBrokerVersions,
ApiVersions apiVersions,
Sensor throttleTimeSensor,
- LogContext logContext) {
+ LogContext logContext,
+ MetadataRecoveryStrategy metadataRecoveryStrategy) {
this(null,
metadata,
selector,
@@ -200,7 +205,8 @@ public NetworkClient(Selectable selector,
throttleTimeSensor,
logContext,
new DefaultHostResolver(),
- null);
+ null,
+ metadataRecoveryStrategy);
}
public NetworkClient(Selectable selector,
@@ -217,7 +223,8 @@ public NetworkClient(Selectable selector,
Time time,
boolean discoverBrokerVersions,
ApiVersions apiVersions,
- LogContext logContext) {
+ LogContext logContext,
+ MetadataRecoveryStrategy metadataRecoveryStrategy) {
this(metadataUpdater,
null,
selector,
@@ -236,7 +243,8 @@ public NetworkClient(Selectable selector,
null,
logContext,
new DefaultHostResolver(),
- null);
+ null,
+ metadataRecoveryStrategy);
}
public NetworkClient(MetadataUpdater metadataUpdater,
@@ -257,7 +265,8 @@ public NetworkClient(MetadataUpdater metadataUpdater,
Sensor throttleTimeSensor,
LogContext logContext,
HostResolver hostResolver,
- ClientTelemetrySender clientTelemetrySender) {
+ ClientTelemetrySender clientTelemetrySender,
+ MetadataRecoveryStrategy metadataRecoveryStrategy) {
/* It would be better if we could pass `DefaultMetadataUpdater` from the public constructor, but it's not
* possible because `DefaultMetadataUpdater` is an inner class and it can only be instantiated after the
* super constructor is invoked.
@@ -288,6 +297,7 @@ public NetworkClient(MetadataUpdater metadataUpdater,
this.log = logContext.logger(NetworkClient.class);
this.state = new AtomicReference<>(State.ACTIVE);
this.telemetrySender = (clientTelemetrySender != null) ? new TelemetrySender(clientTelemetrySender) : null;
+ this.metadataRecoveryStrategy = metadataRecoveryStrategy;
}
/**
@@ -695,7 +705,7 @@ public void close() {
* @return The node with the fewest in-flight requests.
*/
@Override
- public Node leastLoadedNode(long now) {
+ public LeastLoadedNode leastLoadedNode(long now) {
List nodes = this.metadataUpdater.fetchNodes();
if (nodes.isEmpty())
throw new IllegalStateException("There are no nodes in the Kafka cluster");
@@ -705,16 +715,25 @@ public Node leastLoadedNode(long now) {
Node foundCanConnect = null;
Node foundReady = null;
+ boolean atLeastOneConnectionReady = false;
+
int offset = this.randOffset.nextInt(nodes.size());
for (int i = 0; i < nodes.size(); i++) {
int idx = (offset + i) % nodes.size();
Node node = nodes.get(idx);
+
+ if (!atLeastOneConnectionReady
+ && connectionStates.isReady(node.idString(), now)
+ && selector.isChannelReady(node.idString())) {
+ atLeastOneConnectionReady = true;
+ }
+
if (canSendRequest(node.idString(), now)) {
int currInflight = this.inFlightRequests.count(node.idString());
if (currInflight == 0) {
// if we find an established connection with no in-flight requests we can stop right away
log.trace("Found least loaded node {} connected with no in-flight requests", node);
- return node;
+ return new LeastLoadedNode(node, true);
} else if (currInflight < inflight) {
// otherwise if this is the best we have found so far, record that
inflight = currInflight;
@@ -738,16 +757,16 @@ public Node leastLoadedNode(long now) {
// which are being established before connecting to new nodes.
if (foundReady != null) {
log.trace("Found least loaded node {} with {} inflight requests", foundReady, inflight);
- return foundReady;
+ return new LeastLoadedNode(foundReady, atLeastOneConnectionReady);
} else if (foundConnecting != null) {
log.trace("Found least loaded connecting node {}", foundConnecting);
- return foundConnecting;
+ return new LeastLoadedNode(foundConnecting, atLeastOneConnectionReady);
} else if (foundCanConnect != null) {
log.trace("Found least loaded node {} with no active connection", foundCanConnect);
- return foundCanConnect;
+ return new LeastLoadedNode(foundCanConnect, atLeastOneConnectionReady);
} else {
log.trace("Least loaded node selection failed to find an available node");
- return null;
+ return new LeastLoadedNode(null, atLeastOneConnectionReady);
}
}
@@ -1122,13 +1141,22 @@ public long maybeUpdate(long now) {
// Beware that the behavior of this method and the computation of timeouts for poll() are
// highly dependent on the behavior of leastLoadedNode.
- Node node = leastLoadedNode(now);
- if (node == null) {
+ LeastLoadedNode leastLoadedNode = leastLoadedNode(now);
+
+ // Rebootstrap if needed and configured.
+ if (metadataRecoveryStrategy == MetadataRecoveryStrategy.REBOOTSTRAP
+ && !leastLoadedNode.hasNodeAvailableOrConnectionReady()) {
+ metadata.rebootstrap();
+
+ leastLoadedNode = leastLoadedNode(now);
+ }
+
+ if (leastLoadedNode.node() == null) {
log.debug("Give up sending metadata request since no node is available");
return reconnectBackoffMs;
}
- return maybeUpdate(now, node);
+ return maybeUpdate(now, leastLoadedNode.node());
}
@Override
@@ -1266,7 +1294,7 @@ public long maybeUpdate(long now) {
// Per KIP-714, let's continue to re-use the same broker for as long as possible.
if (stickyNode == null) {
- stickyNode = leastLoadedNode(now);
+ stickyNode = leastLoadedNode(now).node();
if (stickyNode == null) {
log.debug("Give up sending telemetry request since no node is available");
return reconnectBackoffMs;
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
index 38b3355fd9..2f73b8c1ac 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
@@ -911,7 +911,7 @@ default DescribeConsumerGroupsResult describeConsumerGroups(Collection g
* List the consumer groups available in the cluster.
*
* @param options The options to use when listing the consumer groups.
- * @return The ListGroupsResult.
+ * @return The ListConsumerGroupsResult.
*/
ListConsumerGroupsResult listConsumerGroups(ListConsumerGroupsOptions options);
@@ -921,7 +921,7 @@ default DescribeConsumerGroupsResult describeConsumerGroups(Collection g
* This is a convenience method for {@link #listConsumerGroups(ListConsumerGroupsOptions)} with default options.
* See the overload for more details.
*
- * @return The ListGroupsResult.
+ * @return The ListConsumerGroupsResult.
*/
default ListConsumerGroupsResult listConsumerGroups() {
return listConsumerGroups(new ListConsumerGroupsOptions());
@@ -931,7 +931,7 @@ default ListConsumerGroupsResult listConsumerGroups() {
* List the consumer group offsets available in the cluster.
*
* @param options The options to use when listing the consumer group offsets.
- * @return The ListGroupOffsetsResult
+ * @return The ListConsumerGroupOffsetsResult
*/
default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) {
@SuppressWarnings("deprecation")
@@ -949,7 +949,7 @@ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId,
* This is a convenience method for {@link #listConsumerGroupOffsets(Map, ListConsumerGroupOffsetsOptions)}
* to list offsets of all partitions of one group with default options.
*
- * @return The ListGroupOffsetsResult.
+ * @return The ListConsumerGroupOffsetsResult.
*/
default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId) {
return listConsumerGroupOffsets(groupId, new ListConsumerGroupOffsetsOptions());
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java
index d740283f50..b64338b1d1 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java
@@ -19,6 +19,7 @@
import org.apache.kafka.clients.ClientDnsLookup;
import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
@@ -139,6 +140,10 @@ public class AdminClientConfig extends AbstractConfig {
public static final String RETRIES_CONFIG = CommonClientConfigs.RETRIES_CONFIG;
public static final String DEFAULT_API_TIMEOUT_MS_CONFIG = CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_CONFIG;
+ public static final String METADATA_RECOVERY_STRATEGY_CONFIG = CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG;
+ public static final String METADATA_RECOVERY_STRATEGY_DOC = CommonClientConfigs.METADATA_RECOVERY_STRATEGY_DOC;
+ public static final String DEFAULT_METADATA_RECOVERY_STRATEGY = CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY;
+
/**
* security.providers
*/
@@ -262,7 +267,14 @@ public class AdminClientConfig extends AbstractConfig {
Importance.MEDIUM,
SECURITY_PROTOCOL_DOC)
.withClientSslSupport()
- .withClientSaslSupport();
+ .withClientSaslSupport()
+ .define(METADATA_RECOVERY_STRATEGY_CONFIG,
+ Type.STRING,
+ DEFAULT_METADATA_RECOVERY_STRATEGY,
+ ConfigDef.CaseInsensitiveValidString
+ .in(Utils.enumOptions(MetadataRecoveryStrategy.class)),
+ Importance.LOW,
+ METADATA_RECOVERY_STRATEGY_DOC);
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
index 249417da68..5982c08192 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
@@ -25,6 +25,8 @@
import org.apache.kafka.clients.DefaultHostResolver;
import org.apache.kafka.clients.HostResolver;
import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.LeastLoadedNode;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.clients.NetworkClient;
import org.apache.kafka.clients.StaleMetadataException;
import org.apache.kafka.clients.admin.CreateTopicsResult.TopicMetadataAndConfig;
@@ -277,7 +279,6 @@
import java.util.OptionalLong;
import java.util.Set;
import java.util.TreeMap;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@@ -400,6 +401,7 @@ public class KafkaAdminClient extends AdminClient {
private final long retryBackoffMaxMs;
private final ExponentialBackoff retryBackoff;
private final boolean clientTelemetryEnabled;
+ private final MetadataRecoveryStrategy metadataRecoveryStrategy;
/**
* The telemetry requests client instance id.
@@ -613,6 +615,7 @@ private KafkaAdminClient(AdminClientConfig config,
retryBackoffMaxMs,
CommonClientConfigs.RETRY_BACKOFF_JITTER);
this.clientTelemetryEnabled = config.getBoolean(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG);
+ this.metadataRecoveryStrategy = MetadataRecoveryStrategy.forName(config.getString(AdminClientConfig.METADATA_RECOVERY_STRATEGY_CONFIG));
config.logUnused();
AppInfoParser.registerAppInfo(JMX_PREFIX, clientId, metrics, time.milliseconds());
log.debug("Kafka admin client initialized");
@@ -699,7 +702,13 @@ private interface NodeProvider {
private class MetadataUpdateNodeIdProvider implements NodeProvider {
@Override
public Node provide() {
- return client.leastLoadedNode(time.milliseconds());
+ LeastLoadedNode leastLoadedNode = client.leastLoadedNode(time.milliseconds());
+ if (metadataRecoveryStrategy == MetadataRecoveryStrategy.REBOOTSTRAP
+ && !leastLoadedNode.hasNodeAvailableOrConnectionReady()) {
+ metadataManager.rebootstrap(time.milliseconds());
+ }
+
+ return leastLoadedNode.node();
}
@Override
@@ -781,7 +790,7 @@ public Node provide() {
if (metadataManager.isReady()) {
// This may return null if all nodes are busy.
// In that case, we will postpone node assignment.
- return client.leastLoadedNode(time.milliseconds());
+ return client.leastLoadedNode(time.milliseconds()).node();
}
metadataManager.requestUpdate();
return null;
@@ -836,7 +845,7 @@ public Node provide() {
} else {
// This may return null if all nodes are busy.
// In that case, we will postpone node assignment.
- return client.leastLoadedNode(time.milliseconds());
+ return client.leastLoadedNode(time.milliseconds()).node();
}
}
metadataManager.requestUpdate();
@@ -2130,7 +2139,7 @@ else if (topics instanceof TopicNameCollection)
throw new IllegalArgumentException("The TopicCollection: " + topics + " provided did not match any supported classes for describeTopics.");
}
- Call generateDescribeTopicsCallWithMetadataApi(
+ private Call generateDescribeTopicsCallWithMetadataApi(
List topicNamesList,
Map> topicFutures,
DescribeTopicsOptions options,
@@ -2193,7 +2202,7 @@ void handleFailure(Throwable throwable) {
};
}
- Call generateDescribeTopicsCallWithDescribeTopicPartitionsApi(
+ private Call generateDescribeTopicsCallWithDescribeTopicPartitionsApi(
List topicNamesList,
Map> topicFutures,
Map nodes,
@@ -2247,7 +2256,7 @@ void handleResponse(AbstractResponse abstractResponse) {
continue;
}
- TopicDescription currentTopicDescription = getTopicDescriptionFromDescribeTopicsResponseTopic(topic, nodes);
+ TopicDescription currentTopicDescription = getTopicDescriptionFromDescribeTopicsResponseTopic(topic, nodes, options.includeAuthorizedOperations());
if (partiallyFinishedTopicDescription != null && partiallyFinishedTopicDescription.name().equals(topicName)) {
// Add the partitions for the cursor topic of the previous batch.
@@ -2320,27 +2329,27 @@ private Map> handleDescribeTopicsByNamesWi
}
if (topicNamesList.isEmpty()) {
- return new HashMap<>(topicFutures);
+ return Collections.unmodifiableMap(topicFutures);
}
// First, we need to retrieve the node info.
DescribeClusterResult clusterResult = describeCluster();
- Map nodes;
- try {
- nodes = clusterResult.nodes().get().stream().collect(Collectors.toMap(Node::id, node -> node));
- } catch (InterruptedException | ExecutionException e) {
- completeAllExceptionally(topicFutures.values(), e.getCause());
- return new HashMap<>(topicFutures);
- }
-
- final long now = time.milliseconds();
+ clusterResult.nodes().whenComplete(
+ (nodes, exception) -> {
+ if (exception != null) {
+ completeAllExceptionally(topicFutures.values(), exception);
+ return;
+ }
- runnable.call(
- generateDescribeTopicsCallWithDescribeTopicPartitionsApi(topicNamesList, topicFutures, nodes, options, now),
- now
- );
+ final long now = time.milliseconds();
+ Map nodeIdMap = nodes.stream().collect(Collectors.toMap(Node::id, node -> node));
+ runnable.call(
+ generateDescribeTopicsCallWithDescribeTopicPartitionsApi(topicNamesList, topicFutures, nodeIdMap, options, now),
+ now
+ );
+ });
- return new HashMap<>(topicFutures);
+ return Collections.unmodifiableMap(topicFutures);
}
private Map> handleDescribeTopicsByIds(Collection topicIds, DescribeTopicsOptions options) {
@@ -2410,14 +2419,16 @@ void handleFailure(Throwable throwable) {
private TopicDescription getTopicDescriptionFromDescribeTopicsResponseTopic(
DescribeTopicPartitionsResponseTopic topic,
- Map nodes
+ Map nodes,
+ boolean includeAuthorizedOperations
) {
List partitionInfos = topic.partitions();
List partitions = new ArrayList<>(partitionInfos.size());
for (DescribeTopicPartitionsResponsePartition partitionInfo : partitionInfos) {
partitions.add(DescribeTopicPartitionsResponse.partitionToTopicPartitionInfo(partitionInfo, nodes));
}
- return new TopicDescription(topic.name(), topic.isInternal(), partitions, validAclOperations(topic.topicAuthorizedOperations()), topic.topicId());
+ Set authorisedOperations = includeAuthorizedOperations ? validAclOperations(topic.topicAuthorizedOperations()) : null;
+ return new TopicDescription(topic.name(), topic.isInternal(), partitions, authorisedOperations, topic.topicId());
}
// AutoMQ for Kafka inject start
@@ -4604,7 +4615,7 @@ public ListTransactionsResult listTransactions(ListTransactionsOptions options)
public FenceProducersResult fenceProducers(Collection transactionalIds, FenceProducersOptions options) {
AdminApiFuture.SimpleAdminApiFuture future =
FenceProducersHandler.newFuture(transactionalIds);
- FenceProducersHandler handler = new FenceProducersHandler(logContext);
+ FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs);
invokeDriver(handler, future, options.timeoutMs);
return new FenceProducersResult(future.all());
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java
index f8123c40ef..239f6eecef 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java
@@ -92,6 +92,11 @@ public class AdminMetadataManager {
*/
private ApiException fatalException = null;
+ /**
+ * The cluster with which the metadata was bootstrapped.
+ */
+ private Cluster bootstrapCluster;
+
public class AdminMetadataUpdater implements MetadataUpdater {
@Override
public List fetchNodes() {
@@ -275,6 +280,7 @@ public void updateFailed(Throwable exception) {
public void update(Cluster cluster, long now) {
if (cluster.isBootstrapConfigured()) {
log.debug("Setting bootstrap cluster metadata {}.", cluster);
+ bootstrapCluster = cluster;
} else {
log.debug("Updating cluster metadata to {}", cluster);
this.lastMetadataUpdateMs = now;
@@ -287,4 +293,12 @@ public void update(Cluster cluster, long now) {
this.cluster = cluster;
}
}
+
+ /**
+ * Rebootstrap metadata with the cluster previously used for bootstrapping.
+ */
+ public void rebootstrap(long now) {
+ log.info("Rebootstrapping with {}", this.bootstrapCluster);
+ update(bootstrapCluster, now);
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java
index 23572dd441..826db904a3 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java
@@ -16,6 +16,7 @@
*/
package org.apache.kafka.clients.admin.internals;
+import org.apache.kafka.clients.admin.FenceProducersOptions;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.ClusterAuthorizationException;
import org.apache.kafka.common.errors.TransactionalIdAuthorizationException;
@@ -38,12 +39,16 @@
public class FenceProducersHandler extends AdminApiHandler.Unbatched {
private final Logger log;
private final AdminApiLookupStrategy lookupStrategy;
+ private final int txnTimeoutMs;
public FenceProducersHandler(
- LogContext logContext
+ FenceProducersOptions options,
+ LogContext logContext,
+ int requestTimeoutMs
) {
this.log = logContext.logger(FenceProducersHandler.class);
this.lookupStrategy = new CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.TRANSACTION, logContext);
+ this.txnTimeoutMs = options.timeoutMs() != null ? options.timeoutMs() : requestTimeoutMs;
}
public static AdminApiFuture.SimpleAdminApiFuture newFuture(
@@ -82,9 +87,8 @@ InitProducerIdRequest.Builder buildSingleRequest(int brokerId, CoordinatorKey ke
.setProducerEpoch(ProducerIdAndEpoch.NONE.epoch)
.setProducerId(ProducerIdAndEpoch.NONE.producerId)
.setTransactionalId(key.idValue)
- // Set transaction timeout to 1 since it's only being initialized to fence out older producers with the same transactional ID,
- // and shouldn't be used for any actual record writes
- .setTransactionTimeoutMs(1);
+ // This timeout is used by the coordinator to append the record with the new producer epoch to the transaction log.
+ .setTransactionTimeoutMs(txnTimeoutMs);
return new InitProducerIdRequest.Builder(data);
}
@@ -130,6 +134,10 @@ private ApiResult handleError(
"coordinator is still in the process of loading state. Will retry",
transactionalIdKey.idValue);
return ApiResult.empty();
+ case CONCURRENT_TRANSACTIONS:
+ log.debug("InitProducerId request for transactionalId `{}` failed because of " +
+ "a concurrent transaction. Will retry", transactionalIdKey.idValue);
+ return ApiResult.empty();
case NOT_COORDINATOR:
case COORDINATOR_NOT_AVAILABLE:
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
index 7ec147e9e3..c4c10c404b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
@@ -18,6 +18,7 @@
import org.apache.kafka.clients.ClientDnsLookup;
import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
@@ -198,7 +199,10 @@ public class ConsumerConfig extends AbstractConfig {
* fetch.max.wait.ms
*/
public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms";
- private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes.";
+ private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before " +
+ "answering the fetch request there isn't sufficient data to immediately satisfy the requirement given by " +
+ "fetch.min.bytes. This config is used only for local log fetch. To tune the remote fetch maximum wait " +
+ "time, please refer to 'remote.fetch.max.wait.ms' broker config";
public static final int DEFAULT_FETCH_MAX_WAIT_MS = 500;
/** metadata.max.age.ms
*/
@@ -653,7 +657,14 @@ public class ConsumerConfig extends AbstractConfig {
Importance.MEDIUM,
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
.withClientSslSupport()
- .withClientSaslSupport();
+ .withClientSaslSupport()
+ .define(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG,
+ Type.STRING,
+ CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY,
+ ConfigDef.CaseInsensitiveValidString
+ .in(Utils.enumOptions(MetadataRecoveryStrategy.class)),
+ Importance.LOW,
+ CommonClientConfigs.METADATA_RECOVERY_STRATEGY_DOC);
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java
index ed1871a3d8..06bd17f4eb 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java
@@ -297,7 +297,7 @@ static List getAssignorInstances(List assigno
// first try to get the class if passed in as a string
if (klass instanceof String) {
try {
- klass = Class.forName((String) klass, true, Utils.getContextOrKafkaClassLoader());
+ klass = Utils.loadClass((String) klass, Object.class);
} catch (ClassNotFoundException classNotFound) {
throw new KafkaException(klass + " ClassNotFoundException exception occurred", classNotFound);
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
index 6930cd0295..6622cb286d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
@@ -26,7 +26,6 @@
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.ConsumerInterceptor;
-import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.GroupProtocol;
@@ -54,7 +53,6 @@
import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
import org.apache.kafka.clients.consumer.internals.events.EventProcessor;
import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent;
-import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseEvent;
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent;
import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent;
import org.apache.kafka.clients.consumer.internals.events.PollEvent;
@@ -91,7 +89,6 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Timer;
-import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.event.Level;
@@ -110,7 +107,6 @@
import java.util.OptionalLong;
import java.util.Set;
import java.util.SortedSet;
-import java.util.TreeSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;
@@ -235,12 +231,12 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
private final SubscriptionState subscriptions;
private final ConsumerMetadata metadata;
+ private int metadataVersionSnapshot;
private final Metrics metrics;
private final long retryBackoffMs;
private final int defaultApiTimeoutMs;
private final boolean autoCommitEnabled;
private volatile boolean closed = false;
- private final List assignors;
private final Optional clientTelemetryReporter;
// to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates
@@ -257,6 +253,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD);
private final AtomicInteger refCount = new AtomicInteger(0);
+ private FetchCommittedOffsetsEvent pendingOffsetFetchEvent;
+
AsyncKafkaConsumer(final ConsumerConfig config,
final Deserializer keyDeserializer,
final Deserializer valueDeserializer) {
@@ -313,6 +311,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
this.metadata = metadataFactory.build(config, subscriptions, logContext, clusterResourceListeners);
final List addresses = ClientUtils.parseAndValidateAddresses(config);
metadata.bootstrap(addresses);
+ this.metadataVersionSnapshot = metadata.updateVersion();
FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics);
FetchConfig fetchConfig = new FetchConfig(config);
@@ -373,10 +372,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
rebalanceListenerInvoker
);
this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext);
- this.assignors = ConsumerPartitionAssignor.getAssignorInstances(
- config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
- config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId))
- );
// The FetchCollector is only used on the application thread.
this.fetchCollector = fetchCollectorFactory.build(logContext,
@@ -424,7 +419,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
ConsumerMetadata metadata,
long retryBackoffMs,
int defaultApiTimeoutMs,
- List assignors,
String groupId,
boolean autoCommitEnabled) {
this.log = logContext.logger(getClass());
@@ -441,11 +435,11 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
this.metrics = metrics;
this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty()));
this.metadata = metadata;
+ this.metadataVersionSnapshot = metadata.updateVersion();
this.retryBackoffMs = retryBackoffMs;
this.defaultApiTimeoutMs = defaultApiTimeoutMs;
this.deserializers = deserializers;
this.applicationEventHandler = applicationEventHandler;
- this.assignors = assignors;
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
this.clientTelemetryReporter = Optional.empty();
this.autoCommitEnabled = autoCommitEnabled;
@@ -460,8 +454,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
Deserializer valueDeserializer,
KafkaClient client,
SubscriptionState subscriptions,
- ConsumerMetadata metadata,
- List assignors) {
+ ConsumerMetadata metadata) {
this.log = logContext.logger(getClass());
this.subscriptions = subscriptions;
this.clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
@@ -472,10 +465,10 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
this.time = time;
this.metrics = new Metrics(time);
this.metadata = metadata;
+ this.metadataVersionSnapshot = metadata.updateVersion();
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG);
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer);
- this.assignors = assignors;
this.clientTelemetryReporter = Optional.empty();
ConsumerMetrics metricsRegistry = new ConsumerMetrics(CONSUMER_METRIC_GROUP_PREFIX);
@@ -1237,8 +1230,8 @@ private void close(Duration timeout, boolean swallowException) {
clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(timeout.toMillis()));
closeTimer.update();
// Prepare shutting down the network thread
- prepareShutdown(closeTimer, firstException);
- closeTimer.update();
+ swallow(log, Level.ERROR, "Failed to release assignment before closing consumer",
+ () -> releaseAssignmentAndLeaveGroup(closeTimer), firstException);
swallow(log, Level.ERROR, "Failed invoking asynchronous commit callback.",
() -> awaitPendingAsyncCommitsAndExecuteCommitCallbacks(closeTimer, false), firstException);
if (applicationEventHandler != null)
@@ -1270,27 +1263,34 @@ private void close(Duration timeout, boolean swallowException) {
/**
* Prior to closing the network thread, we need to make sure the following operations happen in the right sequence:
* 1. autocommit offsets
- * 2. revoke all partitions
- * 3. if partition revocation completes successfully, send leave group
+ * 2. release assignment. This is done via a background unsubscribe event that will
+ * trigger the callbacks, clear the assignment on the subscription state and send the leave group request to the broker
*/
- void prepareShutdown(final Timer timer, final AtomicReference firstException) {
+ private void releaseAssignmentAndLeaveGroup(final Timer timer) {
if (!groupMetadata.get().isPresent())
return;
if (autoCommitEnabled)
- autoCommitSync(timer);
+ commitSyncAllConsumed(timer);
applicationEventHandler.add(new CommitOnCloseEvent());
- completeQuietly(
- () -> {
- maybeRevokePartitions();
- applicationEventHandler.addAndGet(new LeaveOnCloseEvent(calculateDeadlineMs(timer)));
- },
- "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException);
+
+ log.info("Releasing assignment and leaving group before closing consumer");
+ UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(calculateDeadlineMs(timer));
+ applicationEventHandler.add(unsubscribeEvent);
+ try {
+ processBackgroundEvents(unsubscribeEvent.future(), timer);
+ log.info("Completed releasing assignment and sending leave group to close consumer");
+ } catch (TimeoutException e) {
+ log.warn("Consumer triggered an unsubscribe event to leave the group but couldn't " +
+ "complete it within {} ms. It will proceed to close.", timer.timeoutMs());
+ } finally {
+ timer.update();
+ }
}
// Visible for testing
- void autoCommitSync(final Timer timer) {
+ void commitSyncAllConsumed(final Timer timer) {
Map allConsumed = subscriptions.allConsumed();
log.debug("Sending synchronous auto-commit of offsets {} on closing", allConsumed);
try {
@@ -1302,35 +1302,6 @@ void autoCommitSync(final Timer timer) {
timer.update();
}
- // Visible for testing
- void maybeRevokePartitions() {
- if (!subscriptions.hasAutoAssignedPartitions() || subscriptions.assignedPartitions().isEmpty())
- return;
- try {
- SortedSet droppedPartitions = new TreeSet<>(MembershipManagerImpl.TOPIC_PARTITION_COMPARATOR);
- droppedPartitions.addAll(subscriptions.assignedPartitions());
- if (subscriptions.rebalanceListener().isPresent())
- subscriptions.rebalanceListener().get().onPartitionsRevoked(droppedPartitions);
- } catch (Exception e) {
- throw new KafkaException(e);
- } finally {
- subscriptions.assignFromSubscribed(Collections.emptySet());
- }
- }
-
- // Visible for testing
- void completeQuietly(final Utils.ThrowingRunnable function,
- final String msg,
- final AtomicReference firstException) {
- try {
- function.run();
- } catch (TimeoutException e) {
- log.debug("Timeout expired before the {} operation could complete.", msg);
- } catch (Exception e) {
- firstException.compareAndSet(null, e);
- }
- }
-
@Override
public void wakeup() {
wakeupTrigger.wakeup();
@@ -1478,12 +1449,11 @@ public void assign(Collection partitions) {
}
/**
- * TODO: remove this when we implement the KIP-848 protocol.
- *
*
- * The contents of this method are shamelessly stolen from
- * {@link ConsumerCoordinator#updatePatternSubscription(Cluster)} and are used here because we won't have access
- * to a {@link ConsumerCoordinator} in this code. Perhaps it could be moved to a ConsumerUtils class?
+ *
+ * This function evaluates the regex that the consumer subscribed to
+ * against the list of topic names from metadata, and updates
+ * the list of topics in subscription state accordingly
*
* @param cluster Cluster from which we get the topics
*/
@@ -1493,7 +1463,7 @@ private void updatePatternSubscription(Cluster cluster) {
.collect(Collectors.toSet());
if (subscriptions.subscribeFromPattern(topicsToSubscribe)) {
applicationEventHandler.add(new SubscriptionChangeEvent());
- metadata.requestUpdateForNewTopics();
+ this.metadataVersionSnapshot = metadata.requestUpdateForNewTopics();
}
}
@@ -1506,7 +1476,8 @@ public void unsubscribe() {
Timer timer = time.timer(Long.MAX_VALUE);
UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(calculateDeadlineMs(timer));
applicationEventHandler.add(unsubscribeEvent);
- log.info("Unsubscribing all topics or patterns and assigned partitions");
+ log.info("Unsubscribing all topics or patterns and assigned partitions {}",
+ subscriptions.assignedPartitions());
try {
processBackgroundEvents(unsubscribeEvent.future(), timer);
@@ -1516,7 +1487,9 @@ public void unsubscribe() {
}
resetGroupMetadata();
}
- subscriptions.unsubscribe();
+ } catch (Exception e) {
+ log.error("Unsubscribe failed", e);
+ throw e;
} finally {
release();
}
@@ -1670,27 +1643,64 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) {
return true;
log.debug("Refreshing committed offsets for partitions {}", initializingPartitions);
+
+ // The shorter the timeout provided to poll(), the more likely the offsets fetch will time out. To handle
+ // this case, on the first attempt to fetch the committed offsets, a FetchCommittedOffsetsEvent is created
+ // (with potentially a longer timeout) and stored. The event is used for the first attempt, but in the
+ // case it times out, subsequent attempts will also use the event in order to wait for the results.
+ if (!canReusePendingOffsetFetchEvent(initializingPartitions)) {
+ // Give the event a reasonable amount of time to complete.
+ final long timeoutMs = Math.max(defaultApiTimeoutMs, timer.remainingMs());
+ final long deadlineMs = calculateDeadlineMs(time, timeoutMs);
+ pendingOffsetFetchEvent = new FetchCommittedOffsetsEvent(initializingPartitions, deadlineMs);
+ applicationEventHandler.add(pendingOffsetFetchEvent);
+ }
+
+ final CompletableFuture> future = pendingOffsetFetchEvent.future();
+
try {
- final FetchCommittedOffsetsEvent event =
- new FetchCommittedOffsetsEvent(
- initializingPartitions,
- calculateDeadlineMs(timer));
- wakeupTrigger.setActiveTask(event.future());
- final Map offsets = applicationEventHandler.addAndGet(event);
+ wakeupTrigger.setActiveTask(future);
+ final Map offsets = ConsumerUtils.getResult(future, timer);
+
+ // Clear the pending event once its result is successfully retrieved.
+ pendingOffsetFetchEvent = null;
+
refreshCommittedOffsets(offsets, metadata, subscriptions);
return true;
} catch (TimeoutException e) {
- log.error("Couldn't refresh committed offsets before timeout expired");
+ log.debug(
+ "The committed offsets for the following partition(s) could not be refreshed within the timeout: {} ",
+ initializingPartitions
+ );
return false;
+ } catch (InterruptException e) {
+ throw e;
+ } catch (Throwable t) {
+ pendingOffsetFetchEvent = null;
+ throw ConsumerUtils.maybeWrapAsKafkaException(t);
} finally {
wakeupTrigger.clearTask();
}
}
- private void throwIfNoAssignorsConfigured() {
- if (assignors.isEmpty())
- throw new IllegalStateException("Must configure at least one partition assigner class name to " +
- ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG + " configuration property");
+ /**
+ * This determines if the {@link #pendingOffsetFetchEvent pending offset fetch event} can be reused. Reuse
+ * is only possible if all the following conditions are true:
+ *
+ *
+ * A pending offset fetch event exists
+ * The partition set of the pending offset fetch event is the same as the given partition set
+ * The pending offset fetch event has not expired
+ *
+ */
+ private boolean canReusePendingOffsetFetchEvent(Set partitions) {
+ if (pendingOffsetFetchEvent == null)
+ return false;
+
+ if (!pendingOffsetFetchEvent.partitions().equals(partitions))
+ return false;
+
+ return pendingOffsetFetchEvent.deadlineMs() > time.milliseconds();
}
private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAndMetadata offsetAndMetadata) {
@@ -1780,7 +1790,6 @@ private void subscribeInternal(Pattern pattern, Optional topics, Optional currentTopicPartitions = new HashSet<>();
@@ -1818,7 +1825,7 @@ private void subscribeInternal(Collection topics, Optional(topics), listener))
- metadata.requestUpdateForNewTopics();
+ this.metadataVersionSnapshot = metadata.requestUpdateForNewTopics();
// Trigger subscribe event to effectively join the group if not already part of it,
// or just send the new subscription to the broker.
@@ -1996,10 +2003,17 @@ SubscriptionState subscriptions() {
return subscriptions;
}
+ boolean hasPendingOffsetFetchEvent() {
+ return pendingOffsetFetchEvent != null;
+ }
+
private void maybeUpdateSubscriptionMetadata() {
- if (subscriptions.hasPatternSubscription()) {
- updatePatternSubscription(metadata.fetch());
+ if (this.metadataVersionSnapshot < metadata.updateVersion()) {
+ this.metadataVersionSnapshot = metadata.updateVersion();
+ if (subscriptions.hasPatternSubscription()) {
+ updatePatternSubscription(metadata.fetch());
+ }
}
}
-
}
+
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java
index 577cf7dee6..000797dba0 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java
@@ -69,6 +69,7 @@
import static org.apache.kafka.common.protocol.Errors.COORDINATOR_LOAD_IN_PROGRESS;
public class CommitRequestManager implements RequestManager, MemberStateListener {
+ private final Time time;
private final SubscriptionState subscriptions;
private final LogContext logContext;
private final Logger log;
@@ -133,6 +134,7 @@ public CommitRequestManager(
final OptionalDouble jitter,
final Metrics metrics) {
Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets");
+ this.time = time;
this.logContext = logContext;
this.log = logContext.logger(getClass());
this.pendingRequests = new PendingRequests();
@@ -205,6 +207,13 @@ private static long findMinTime(final Collection extends RequestState> request
.orElse(Long.MAX_VALUE);
}
+ private KafkaException maybeWrapAsTimeoutException(Throwable t) {
+ if (t instanceof TimeoutException)
+ return (TimeoutException) t;
+ else
+ return new TimeoutException(t);
+ }
+
/**
* Generate a request to commit consumed offsets. Add the request to the queue of pending
* requests to be sent out on the next call to {@link #poll(long)}. If there are empty
@@ -245,7 +254,7 @@ public void maybeAutoCommitAsync() {
if (autoCommitEnabled() && autoCommitState.get().shouldAutoCommit()) {
OffsetCommitRequestState requestState = createOffsetCommitRequest(
subscriptions.allConsumed(),
- Optional.empty());
+ Long.MAX_VALUE);
CompletableFuture> result = requestAutoCommit(requestState);
// Reset timer to the interval (even if no request was generated), but ensure that if
// the request completes with a retriable error, the timer is reset to send the next
@@ -294,14 +303,14 @@ private void maybeResetTimerWithBackoff(final CompletableFuture maybeAutoCommitSyncBeforeRevocation(final long retryExpirationTimeMs) {
+ public CompletableFuture maybeAutoCommitSyncBeforeRevocation(final long deadlineMs) {
if (!autoCommitEnabled()) {
return CompletableFuture.completedFuture(null);
}
CompletableFuture result = new CompletableFuture<>();
OffsetCommitRequestState requestState =
- createOffsetCommitRequest(subscriptions.allConsumed(), Optional.of(retryExpirationTimeMs));
+ createOffsetCommitRequest(subscriptions.allConsumed(), deadlineMs);
autoCommitSyncBeforeRevocationWithRetries(requestState, result);
return result;
}
@@ -314,9 +323,9 @@ private void autoCommitSyncBeforeRevocationWithRetries(OffsetCommitRequestState
result.complete(null);
} else {
if (error instanceof RetriableException || isStaleEpochErrorAndValidEpochAvailable(error)) {
- if (error instanceof TimeoutException && requestAttempt.isExpired) {
+ if (requestAttempt.isExpired()) {
log.debug("Auto-commit sync before revocation timed out and won't be retried anymore");
- result.completeExceptionally(error);
+ result.completeExceptionally(maybeWrapAsTimeoutException(error));
} else if (error instanceof UnknownTopicOrPartitionException) {
log.debug("Auto-commit sync before revocation failed because topic or partition were deleted");
result.completeExceptionally(error);
@@ -367,7 +376,7 @@ public CompletableFuture commitAsync(final Map asyncCommitResult = new CompletableFuture<>();
@@ -385,28 +394,26 @@ public CompletableFuture commitAsync(final Map commitSync(final Map offsets,
- final long retryExpirationTimeMs) {
+ final long deadlineMs) {
CompletableFuture result = new CompletableFuture<>();
- OffsetCommitRequestState requestState = createOffsetCommitRequest(
- offsets,
- Optional.of(retryExpirationTimeMs));
+ OffsetCommitRequestState requestState = createOffsetCommitRequest(offsets, deadlineMs);
commitSyncWithRetries(requestState, result);
return result;
}
private OffsetCommitRequestState createOffsetCommitRequest(final Map offsets,
- final Optional expirationTimeMs) {
+ final long deadlineMs) {
return jitter.isPresent() ?
new OffsetCommitRequestState(
offsets,
groupId,
groupInstanceId,
- expirationTimeMs,
+ deadlineMs,
retryBackoffMs,
retryBackoffMaxMs,
jitter.getAsDouble(),
@@ -415,7 +422,7 @@ private OffsetCommitRequestState createOffsetCommitRequest(final Map> fetchOffsets(
final Set partitions,
- final long expirationTimeMs) {
+ final long deadlineMs) {
if (partitions.isEmpty()) {
return CompletableFuture.completedFuture(Collections.emptyMap());
}
CompletableFuture> result = new CompletableFuture<>();
- OffsetFetchRequestState request = createOffsetFetchRequest(partitions, expirationTimeMs);
+ OffsetFetchRequestState request = createOffsetFetchRequest(partitions, deadlineMs);
fetchOffsetsWithRetries(request, result);
return result;
}
private OffsetFetchRequestState createOffsetFetchRequest(final Set partitions,
- final long expirationTimeMs) {
+ final long deadlineMs) {
return jitter.isPresent() ?
new OffsetFetchRequestState(
partitions,
retryBackoffMs,
retryBackoffMaxMs,
- expirationTimeMs,
+ deadlineMs,
jitter.getAsDouble(),
memberInfo) :
new OffsetFetchRequestState(
partitions,
retryBackoffMs,
retryBackoffMaxMs,
- expirationTimeMs,
+ deadlineMs,
memberInfo);
}
@@ -516,8 +523,9 @@ private void fetchOffsetsWithRetries(final OffsetFetchRequestState fetchRequest,
result.complete(res);
} else {
if (error instanceof RetriableException || isStaleEpochErrorAndValidEpochAvailable(error)) {
- if (error instanceof TimeoutException && fetchRequest.isExpired) {
- result.completeExceptionally(error);
+ if (fetchRequest.isExpired()) {
+ log.debug("OffsetFetch request for {} timed out and won't be retried anymore", fetchRequest.requestedPartitions);
+ result.completeExceptionally(maybeWrapAsTimeoutException(error));
} else {
fetchRequest.resetFuture();
fetchOffsetsWithRetries(fetchRequest, result);
@@ -612,12 +620,12 @@ private class OffsetCommitRequestState extends RetriableRequestState {
OffsetCommitRequestState(final Map offsets,
final String groupId,
final Optional groupInstanceId,
- final Optional expirationTimeMs,
+ final long deadlineMs,
final long retryBackoffMs,
final long retryBackoffMaxMs,
final MemberInfo memberInfo) {
super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs,
- retryBackoffMaxMs, memberInfo, expirationTimeMs);
+ retryBackoffMaxMs, memberInfo, deadlineTimer(time, deadlineMs));
this.offsets = offsets;
this.groupId = groupId;
this.groupInstanceId = groupInstanceId;
@@ -628,13 +636,13 @@ private class OffsetCommitRequestState extends RetriableRequestState {
OffsetCommitRequestState(final Map offsets,
final String groupId,
final Optional groupInstanceId,
- final Optional expirationTimeMs,
+ final long deadlineMs,
final long retryBackoffMs,
final long retryBackoffMaxMs,
final double jitter,
final MemberInfo memberInfo) {
super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs, 2,
- retryBackoffMaxMs, jitter, memberInfo, expirationTimeMs);
+ retryBackoffMaxMs, jitter, memberInfo, deadlineTimer(time, deadlineMs));
this.offsets = offsets;
this.groupId = groupId;
this.groupInstanceId = groupInstanceId;
@@ -780,40 +788,24 @@ void removeRequest() {
* Represents a request that can be retried or aborted, based on member ID and epoch
* information.
*/
- abstract class RetriableRequestState extends RequestState {
+ abstract class RetriableRequestState extends TimedRequestState {
/**
* Member info (ID and epoch) to be included in the request if present.
*/
final MemberInfo memberInfo;
- /**
- * Time until which the request should be retried if it fails with retriable
- * errors. If not present, the request is triggered without waiting for a response or
- * retrying.
- */
- private final Optional expirationTimeMs;
-
- /**
- * True if the request expiration time has been reached. This is set when validating the
- * request expiration on {@link #poll(long)} before sending it. It is used to know if a
- * request should be retried on TimeoutException.
- */
- boolean isExpired;
-
RetriableRequestState(LogContext logContext, String owner, long retryBackoffMs,
- long retryBackoffMaxMs, MemberInfo memberInfo, Optional expirationTimeMs) {
- super(logContext, owner, retryBackoffMs, retryBackoffMaxMs);
+ long retryBackoffMaxMs, MemberInfo memberInfo, Timer timer) {
+ super(logContext, owner, retryBackoffMs, retryBackoffMaxMs, timer);
this.memberInfo = memberInfo;
- this.expirationTimeMs = expirationTimeMs;
}
// Visible for testing
RetriableRequestState(LogContext logContext, String owner, long retryBackoffMs, int retryBackoffExpBase,
- long retryBackoffMaxMs, double jitter, MemberInfo memberInfo, Optional expirationTimeMs) {
- super(logContext, owner, retryBackoffMs, retryBackoffExpBase, retryBackoffMaxMs, jitter);
+ long retryBackoffMaxMs, double jitter, MemberInfo memberInfo, Timer timer) {
+ super(logContext, owner, retryBackoffMs, retryBackoffExpBase, retryBackoffMaxMs, jitter, timer);
this.memberInfo = memberInfo;
- this.expirationTimeMs = expirationTimeMs;
}
/**
@@ -828,13 +820,12 @@ abstract class RetriableRequestState extends RequestState {
abstract CompletableFuture> future();
/**
- * Complete the request future with a TimeoutException if the request timeout has been
- * reached, based on the provided current time.
+ * Complete the request future with a TimeoutException if the request has been sent out
+ * at least once and the timeout has been reached.
*/
- void maybeExpire(long currentTimeMs) {
- if (retryTimeoutExpired(currentTimeMs)) {
+ void maybeExpire() {
+ if (numAttempts > 0 && isExpired()) {
removeRequest();
- isExpired = true;
future().completeExceptionally(new TimeoutException(requestDescription() +
" could not complete before timeout expired."));
}
@@ -846,11 +837,12 @@ void maybeExpire(long currentTimeMs) {
NetworkClientDelegate.UnsentRequest buildRequestWithResponseHandling(final AbstractRequest.Builder> builder) {
NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest(
builder,
- coordinatorRequestManager.coordinator());
+ coordinatorRequestManager.coordinator()
+ );
request.whenComplete(
(response, throwable) -> {
- long currentTimeMs = request.handler().completionTimeMs();
- handleClientResponse(response, throwable, currentTimeMs);
+ long completionTimeMs = request.handler().completionTimeMs();
+ handleClientResponse(response, throwable, completionTimeMs);
});
return request;
}
@@ -875,10 +867,6 @@ private void handleClientResponse(final ClientResponse response,
abstract void onResponse(final ClientResponse response);
- boolean retryTimeoutExpired(long currentTimeMs) {
- return expirationTimeMs.isPresent() && expirationTimeMs.get() <= currentTimeMs;
- }
-
abstract void removeRequest();
}
@@ -898,10 +886,10 @@ class OffsetFetchRequestState extends RetriableRequestState {
public OffsetFetchRequestState(final Set partitions,
final long retryBackoffMs,
final long retryBackoffMaxMs,
- final long expirationTimeMs,
+ final long deadlineMs,
final MemberInfo memberInfo) {
super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs,
- retryBackoffMaxMs, memberInfo, Optional.of(expirationTimeMs));
+ retryBackoffMaxMs, memberInfo, deadlineTimer(time, deadlineMs));
this.requestedPartitions = partitions;
this.future = new CompletableFuture<>();
}
@@ -909,11 +897,11 @@ public OffsetFetchRequestState(final Set partitions,
public OffsetFetchRequestState(final Set partitions,
final long retryBackoffMs,
final long retryBackoffMaxMs,
- final long expirationTimeMs,
+ final long deadlineMs,
final double jitter,
final MemberInfo memberInfo) {
super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs, 2,
- retryBackoffMaxMs, jitter, memberInfo, Optional.of(expirationTimeMs));
+ retryBackoffMaxMs, jitter, memberInfo, deadlineTimer(time, deadlineMs));
this.requestedPartitions = partitions;
this.future = new CompletableFuture<>();
}
@@ -1145,9 +1133,10 @@ private CompletableFuture> addOffsetFetch
inflightOffsetFetches.stream().filter(r -> r.sameRequest(request)).findAny();
if (dupe.isPresent() || inflight.isPresent()) {
- log.info("Duplicated OffsetFetchRequest: " + request.requestedPartitions);
+ log.debug("Duplicated unsent offset fetch request found for partitions: {}", request.requestedPartitions);
dupe.orElseGet(inflight::get).chainFuture(request.future);
} else {
+ log.debug("Enqueuing offset fetch request for partitions: {}", request.requestedPartitions);
this.unsentOffsetFetches.add(request);
}
return request.future;
@@ -1165,7 +1154,7 @@ List drain(final long currentTimeMs) {
.filter(request -> !request.canSendRequest(currentTimeMs))
.collect(Collectors.toList());
- failAndRemoveExpiredCommitRequests(currentTimeMs);
+ failAndRemoveExpiredCommitRequests();
// Add all unsent offset commit requests to the unsentRequests list
List unsentRequests = unsentOffsetCommits.stream()
@@ -1179,7 +1168,7 @@ List drain(final long currentTimeMs) {
unsentOffsetFetches.stream()
.collect(Collectors.partitioningBy(request -> request.canSendRequest(currentTimeMs)));
- failAndRemoveExpiredFetchRequests(currentTimeMs);
+ failAndRemoveExpiredFetchRequests();
// Add all sendable offset fetch requests to the unsentRequests list and to the inflightOffsetFetches list
for (OffsetFetchRequestState request : partitionedBySendability.get(true)) {
@@ -1200,18 +1189,18 @@ List drain(final long currentTimeMs) {
* Find the unsent commit requests that have expired, remove them and complete their
* futures with a TimeoutException.
*/
- private void failAndRemoveExpiredCommitRequests(final long currentTimeMs) {
+ private void failAndRemoveExpiredCommitRequests() {
Queue requestsToPurge = new LinkedList<>(unsentOffsetCommits);
- requestsToPurge.forEach(req -> req.maybeExpire(currentTimeMs));
+ requestsToPurge.forEach(RetriableRequestState::maybeExpire);
}
/**
* Find the unsent fetch requests that have expired, remove them and complete their
* futures with a TimeoutException.
*/
- private void failAndRemoveExpiredFetchRequests(final long currentTimeMs) {
+ private void failAndRemoveExpiredFetchRequests() {
Queue requestsToPurge = new LinkedList<>(unsentOffsetFetches);
- requestsToPurge.forEach(req -> req.maybeExpire(currentTimeMs));
+ requestsToPurge.forEach(RetriableRequestState::maybeExpire);
}
private void clearAll() {
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java
index bd95e06c86..81c45aba69 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java
@@ -91,8 +91,7 @@ public ConsumerDelegate create(LogContext logContext,
valueDeserializer,
client,
subscriptions,
- metadata,
- assignors
+ metadata
);
else
return new LegacyKafkaConsumer<>(
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
index 4699f00c15..50c7eb5b02 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
@@ -139,7 +139,7 @@ public RequestFuture send(Node node,
public Node leastLoadedNode() {
lock.lock();
try {
- return client.leastLoadedNode(time.milliseconds());
+ return client.leastLoadedNode(time.milliseconds()).node();
} finally {
lock.unlock();
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
index adee659460..64bba14837 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java
@@ -207,8 +207,7 @@ private void reapExpiredApplicationEvents(long currentTimeMs) {
*/
// Visible for testing
static void runAtClose(final Collection> requestManagers,
- final NetworkClientDelegate networkClientDelegate,
- final Timer timer) {
+ final NetworkClientDelegate networkClientDelegate) {
// These are the optional outgoing requests at the
requestManagers.stream()
.filter(Optional::isPresent)
@@ -293,21 +292,28 @@ private void closeInternal(final Duration timeout) {
* Check the unsent queue one last time and poll until all requests are sent or the timer runs out.
*/
private void sendUnsentRequests(final Timer timer) {
- if (networkClientDelegate.unsentRequests().isEmpty())
+ if (!networkClientDelegate.hasAnyPendingRequests())
return;
+
do {
networkClientDelegate.poll(timer.remainingMs(), timer.currentTimeMs());
timer.update();
- } while (timer.notExpired() && !networkClientDelegate.unsentRequests().isEmpty());
+ } while (timer.notExpired() && networkClientDelegate.hasAnyPendingRequests());
+
+ if (networkClientDelegate.hasAnyPendingRequests()) {
+ log.warn("Close timeout of {} ms expired before the consumer network thread was able " +
+ "to complete pending requests. Inflight request count: {}, Unsent request count: {}",
+ timer.timeoutMs(), networkClientDelegate.inflightRequestCount(), networkClientDelegate.unsentRequests().size());
+ }
}
void cleanup() {
log.trace("Closing the consumer network thread");
Timer timer = time.timer(closeTimeout);
try {
- runAtClose(requestManagers.entries(), networkClientDelegate, timer);
+ runAtClose(requestManagers.entries(), networkClientDelegate);
} catch (Exception e) {
- log.error("Unexpected error during shutdown. Proceed with closing.", e);
+ log.error("Unexpected error during shutdown. Proceed with closing.", e);
} finally {
sendUnsentRequests(timer);
applicationEventReaper.reap(applicationEventQueue);
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java
index a956ef3a93..43e15a4def 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java
@@ -45,6 +45,7 @@
import java.util.TreeSet;
import java.util.stream.Collectors;
+
/**
* Manages the request creation and response handling for the heartbeat. The module creates a
* {@link ConsumerGroupHeartbeatRequest} using the state stored in the {@link MembershipManager} and enqueue it to
@@ -208,7 +209,11 @@ public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(leaveHeartbeat));
}
- boolean heartbeatNow = membershipManager.shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight();
+ // Case 1: The member is leaving
+ boolean heartbeatNow = membershipManager.state() == MemberState.LEAVING ||
+ // Case 2: The member state indicates it should send a heartbeat without waiting for the interval, and there is no heartbeat request currently in-flight
+ (membershipManager.shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight());
+
if (!heartbeatRequestState.canSendRequest(currentTimeMs) && !heartbeatNow) {
return new NetworkClientDelegate.PollResult(heartbeatRequestState.timeToNextHeartbeatMs(currentTimeMs));
}
@@ -258,7 +263,8 @@ public void resetPollTimer(final long pollMs) {
pollTimer.update(pollMs);
if (pollTimer.isExpired()) {
logger.warn("Time between subsequent calls to poll() was longer than the configured " +
- "max.poll.interval.ms, exceeded approximately by {} ms.", pollTimer.isExpiredBy());
+ "max.poll.interval.ms, exceeded approximately by {} ms. Member {} will rejoin the group now.",
+ pollTimer.isExpiredBy(), membershipManager.memberId());
membershipManager.maybeRejoinStaleMember();
}
pollTimer.reset(maxPollIntervalMs);
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java
index 76a550ad71..763fc02769 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java
@@ -46,6 +46,7 @@
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -520,12 +521,15 @@ private void replaceTargetAssignmentWithNewAssignment(
@Override
public void transitionToFenced() {
if (state == MemberState.PREPARE_LEAVING) {
- log.debug("Member {} with epoch {} got fenced but it is already preparing to leave " +
+ log.info("Member {} with epoch {} got fenced but it is already preparing to leave " +
"the group, so it will stop sending heartbeat and won't attempt to rejoin.",
memberId, memberEpoch);
- // Transition to UNSUBSCRIBED, ensuring that the member (that is not part of the
- // group anymore from the broker point of view) will stop sending heartbeats while it
- // completes the ongoing leaving operation.
+ // Briefly transition to LEAVING to ensure all required actions are applied even
+ // though there is no need to send a leave group heartbeat (ex. clear epoch and
+ // notify epoch listeners). Then transition to UNSUBSCRIBED, ensuring that the member
+ // (that is not part of the group anymore from the broker point of view) will stop
+ // sending heartbeats while it completes the ongoing leaving operation.
+ transitionToSendingLeaveGroup(false);
transitionTo(MemberState.UNSUBSCRIBED);
return;
}
@@ -552,7 +556,7 @@ public void transitionToFenced() {
log.error("onPartitionsLost callback invocation failed while releasing assignment" +
" after member got fenced. Member will rejoin the group anyways.", error);
}
- clearSubscription();
+ clearAssignment();
if (state == MemberState.FENCED) {
transitionToJoining();
} else {
@@ -589,7 +593,7 @@ public void transitionToFatal() {
log.error("onPartitionsLost callback invocation failed while releasing assignment" +
"after member failed with fatal error.", error);
}
- clearSubscription();
+ clearAssignment();
});
}
@@ -605,7 +609,7 @@ public void onSubscriptionUpdated() {
/**
* Clear the assigned partitions in the member subscription, pending assignments and metadata cache.
*/
- private void clearSubscription() {
+ private void clearAssignment() {
if (subscriptions.hasAutoAssignedPartitions()) {
subscriptions.assignFromSubscribed(Collections.emptySet());
}
@@ -655,7 +659,7 @@ public void transitionToJoining() {
public CompletableFuture leaveGroup() {
if (isNotInGroup()) {
if (state == MemberState.FENCED) {
- clearSubscription();
+ clearAssignment();
transitionTo(MemberState.UNSUBSCRIBED);
}
return CompletableFuture.completedFuture(null);
@@ -664,6 +668,7 @@ public CompletableFuture leaveGroup() {
if (state == MemberState.PREPARE_LEAVING || state == MemberState.LEAVING) {
// Member already leaving. No-op and return existing leave group future that will
// complete when the ongoing leave operation completes.
+ log.debug("Leave group operation already in progress for member {}", memberId);
return leaveGroupInProgress.get();
}
@@ -673,8 +678,16 @@ public CompletableFuture leaveGroup() {
CompletableFuture callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment();
callbackResult.whenComplete((result, error) -> {
+ if (error != null) {
+ log.error("Member {} callback to release assignment failed. Member will proceed " +
+ "to send leave group heartbeat", memberId, error);
+ } else {
+ log.debug("Member {} completed callback to release assignment and will send leave " +
+ "group heartbeat", memberId);
+ }
// Clear the subscription, no matter if the callback execution failed or succeeded.
- clearSubscription();
+ subscriptions.unsubscribe();
+ clearAssignment();
// Transition to ensure that a heartbeat request is sent out to effectively leave the
// group (even in the case where the member had no assignment to release or when the
@@ -705,6 +718,9 @@ private CompletableFuture invokeOnPartitionsRevokedOrLostToReleaseAssignme
SortedSet droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
droppedPartitions.addAll(subscriptions.assignedPartitions());
+ log.debug("Member {} is triggering callbacks to release assignment {} and leave group",
+ memberId, droppedPartitions);
+
CompletableFuture callbackResult;
if (droppedPartitions.isEmpty()) {
// No assignment to release.
@@ -764,7 +780,7 @@ public void transitionToSendingLeaveGroup(boolean dueToExpiredPollTimer) {
* This also includes the latest member ID in the notification. If the member fails or leaves
* the group, this will be invoked with empty epoch and member ID.
*/
- private void notifyEpochChange(Optional epoch, Optional memberId) {
+ void notifyEpochChange(Optional epoch, Optional memberId) {
stateUpdatesListeners.forEach(stateListener -> stateListener.onMemberEpochUpdated(epoch, memberId));
}
@@ -794,8 +810,12 @@ public void onHeartbeatRequestSent() {
}
} else if (state == MemberState.LEAVING) {
if (isPollTimerExpired) {
+ log.debug("Member {} sent heartbeat to leave due to expired poll timer. It will " +
+ "remain stale (no heartbeat) until it rejoins the group on the next consumer " +
+ "poll.", memberId);
transitionToStale();
} else {
+ log.debug("Member {} sent heartbeat to leave group.", memberId);
transitionToUnsubscribed();
}
}
@@ -875,7 +895,7 @@ private void transitionToStale() {
log.error("onPartitionsLost callback invocation failed while releasing assignment" +
" after member left group due to expired poll timer.", error);
}
- clearSubscription();
+ clearAssignment();
log.debug("Member {} sent leave group heartbeat and released its assignment. It will remain " +
"in {} state until the poll timer is reset, and it will then rejoin the group",
memberId, MemberState.STALE);
@@ -939,11 +959,13 @@ void maybeReconcile() {
revokedPartitions.addAll(ownedPartitions);
revokedPartitions.removeAll(assignedTopicPartitions);
- log.info("Updating assignment with local epoch {}\n" +
+ log.info("Reconciling assignment with local epoch {}\n" +
+ "\tMember: {}\n" +
"\tAssigned partitions: {}\n" +
"\tCurrent owned partitions: {}\n" +
"\tAdded partitions (assigned - owned): {}\n" +
"\tRevoked partitions (owned - assigned): {}\n",
+ memberId,
resolvedAssignment.localEpoch,
assignedTopicPartitions,
ownedPartitions,
@@ -960,7 +982,7 @@ void maybeReconcile() {
// best effort to commit the offsets in the case where the epoch might have changed while
// the current reconciliation is in process. Note this is using the rebalance timeout as
// it is the limit enforced by the broker to complete the reconciliation process.
- commitResult = commitRequestManager.maybeAutoCommitSyncBeforeRevocation(getExpirationTimeForTimeout(rebalanceTimeoutMs));
+ commitResult = commitRequestManager.maybeAutoCommitSyncBeforeRevocation(getDeadlineMsForTimeout(rebalanceTimeoutMs));
// Execute commit -> onPartitionsRevoked -> onPartitionsAssigned.
commitResult.whenComplete((__, commitReqError) -> {
@@ -986,7 +1008,7 @@ void maybeReconcile() {
});
}
- long getExpirationTimeForTimeout(final long timeoutMs) {
+ long getDeadlineMsForTimeout(final long timeoutMs) {
long expiration = time.milliseconds() + timeoutMs;
if (expiration < 0) {
return Long.MAX_VALUE;
@@ -1175,11 +1197,14 @@ private Optional findTopicNameInGlobalOrLocalCache(Uuid topicId) {
* request fails, this will proceed to invoke the user callbacks anyway,
* returning a future that will complete or fail depending on the callback execution only.
*
- * @param revokedPartitions Partitions to revoke.
+ * @param partitionsToRevoke Partitions to revoke.
* @return Future that will complete when the commit request and user callback completes.
* Visible for testing
*/
- CompletableFuture revokePartitions(Set revokedPartitions) {
+ CompletableFuture revokePartitions(Set partitionsToRevoke) {
+ // Ensure the set of partitions to revoke are still assigned
+ Set revokedPartitions = new HashSet<>(partitionsToRevoke);
+ revokedPartitions.retainAll(subscriptions.assignedPartitions());
log.info("Revoking previously assigned partitions {}", revokedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", ")));
logPausedPartitionsBeingRevoked(revokedPartitions);
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java
index 141f5f955c..83015acd89 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java
@@ -81,6 +81,10 @@ Queue unsentRequests() {
return unsentRequests;
}
+ public int inflightRequestCount() {
+ return client.inFlightRequestCount();
+ }
+
/**
* Check if the node is disconnected and unavailable for immediate reconnection (i.e. if it is in
* reconnect backoff window following the disconnect).
@@ -130,6 +134,13 @@ public void poll(final long timeoutMs, final long currentTimeMs) {
checkDisconnects(currentTimeMs);
}
+ /**
+ * Return true if there is at least one in-flight request or unsent request.
+ */
+ public boolean hasAnyPendingRequests() {
+ return client.hasInFlightRequests() || !unsentRequests.isEmpty();
+ }
+
/**
* Tries to send the requests in the unsentRequest queue. If the request doesn't have an assigned node, it will
* find the leastLoadedOne, and will be retried in the next {@code poll()}. If the request is expired, a
@@ -156,7 +167,7 @@ private void trySend(final long currentTimeMs) {
}
boolean doSend(final UnsentRequest r, final long currentTimeMs) {
- Node node = r.node.orElse(client.leastLoadedNode(currentTimeMs));
+ Node node = r.node.orElse(client.leastLoadedNode(currentTimeMs).node());
if (node == null || nodeUnavailable(node)) {
log.debug("No broker available to send the request: {}. Retrying.", r);
return false;
@@ -201,7 +212,7 @@ private ClientRequest makeClientRequest(
}
public Node leastLoadedNode() {
- return this.client.leastLoadedNode(time.milliseconds());
+ return this.client.leastLoadedNode(time.milliseconds()).node();
}
public void wakeup() {
@@ -309,11 +320,20 @@ Optional node() {
@Override
public String toString() {
+ String remainingMs;
+
+ if (timer != null) {
+ timer.update();
+ remainingMs = String.valueOf(timer.remainingMs());
+ } else {
+ remainingMs = "";
+ }
+
return "UnsentRequest{" +
"requestBuilder=" + requestBuilder +
", handler=" + handler +
", node=" + node +
- ", timer=" + timer +
+ ", remainingMs=" + remainingMs +
'}';
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java
index 75d87432db..edd8ca9721 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java
@@ -155,6 +155,7 @@ protected RequestManagers create() {
apiVersions);
final TopicMetadataRequestManager topic = new TopicMetadataRequestManager(
logContext,
+ time,
config);
HeartbeatRequestManager heartbeatRequestManager = null;
MembershipManager membershipManager = null;
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TimedRequestState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TimedRequestState.java
new file mode 100644
index 0000000000..c61032cea7
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TimedRequestState.java
@@ -0,0 +1,71 @@
+/*
+ * 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.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+
+/**
+ * {@code TimedRequestState} adds to a {@link RequestState} a {@link Timer} with which to keep track
+ * of the request's expiration.
+ */
+public class TimedRequestState extends RequestState {
+
+ private final Timer timer;
+
+ public TimedRequestState(final LogContext logContext,
+ final String owner,
+ final long retryBackoffMs,
+ final long retryBackoffMaxMs,
+ final Timer timer) {
+ super(logContext, owner, retryBackoffMs, retryBackoffMaxMs);
+ this.timer = timer;
+ }
+
+ public TimedRequestState(final LogContext logContext,
+ final String owner,
+ final long retryBackoffMs,
+ final int retryBackoffExpBase,
+ final long retryBackoffMaxMs,
+ final double jitter,
+ final Timer timer) {
+ super(logContext, owner, retryBackoffMs, retryBackoffExpBase, retryBackoffMaxMs, jitter);
+ this.timer = timer;
+ }
+
+ public boolean isExpired() {
+ timer.update();
+ return timer.isExpired();
+ }
+
+ public long remainingMs() {
+ timer.update();
+ return timer.remainingMs();
+ }
+
+ public static Timer deadlineTimer(final Time time, final long deadlineMs) {
+ long diff = Math.max(0, deadlineMs - time.milliseconds());
+ return time.timer(diff);
+ }
+
+
+ @Override
+ protected String toStringBase() {
+ return super.toStringBase() + ", remainingMs=" + remainingMs();
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java
index 75a5ed08d1..a555d6ce7f 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java
@@ -29,6 +29,7 @@
import org.apache.kafka.common.requests.MetadataRequest;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
import org.slf4j.Logger;
import java.util.Collections;
@@ -61,6 +62,7 @@
*/
public class TopicMetadataRequestManager implements RequestManager {
+ private final Time time;
private final boolean allowAutoTopicCreation;
private final List inflightRequests;
private final long retryBackoffMs;
@@ -68,9 +70,10 @@ public class TopicMetadataRequestManager implements RequestManager {
private final Logger log;
private final LogContext logContext;
- public TopicMetadataRequestManager(final LogContext context, final ConsumerConfig config) {
+ public TopicMetadataRequestManager(final LogContext context, final Time time, final ConsumerConfig config) {
logContext = context;
log = logContext.logger(getClass());
+ this.time = time;
inflightRequests = new LinkedList<>();
retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
@@ -81,7 +84,7 @@ public TopicMetadataRequestManager(final LogContext context, final ConsumerConfi
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
// Prune any requests which have timed out
List expiredRequests = inflightRequests.stream()
- .filter(req -> req.isExpired(currentTimeMs))
+ .filter(TimedRequestState::isExpired)
.collect(Collectors.toList());
expiredRequests.forEach(TopicMetadataRequestState::expire);
@@ -99,10 +102,10 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
*
* @return the future of the metadata request.
*/
- public CompletableFuture>> requestAllTopicsMetadata(final long expirationTimeMs) {
+ public CompletableFuture>> requestAllTopicsMetadata(final long deadlineMs) {
TopicMetadataRequestState newRequest = new TopicMetadataRequestState(
logContext,
- expirationTimeMs,
+ deadlineMs,
retryBackoffMs,
retryBackoffMaxMs);
inflightRequests.add(newRequest);
@@ -115,11 +118,11 @@ public CompletableFuture>> requestAllTopicsMetad
* @param topic to be requested.
* @return the future of the metadata request.
*/
- public CompletableFuture>> requestTopicMetadata(final String topic, final long expirationTimeMs) {
+ public CompletableFuture>> requestTopicMetadata(final String topic, final long deadlineMs) {
TopicMetadataRequestState newRequest = new TopicMetadataRequestState(
logContext,
topic,
- expirationTimeMs,
+ deadlineMs,
retryBackoffMs,
retryBackoffMaxMs);
inflightRequests.add(newRequest);
@@ -131,35 +134,32 @@ List inflightRequests() {
return inflightRequests;
}
- class TopicMetadataRequestState extends RequestState {
+ class TopicMetadataRequestState extends TimedRequestState {
private final String topic;
private final boolean allTopics;
- private final long expirationTimeMs;
CompletableFuture>> future;
public TopicMetadataRequestState(final LogContext logContext,
- final long expirationTimeMs,
+ final long deadlineMs,
final long retryBackoffMs,
final long retryBackoffMaxMs) {
super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs,
- retryBackoffMaxMs);
+ retryBackoffMaxMs, deadlineTimer(time, deadlineMs));
future = new CompletableFuture<>();
this.topic = null;
this.allTopics = true;
- this.expirationTimeMs = expirationTimeMs;
}
public TopicMetadataRequestState(final LogContext logContext,
final String topic,
- final long expirationTimeMs,
+ final long deadlineMs,
final long retryBackoffMs,
final long retryBackoffMaxMs) {
super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs,
- retryBackoffMaxMs);
+ retryBackoffMaxMs, deadlineTimer(time, deadlineMs));
future = new CompletableFuture<>();
this.topic = topic;
this.allTopics = false;
- this.expirationTimeMs = expirationTimeMs;
}
/**
@@ -167,10 +167,6 @@ public TopicMetadataRequestState(final LogContext logContext,
* {@link org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest} if needed.
*/
private Optional send(final long currentTimeMs) {
- if (currentTimeMs >= expirationTimeMs) {
- return Optional.empty();
- }
-
if (!canSendRequest(currentTimeMs)) {
return Optional.empty();
}
@@ -183,10 +179,6 @@ private Optional send(final long currentTim
return Optional.of(createUnsentRequest(request));
}
- private boolean isExpired(final long currentTimeMs) {
- return currentTimeMs >= expirationTimeMs;
- }
-
private void expire() {
completeFutureAndRemoveRequest(
new TimeoutException("Timeout expired while fetching topic metadata"));
@@ -210,9 +202,8 @@ private NetworkClientDelegate.UnsentRequest createUnsentRequest(
private void handleError(final Throwable exception,
final long completionTimeMs) {
if (exception instanceof RetriableException) {
- if (completionTimeMs >= expirationTimeMs) {
- completeFutureAndRemoveRequest(
- new TimeoutException("Timeout expired while fetching topic metadata"));
+ if (isExpired()) {
+ completeFutureAndRemoveRequest(new TimeoutException("Timeout expired while fetching topic metadata"));
} else {
onFailedAttempt(completionTimeMs);
}
@@ -222,20 +213,12 @@ private void handleError(final Throwable exception,
}
private void handleResponse(final ClientResponse response) {
- long responseTimeMs = response.receivedTimeMs();
try {
Map> res = handleTopicMetadataResponse((MetadataResponse) response.responseBody());
future.complete(res);
inflightRequests.remove(this);
- } catch (RetriableException e) {
- if (responseTimeMs >= expirationTimeMs) {
- completeFutureAndRemoveRequest(
- new TimeoutException("Timeout expired while fetching topic metadata"));
- } else {
- onFailedAttempt(responseTimeMs);
- }
- } catch (Exception t) {
- completeFutureAndRemoveRequest(t);
+ } catch (Exception e) {
+ handleError(e, response.receivedTimeMs());
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java
index 2897117da8..ae5f7faeae 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java
@@ -31,7 +31,7 @@ public enum Type {
COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE,
LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, ALL_TOPICS_METADATA, SUBSCRIPTION_CHANGE,
UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED,
- COMMIT_ON_CLOSE, LEAVE_ON_CLOSE
+ COMMIT_ON_CLOSE
}
private final Type type;
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java
index 7ee0c09d40..9d0062f546 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java
@@ -32,7 +32,6 @@
import java.util.List;
import java.util.Map;
-import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.function.BiConsumer;
import java.util.function.Supplier;
@@ -119,10 +118,6 @@ public void process(ApplicationEvent event) {
process((CommitOnCloseEvent) event);
return;
- case LEAVE_ON_CLOSE:
- process((LeaveOnCloseEvent) event);
- return;
-
default:
log.warn("Application event type " + event.type() + " was not expected");
}
@@ -268,20 +263,6 @@ private void process(@SuppressWarnings("unused") final CommitOnCloseEvent event)
requestManagers.commitRequestManager.get().signalClose();
}
- private void process(final LeaveOnCloseEvent event) {
- if (!requestManagers.heartbeatRequestManager.isPresent()) {
- event.future().complete(null);
- return;
- }
- MembershipManager membershipManager =
- Objects.requireNonNull(requestManagers.heartbeatRequestManager.get().membershipManager(), "Expecting " +
- "membership manager to be non-null");
- log.debug("Leaving group before closing");
- CompletableFuture future = membershipManager.leaveGroup();
- // The future will be completed on heartbeat sent
- future.whenComplete(complete(event.future()));
- }
-
private BiConsumer super T, ? super Throwable> complete(final CompletableFuture b) {
return (value, exception) -> {
if (exception != null)
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java
index dffac12902..8cd17d19fe 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java
@@ -43,6 +43,7 @@ public CompletableFuture future() {
return future;
}
+ @Override
public long deadlineMs() {
return deadlineMs;
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
index c67d60a180..a59ee81b4a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
@@ -18,6 +18,7 @@
import org.apache.kafka.clients.ClientDnsLookup;
import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.common.compress.GzipCompression;
import org.apache.kafka.common.compress.Lz4Compression;
import org.apache.kafka.common.compress.ZstdCompression;
@@ -528,7 +529,14 @@ public class ProducerConfig extends AbstractConfig {
null,
new ConfigDef.NonEmptyString(),
Importance.LOW,
- TRANSACTIONAL_ID_DOC);
+ TRANSACTIONAL_ID_DOC)
+ .define(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG,
+ Type.STRING,
+ CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY,
+ ConfigDef.CaseInsensitiveValidString
+ .in(Utils.enumOptions(MetadataRecoveryStrategy.class)),
+ Importance.LOW,
+ CommonClientConfigs.METADATA_RECOVERY_STRATEGY_DOC);
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index c4e2b73e8b..b1a3ab9e29 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -484,7 +484,7 @@ private boolean maybeSendAndPollTransactionalRequest() {
FindCoordinatorRequest.CoordinatorType coordinatorType = nextRequestHandler.coordinatorType();
targetNode = coordinatorType != null ?
transactionManager.coordinator(coordinatorType) :
- client.leastLoadedNode(time.milliseconds());
+ client.leastLoadedNode(time.milliseconds()).node();
if (targetNode != null) {
if (!awaitNodeReady(targetNode, coordinatorType)) {
log.trace("Target node {} not ready within request timeout, will retry when node is ready.", targetNode);
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
index d82d06fa16..fd40c031ee 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
@@ -766,14 +766,7 @@ else if (value instanceof String)
if (value instanceof Class)
return value;
else if (value instanceof String) {
- ClassLoader contextOrKafkaClassLoader = Utils.getContextOrKafkaClassLoader();
- // Use loadClass here instead of Class.forName because the name we use here may be an alias
- // and not match the name of the class that gets loaded. If that happens, Class.forName can
- // throw an exception.
- Class> klass = contextOrKafkaClassLoader.loadClass(trimmed);
- // Invoke forName here with the true name of the requested class to cause class
- // initialization to take place.
- return Class.forName(klass.getName(), true, contextOrKafkaClassLoader);
+ return Utils.loadClass(trimmed, Object.class);
} else
throw new ConfigException(name, value, "Expected a Class instance or class name.");
default:
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
index c3083457ad..84289a0cd3 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
@@ -871,6 +871,18 @@ public boolean hasRoomFor(long timestamp, ByteBuffer key, ByteBuffer value, Head
return this.writeLimit >= estimatedBytesWritten() + recordSize;
}
+ /**
+ * Check if we have room for a given number of bytes.
+ */
+ public boolean hasRoomFor(int estimatedRecordsSize) {
+ if (isFull()) return false;
+ return this.writeLimit >= estimatedBytesWritten() + estimatedRecordsSize;
+ }
+
+ public int maxAllowedBytes() {
+ return this.writeLimit - this.batchHeaderSizeInBytes;
+ }
+
public boolean isClosed() {
return builtRecords != null;
}
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java
index fc996453d6..efdc7da2af 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java
@@ -47,8 +47,6 @@ public class ListOffsetsRequest extends AbstractRequest {
*/
public static final long EARLIEST_LOCAL_TIMESTAMP = -4L;
- public static final long LATEST_TIERED_TIMESTAMP = -5L;
-
public static final int CONSUMER_REPLICA_ID = -1;
public static final int DEBUGGING_REPLICA_ID = -2;
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index b25383aaad..75a024e197 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -94,7 +94,7 @@ private Utils() {}
// This matches URIs of formats: host:port and protocol://host:port
// IPv6 is supported with [ip] pattern
- private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(?:[a-zA-Z][a-zA-Z\\d+-.]*://)?\\[?([0-9a-zA-Z\\-._%:]+)\\]?:([0-9]+)$");
+ private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(?:[0-9a-zA-Z\\-%._]*://)?\\[?([0-9a-zA-Z\\-%._:]*)]?:([0-9]+)");
private static final Pattern VALID_HOST_CHARACTERS = Pattern.compile("([0-9a-zA-Z\\-%._:]*)");
@@ -451,7 +451,14 @@ public static T newInstance(String klass, Class base) throws ClassNotFoun
* @return the new class
*/
public static Class extends T> loadClass(String klass, Class base) throws ClassNotFoundException {
- return Class.forName(klass, true, Utils.getContextOrKafkaClassLoader()).asSubclass(base);
+ ClassLoader contextOrKafkaClassLoader = Utils.getContextOrKafkaClassLoader();
+ // Use loadClass here instead of Class.forName because the name we use here may be an alias
+ // and not match the name of the class that gets loaded. If that happens, Class.forName can
+ // throw an exception.
+ Class> loadedClass = contextOrKafkaClassLoader.loadClass(klass);
+ // Invoke forName here with the true name of the requested class to cause class
+ // initialization to take place.
+ return Class.forName(loadedClass.getName(), true, contextOrKafkaClassLoader).asSubclass(base);
}
/**
@@ -480,7 +487,7 @@ public static T newParameterizedInstance(String className, Object... params)
Class>[] argTypes = new Class>[params.length / 2];
Object[] args = new Object[params.length / 2];
try {
- Class> c = Class.forName(className, true, Utils.getContextOrKafkaClassLoader());
+ Class> c = Utils.loadClass(className, Object.class);
for (int i = 0; i < params.length / 2; i++) {
argTypes[i] = (Class>) params[2 * i];
args[i] = params[(2 * i) + 1];
diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java
index d6cdd14f36..86d1ddf5f4 100644
--- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java
+++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java
@@ -319,7 +319,7 @@ public List poll(long timeoutMs, long now) {
checkTimeoutOfPendingRequests(now);
// We skip metadata updates if all nodes are currently blacked out
- if (metadataUpdater.isUpdateNeeded() && leastLoadedNode(now) != null) {
+ if (metadataUpdater.isUpdateNeeded() && leastLoadedNode(now).node() != null) {
MetadataUpdate metadataUpdate = metadataUpdates.poll();
if (metadataUpdate != null) {
metadataUpdater.update(time, metadataUpdate);
@@ -588,13 +588,13 @@ public void close(String node) {
}
@Override
- public Node leastLoadedNode(long now) {
+ public LeastLoadedNode leastLoadedNode(long now) {
// Consistent with NetworkClient, we do not return nodes awaiting reconnect backoff
for (Node node : metadataUpdater.fetchNodes()) {
if (!connectionState(node.idString()).isBackingOff(now))
- return node;
+ return new LeastLoadedNode(node, true);
}
- return null;
+ return new LeastLoadedNode(null, false);
}
public void setWakeupHook(Runnable wakeupHook) {
diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
index cd3ec36f38..4369c8404e 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -128,7 +128,16 @@ public class NetworkClientTest {
private NetworkClient createNetworkClient(long reconnectBackoffMaxMs) {
return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE,
reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024,
- defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext());
+ defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(),
+ MetadataRecoveryStrategy.NONE);
+ }
+
+ private NetworkClient createNetworkClientWithMaxInFlightRequestsPerConnection(
+ int maxInFlightRequestsPerConnection, long reconnectBackoffMaxMs) {
+ return new NetworkClient(selector, metadataUpdater, "mock", maxInFlightRequestsPerConnection,
+ reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024,
+ defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(),
+ MetadataRecoveryStrategy.NONE);
}
private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoffMaxMs, long connectionSetupTimeoutMsTest, int nodeNumber) {
@@ -136,26 +145,30 @@ private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoff
TestMetadataUpdater metadataUpdater = new TestMetadataUpdater(nodes);
return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE,
reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024,
- defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext());
+ defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(),
+ MetadataRecoveryStrategy.NONE);
}
private NetworkClient createNetworkClientWithStaticNodes() {
return new NetworkClient(selector, metadataUpdater,
"mock-static", Integer.MAX_VALUE, 0, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs,
- connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext());
+ connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(),
+ MetadataRecoveryStrategy.NONE);
}
private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadata) {
return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE,
reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024,
- defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext());
+ defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(),
+ MetadataRecoveryStrategy.NONE);
}
private NetworkClient createNetworkClientWithNoVersionDiscovery() {
return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE,
reconnectBackoffMsTest, reconnectBackoffMaxMsTest,
64 * 1024, 64 * 1024, defaultRequestTimeoutMs,
- connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext());
+ connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(),
+ MetadataRecoveryStrategy.NONE);
}
@BeforeEach
@@ -698,14 +711,18 @@ private void sendThrottledProduceResponse(int correlationId, int throttleMs, sho
public void testLeastLoadedNode() {
client.ready(node, time.milliseconds());
assertFalse(client.isReady(node, time.milliseconds()));
- assertEquals(node, client.leastLoadedNode(time.milliseconds()));
+ LeastLoadedNode leastLoadedNode = client.leastLoadedNode(time.milliseconds());
+ assertEquals(node, leastLoadedNode.node());
+ assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady());
awaitReady(client, node);
client.poll(1, time.milliseconds());
assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready");
// leastloadednode should be our single node
- Node leastNode = client.leastLoadedNode(time.milliseconds());
+ leastLoadedNode = client.leastLoadedNode(time.milliseconds());
+ assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady());
+ Node leastNode = leastLoadedNode.node();
assertEquals(leastNode.id(), node.id(), "There should be one leastloadednode");
// sleep for longer than reconnect backoff
@@ -716,8 +733,29 @@ public void testLeastLoadedNode() {
client.poll(1, time.milliseconds());
assertFalse(client.ready(node, time.milliseconds()), "After we forced the disconnection the client is no longer ready.");
- leastNode = client.leastLoadedNode(time.milliseconds());
- assertNull(leastNode, "There should be NO leastloadednode");
+ leastLoadedNode = client.leastLoadedNode(time.milliseconds());
+ assertFalse(leastLoadedNode.hasNodeAvailableOrConnectionReady());
+ assertNull(leastLoadedNode.node(), "There should be NO leastloadednode");
+ }
+
+ @Test
+ public void testHasNodeAvailableOrConnectionReady() {
+ NetworkClient client = createNetworkClientWithMaxInFlightRequestsPerConnection(1, reconnectBackoffMaxMsTest);
+ awaitReady(client, node);
+
+ long now = time.milliseconds();
+ LeastLoadedNode leastLoadedNode = client.leastLoadedNode(now);
+ assertEquals(node, leastLoadedNode.node());
+ assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady());
+
+ MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.emptyList(), true);
+ ClientRequest request = client.newClientRequest(node.idString(), builder, now, true);
+ client.send(request, now);
+ client.poll(defaultRequestTimeoutMs, now);
+
+ leastLoadedNode = client.leastLoadedNode(now);
+ assertNull(leastLoadedNode.node());
+ assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady());
}
@Test
@@ -727,7 +765,7 @@ public void testLeastLoadedNodeProvideDisconnectedNodesPrioritizedByLastConnecti
Set providedNodeIds = new HashSet<>();
for (int i = 0; i < nodeNumber * 10; i++) {
- Node node = client.leastLoadedNode(time.milliseconds());
+ Node node = client.leastLoadedNode(time.milliseconds()).node();
assertNotNull(node, "Should provide a node");
providedNodeIds.add(node);
client.ready(node, time.milliseconds());
@@ -800,7 +838,7 @@ public void testLeastLoadedNodeConsidersThrottledConnections() {
client.poll(1, time.milliseconds());
// leastloadednode should return null since the node is throttled
- assertNull(client.leastLoadedNode(time.milliseconds()));
+ assertNull(client.leastLoadedNode(time.milliseconds()).node());
}
@Test
@@ -1046,7 +1084,8 @@ public void testReconnectAfterAddressChange() {
NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE,
reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024,
defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest,
- time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender);
+ time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender,
+ MetadataRecoveryStrategy.NONE);
// Connect to one the initial addresses, then change the addresses and disconnect
client.ready(node, time.milliseconds());
@@ -1106,7 +1145,8 @@ public void testFailedConnectionToFirstAddress() {
NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE,
reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024,
defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest,
- time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender);
+ time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender,
+ MetadataRecoveryStrategy.NONE);
// First connection attempt should fail
client.ready(node, time.milliseconds());
@@ -1158,7 +1198,8 @@ public void testFailedConnectionToFirstAddressAfterReconnect() {
NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE,
reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024,
defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest,
- time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender);
+ time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender,
+ MetadataRecoveryStrategy.NONE);
// Connect to one the initial addresses, then change the addresses and disconnect
client.ready(node, time.milliseconds());
@@ -1266,7 +1307,8 @@ public void testTelemetryRequest() {
NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE,
reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024,
defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest,
- time, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender);
+ time, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender,
+ MetadataRecoveryStrategy.NONE);
// Send the ApiVersionsRequest
client.ready(node, time.milliseconds());
diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientConfigTest.java
new file mode 100644
index 0000000000..92dc56fde4
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientConfigTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.admin;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
+import org.apache.kafka.common.config.ConfigException;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class AdminClientConfigTest {
+ @Test
+ public void testDefaultMetadataRecoveryStrategy() {
+ Map configs = new HashMap<>();
+ final AdminClientConfig adminClientConfig = new AdminClientConfig(configs);
+ assertEquals(MetadataRecoveryStrategy.NONE.name, adminClientConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG));
+ }
+
+ @Test
+ public void testInvalidMetadataRecoveryStrategy() {
+ Map configs = new HashMap<>();
+ configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc");
+ ConfigException ce = assertThrows(ConfigException.class, () -> new AdminClientConfig(configs));
+ assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG));
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
index ea1305e533..e04de635f7 100644
--- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
@@ -1464,6 +1464,7 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiBasic() {
assertEquals(0, topicDescription.partitions().get(0).partition());
assertEquals(1, topicDescription.partitions().get(1).partition());
topicDescription = topicDescriptions.get(topicName1);
+ assertNull(topicDescription.authorizedOperations());
assertEquals(1, topicDescription.partitions().size());
} catch (Exception e) {
fail("describe using DescribeTopics API should not fail", e);
@@ -1471,6 +1472,77 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiBasic() {
}
}
+ @Test
+ public void testDescribeTopicPartitionsApiWithAuthorizedOps() throws ExecutionException, InterruptedException {
+ try (AdminClientUnitTestEnv env = mockClientEnv()) {
+ env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
+ String topicName0 = "test-0";
+ Uuid topicId = Uuid.randomUuid();
+
+ int authorisedOperations = Utils.to32BitField(Utils.mkSet(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code()));
+ env.kafkaClient().prepareResponse(
+ prepareDescribeClusterResponse(0,
+ env.cluster().nodes(),
+ env.cluster().clusterResource().clusterId(),
+ 2,
+ authorisedOperations)
+ );
+
+ DescribeTopicPartitionsResponseData responseData = new DescribeTopicPartitionsResponseData();
+ responseData.topics().add(new DescribeTopicPartitionsResponseTopic()
+ .setErrorCode((short) 0)
+ .setTopicId(topicId)
+ .setName(topicName0)
+ .setIsInternal(false)
+ .setTopicAuthorizedOperations(authorisedOperations));
+ env.kafkaClient().prepareResponse(new DescribeTopicPartitionsResponse(responseData));
+
+ DescribeTopicsResult result = env.adminClient().describeTopics(
+ singletonList(topicName0), new DescribeTopicsOptions().includeAuthorizedOperations(true)
+ );
+
+ Map topicDescriptions = result.allTopicNames().get();
+ TopicDescription topicDescription = topicDescriptions.get(topicName0);
+ assertEquals(new HashSet<>(asList(AclOperation.DESCRIBE, AclOperation.ALTER)),
+ topicDescription.authorizedOperations());
+ }
+ }
+
+ @Test
+ public void testDescribeTopicPartitionsApiWithoutAuthorizedOps() throws ExecutionException, InterruptedException {
+ try (AdminClientUnitTestEnv env = mockClientEnv()) {
+ env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
+ String topicName0 = "test-0";
+ Uuid topicId = Uuid.randomUuid();
+
+ int authorisedOperations = Utils.to32BitField(Utils.mkSet(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code()));
+ env.kafkaClient().prepareResponse(
+ prepareDescribeClusterResponse(0,
+ env.cluster().nodes(),
+ env.cluster().clusterResource().clusterId(),
+ 2,
+ authorisedOperations)
+ );
+
+ DescribeTopicPartitionsResponseData responseData = new DescribeTopicPartitionsResponseData();
+ responseData.topics().add(new DescribeTopicPartitionsResponseTopic()
+ .setErrorCode((short) 0)
+ .setTopicId(topicId)
+ .setName(topicName0)
+ .setIsInternal(false)
+ .setTopicAuthorizedOperations(authorisedOperations));
+ env.kafkaClient().prepareResponse(new DescribeTopicPartitionsResponse(responseData));
+
+ DescribeTopicsResult result = env.adminClient().describeTopics(
+ singletonList(topicName0), new DescribeTopicsOptions().includeAuthorizedOperations(false)
+ );
+
+ Map topicDescriptions = result.allTopicNames().get();
+ TopicDescription topicDescription = topicDescriptions.get(topicName0);
+ assertNull(topicDescription.authorizedOperations());
+ }
+ }
+
@SuppressWarnings({"NPathComplexity", "CyclomaticComplexity"})
@Test
public void testDescribeTopicsWithDescribeTopicPartitionsApiEdgeCase() {
@@ -1547,6 +1619,7 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiEdgeCase() {
assertEquals(2, topicDescription.partitions().size());
topicDescription = topicDescriptions.get(topicName2);
assertEquals(2, topicDescription.partitions().size());
+ assertNull(topicDescription.authorizedOperations());
} catch (Exception e) {
fail("describe using DescribeTopics API should not fail", e);
}
diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java
index 34ed2e6772..62258289a9 100644
--- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java
@@ -16,6 +16,7 @@
*/
package org.apache.kafka.clients.admin.internals;
+import org.apache.kafka.clients.admin.FenceProducersOptions;
import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.message.InitProducerIdResponseData;
@@ -39,11 +40,21 @@
public class FenceProducersHandlerTest {
private final LogContext logContext = new LogContext();
private final Node node = new Node(1, "host", 1234);
+ private final int requestTimeoutMs = 30000;
+ private final FenceProducersOptions options = new FenceProducersOptions();
@Test
public void testBuildRequest() {
- FenceProducersHandler handler = new FenceProducersHandler(logContext);
- mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId));
+ FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs);
+ mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, requestTimeoutMs));
+ }
+
+ @Test
+ public void testBuildRequestOptionsTimeout() {
+ final int optionsTimeoutMs = 50000;
+ options.timeoutMs(optionsTimeoutMs);
+ FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs);
+ mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, optionsTimeoutMs));
}
@Test
@@ -51,7 +62,7 @@ public void testHandleSuccessfulResponse() {
String transactionalId = "foo";
CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
- FenceProducersHandler handler = new FenceProducersHandler(logContext);
+ FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs);
short epoch = 57;
long producerId = 7;
@@ -73,7 +84,7 @@ public void testHandleSuccessfulResponse() {
@Test
public void testHandleErrorResponse() {
String transactionalId = "foo";
- FenceProducersHandler handler = new FenceProducersHandler(logContext);
+ FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs);
assertFatalError(handler, transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED);
assertFatalError(handler, transactionalId, Errors.CLUSTER_AUTHORIZATION_FAILED);
assertFatalError(handler, transactionalId, Errors.UNKNOWN_SERVER_ERROR);
@@ -83,6 +94,7 @@ public void testHandleErrorResponse() {
assertRetriableError(handler, transactionalId, Errors.COORDINATOR_LOAD_IN_PROGRESS);
assertUnmappedKey(handler, transactionalId, Errors.NOT_COORDINATOR);
assertUnmappedKey(handler, transactionalId, Errors.COORDINATOR_NOT_AVAILABLE);
+ assertRetriableError(handler, transactionalId, Errors.CONCURRENT_TRANSACTIONS);
}
private void assertFatalError(
@@ -136,10 +148,10 @@ private ApiResult handleResponseError(
return result;
}
- private void assertLookup(FenceProducersHandler handler, String transactionalId) {
+ private void assertLookup(FenceProducersHandler handler, String transactionalId, int txnTimeoutMs) {
CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
InitProducerIdRequest.Builder request = handler.buildSingleRequest(1, key);
assertEquals(transactionalId, request.data.transactionalId());
- assertEquals(1, request.data.transactionTimeoutMs());
+ assertEquals(txnTimeoutMs, request.data.transactionTimeoutMs());
}
}
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java
index 8e9fa5722f..0fc8e6ca48 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java
@@ -17,6 +17,7 @@
package org.apache.kafka.clients.consumer;
import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.security.auth.SecurityProtocol;
@@ -191,6 +192,25 @@ public void testRemoteAssignorConfig() {
assertEquals(remoteAssignorName, consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG));
}
+ @Test
+ public void testDefaultMetadataRecoveryStrategy() {
+ Map configs = new HashMap<>();
+ configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass);
+ configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass);
+ final ConsumerConfig consumerConfig = new ConsumerConfig(configs);
+ assertEquals(MetadataRecoveryStrategy.NONE.name, consumerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG));
+ }
+
+ @Test
+ public void testInvalidMetadataRecoveryStrategy() {
+ Map configs = new HashMap<>();
+ configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass);
+ configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass);
+ configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc");
+ ConfigException ce = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs));
+ assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG));
+ }
+
@ParameterizedTest
@CsvSource({"consumer, true", "classic, true", "Consumer, true", "Classic, true", "invalid, false"})
public void testProtocolConfigValidation(String protocol, boolean isValid) {
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
index 850ac2bd8f..cfe4d03e6a 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
@@ -435,7 +435,7 @@ public void shouldIgnoreGroupInstanceIdForEmptyGroupId(GroupProtocol groupProtoc
}
@ParameterizedTest
- @EnumSource(GroupProtocol.class)
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testSubscription(GroupProtocol groupProtocol) {
consumer = newConsumer(groupProtocol, groupId);
@@ -495,7 +495,7 @@ public void testSubscriptionOnEmptyPattern(GroupProtocol groupProtocol) {
}
@ParameterizedTest
- @EnumSource(GroupProtocol.class)
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testSubscriptionWithEmptyPartitionAssignment(GroupProtocol groupProtocol) {
Properties props = new Properties();
props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name());
@@ -3227,7 +3227,7 @@ public void testUnusedConfigs(GroupProtocol groupProtocol) {
}
@ParameterizedTest
- @EnumSource(GroupProtocol.class)
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testAssignorNameConflict(GroupProtocol groupProtocol) {
Map configs = new HashMap<>();
configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name());
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java
index 66ee724a0e..09efcbbad9 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java
@@ -19,7 +19,6 @@
import org.apache.kafka.clients.Metadata.LeaderAndEpoch;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
-import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
@@ -30,7 +29,6 @@
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
-import org.apache.kafka.clients.consumer.RoundRobinAssignor;
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler;
import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent;
@@ -44,7 +42,6 @@
import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
import org.apache.kafka.clients.consumer.internals.events.EventProcessor;
import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent;
-import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseEvent;
import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent;
import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent;
import org.apache.kafka.clients.consumer.internals.events.PollEvent;
@@ -56,6 +53,7 @@
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.Node;
+import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.FencedInstanceIdException;
import org.apache.kafka.common.errors.GroupAuthorizationException;
@@ -105,10 +103,10 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
import java.util.stream.Stream;
import static java.util.Arrays.asList;
-import static java.util.Collections.emptySet;
import static java.util.Collections.singleton;
import static java.util.Collections.singletonList;
import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED;
@@ -131,7 +129,6 @@
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.atLeast;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
@@ -139,8 +136,10 @@
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.mockStatic;
import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.clearInvocations;
@SuppressWarnings("unchecked")
public class AsyncKafkaConsumerTest {
@@ -157,9 +156,15 @@ public class AsyncKafkaConsumerTest {
public void resetAll() {
backgroundEventQueue.clear();
if (consumer != null) {
- consumer.close(Duration.ZERO);
+ try {
+ consumer.close(Duration.ZERO);
+ } catch (Exception e) {
+ // best effort to clean up after each test, but may throw (ex. if callbacks where
+ // throwing errors)
+ }
}
consumer = null;
+
Mockito.framework().clearInlineMocks();
MockConsumerInterceptor.resetCounters();
}
@@ -205,7 +210,6 @@ private AsyncKafkaConsumer newConsumer(
ConsumerInterceptors interceptors,
ConsumerRebalanceListenerInvoker rebalanceListenerInvoker,
SubscriptionState subscriptions,
- List assignors,
String groupId,
String clientId) {
long retryBackoffMs = 100L;
@@ -228,7 +232,6 @@ private AsyncKafkaConsumer newConsumer(
metadata,
retryBackoffMs,
defaultApiTimeoutMs,
- assignors,
groupId,
autoCommitEnabled);
}
@@ -236,6 +239,7 @@ private AsyncKafkaConsumer newConsumer(
@Test
public void testSuccessfulStartupShutdown() {
consumer = newConsumer();
+ completeUnsubscribeApplicationEventSuccessfully();
assertDoesNotThrow(() -> consumer.close());
}
@@ -248,6 +252,7 @@ public void testInvalidGroupId() {
@Test
public void testFailOnClosedConsumer() {
consumer = newConsumer();
+ completeUnsubscribeApplicationEventSuccessfully();
consumer.close();
final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment);
assertEquals("This consumer has already been closed.", res.getMessage());
@@ -480,6 +485,44 @@ public void onPartitionsAssigned(final Collection partitions) {
assertTrue(callbackExecuted.get());
}
+ @Test
+ public void testSubscriptionRegexEvalOnPollOnlyIfMetadataChanges() {
+ SubscriptionState subscriptions = mock(SubscriptionState.class);
+ Cluster cluster = mock(Cluster.class);
+
+ consumer = newConsumer(
+ mock(FetchBuffer.class),
+ mock(ConsumerInterceptors.class),
+ mock(ConsumerRebalanceListenerInvoker.class),
+ subscriptions,
+ "group-id",
+ "client-id");
+
+ final String topicName = "foo";
+ final int partition = 3;
+ final TopicPartition tp = new TopicPartition(topicName, partition);
+ doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+ Map offsets = Collections.singletonMap(tp, new OffsetAndMetadata(1));
+ completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets);
+ doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+ doReturn(cluster).when(metadata).fetch();
+ doReturn(Collections.singleton(topicName)).when(cluster).topics();
+
+ consumer.subscribe(Pattern.compile("f*"));
+ verify(metadata).requestUpdateForNewTopics();
+ verify(subscriptions).matchesSubscribedPattern(topicName);
+ clearInvocations(subscriptions);
+
+ when(subscriptions.hasPatternSubscription()).thenReturn(true);
+ consumer.poll(Duration.ZERO);
+ verify(subscriptions, never()).matchesSubscribedPattern(topicName);
+
+ when(metadata.updateVersion()).thenReturn(2);
+ when(subscriptions.hasPatternSubscription()).thenReturn(true);
+ consumer.poll(Duration.ZERO);
+ verify(subscriptions).matchesSubscribedPattern(topicName);
+ }
+
@Test
public void testClearWakeupTriggerAfterPoll() {
consumer = newConsumer();
@@ -537,6 +580,151 @@ public void testPollLongThrowsException() {
"This method is deprecated and will be removed in the next major release.", e.getMessage());
}
+ @Test
+ public void testOffsetFetchStoresPendingEvent() {
+ consumer = newConsumer();
+ long timeoutMs = 0;
+ doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+ consumer.assign(Collections.singleton(new TopicPartition("topic1", 0)));
+
+ // The first attempt at poll() creates an event, enqueues it, but its Future does not complete within the
+ // timeout, leaving a pending fetch.
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class));
+ CompletableApplicationEvent> event = getLastEnqueuedEvent();
+ assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event.future(), time.timer(timeoutMs)));
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+
+ clearInvocations(applicationEventHandler);
+
+ // For the second attempt, the event is reused, so first verify that another FetchCommittedOffsetsEvent
+ // was not enqueued. On this attempt the Future returns successfully, clearing the pending fetch.
+ event.future().complete(Collections.emptyMap());
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler, never()).add(any(FetchCommittedOffsetsEvent.class));
+ assertDoesNotThrow(() -> ConsumerUtils.getResult(event.future(), time.timer(timeoutMs)));
+ assertFalse(consumer.hasPendingOffsetFetchEvent());
+ }
+
+ @Test
+ public void testOffsetFetchDoesNotReuseMismatchedPendingEvent() {
+ consumer = newConsumer();
+ long timeoutMs = 0;
+ doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+
+ // The first attempt at poll() retrieves data for partition 0 of the topic. poll() creates an event,
+ // enqueues it, but its Future does not complete within the timeout, leaving a pending fetch.
+ consumer.assign(Collections.singleton(new TopicPartition("topic1", 0)));
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class));
+ CompletableApplicationEvent> event1 = getLastEnqueuedEvent();
+ assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event1.future(), time.timer(timeoutMs)));
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+
+ clearInvocations(applicationEventHandler);
+
+ // For the second attempt, the set of partitions is reassigned, causing the pending offset to be replaced.
+ // Verify that another FetchCommittedOffsetsEvent is enqueued.
+ consumer.assign(Collections.singleton(new TopicPartition("topic1", 1)));
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class));
+ CompletableApplicationEvent> event2 = getLastEnqueuedEvent();
+ assertNotEquals(event1, event2);
+ assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future(), time.timer(timeoutMs)));
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+
+ clearInvocations(applicationEventHandler);
+
+ // For the third attempt, the event from attempt 2 is reused, so there should not have been another
+ // FetchCommittedOffsetsEvent enqueued. The Future is completed to make it return successfully in poll().
+ // This will finally clear out the pending fetch.
+ event2.future().complete(Collections.emptyMap());
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler, never()).add(any(FetchCommittedOffsetsEvent.class));
+ assertDoesNotThrow(() -> ConsumerUtils.getResult(event2.future(), time.timer(timeoutMs)));
+ assertFalse(consumer.hasPendingOffsetFetchEvent());
+ }
+
+ @Test
+ public void testOffsetFetchDoesNotReuseExpiredPendingEvent() {
+ consumer = newConsumer();
+ long timeoutMs = 0;
+ doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+ consumer.assign(Collections.singleton(new TopicPartition("topic1", 0)));
+
+ // The first attempt at poll() creates an event, enqueues it, but its Future does not complete within
+ // the timeout, leaving a pending fetch.
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class));
+ CompletableApplicationEvent> event1 = getLastEnqueuedEvent();
+ assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event1.future(), time.timer(timeoutMs)));
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+
+ clearInvocations(applicationEventHandler);
+
+ // Sleep past the event's expiration, causing the poll() to *not* reuse the pending fetch. A new event
+ // is created and added to the application event queue.
+ time.sleep(event1.deadlineMs() - time.milliseconds());
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class));
+ CompletableApplicationEvent> event2 = getLastEnqueuedEvent();
+ assertNotEquals(event1, event2);
+ assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future(), time.timer(timeoutMs)));
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+ }
+
+ @Test
+ public void testOffsetFetchTimeoutExceptionKeepsPendingEvent() {
+ consumer = newConsumer();
+ long timeoutMs = 0;
+ doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+ consumer.assign(Collections.singleton(new TopicPartition("topic1", 0)));
+
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class));
+ CompletableApplicationEvent> event = getLastEnqueuedEvent();
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+
+ event.future().completeExceptionally(new TimeoutException("Test error"));
+ assertDoesNotThrow(() -> consumer.poll(Duration.ofMillis(timeoutMs)));
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+ }
+
+ @Test
+ public void testOffsetFetchInterruptExceptionKeepsPendingEvent() {
+ consumer = newConsumer();
+ long timeoutMs = 0;
+ doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+ consumer.assign(Collections.singleton(new TopicPartition("topic1", 0)));
+
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class));
+ CompletableApplicationEvent> event = getLastEnqueuedEvent();
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+
+ event.future().completeExceptionally(new InterruptException("Test error"));
+ assertThrows(InterruptException.class, () -> consumer.poll(Duration.ofMillis(timeoutMs)));
+ assertTrue(Thread.interrupted());
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+ }
+
+ @Test
+ public void testOffsetFetchUnexpectedExceptionClearsPendingEvent() {
+ consumer = newConsumer();
+ long timeoutMs = 0;
+ doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+ consumer.assign(Collections.singleton(new TopicPartition("topic1", 0)));
+
+ consumer.poll(Duration.ofMillis(timeoutMs));
+ verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class));
+ CompletableApplicationEvent> event = getLastEnqueuedEvent();
+ assertTrue(consumer.hasPendingOffsetFetchEvent());
+
+ event.future().completeExceptionally(new NullPointerException("Test error"));
+ assertThrows(KafkaException.class, () -> consumer.poll(Duration.ofMillis(timeoutMs)));
+ assertFalse(consumer.hasPendingOffsetFetchEvent());
+ }
+
@Test
public void testCommitSyncLeaderEpochUpdate() {
consumer = newConsumer();
@@ -564,7 +752,6 @@ public void testCommitAsyncLeaderEpochUpdate() {
new ConsumerInterceptors<>(Collections.emptyList()),
mock(ConsumerRebalanceListenerInvoker.class),
subscriptions,
- singletonList(new RoundRobinAssignor()),
"group-id",
"client-id");
completeCommitSyncApplicationEventSuccessfully();
@@ -705,17 +892,17 @@ private CompletableFuture setUpConsumerWithIncompleteAsyncCommit(TopicPart
consumer.seek(tp, 20);
consumer.commitAsync();
- return getLastEnqueuedEventFuture();
+ CompletableApplicationEvent event = getLastEnqueuedEvent();
+ return event.future();
}
// ArgumentCaptor's type-matching does not work reliably with Java 8, so we cannot directly capture the AsyncCommitEvent
// Instead, we capture the super-class CompletableApplicationEvent and fetch the last captured event.
- private CompletableFuture getLastEnqueuedEventFuture() {
- final ArgumentCaptor> eventArgumentCaptor = ArgumentCaptor.forClass(CompletableApplicationEvent.class);
+ private CompletableApplicationEvent getLastEnqueuedEvent() {
+ final ArgumentCaptor> eventArgumentCaptor = ArgumentCaptor.forClass(CompletableApplicationEvent.class);
verify(applicationEventHandler, atLeast(1)).add(eventArgumentCaptor.capture());
- final List> allValues = eventArgumentCaptor.getAllValues();
- final CompletableApplicationEvent lastEvent = allValues.get(allValues.size() - 1);
- return lastEvent.future();
+ final List> allValues = eventArgumentCaptor.getAllValues();
+ return allValues.get(allValues.size() - 1);
}
@Test
@@ -758,6 +945,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() {
@Test
public void testEnsureShutdownExecutedCommitAsyncCallbacks() {
consumer = newConsumer();
+ completeUnsubscribeApplicationEventSuccessfully();
MockCommitCallback callback = new MockCommitCallback();
completeCommitAsyncApplicationEventSuccessfully();
assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback));
@@ -769,70 +957,45 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() {
@Test
public void testVerifyApplicationEventOnShutdown() {
consumer = newConsumer();
+ completeUnsubscribeApplicationEventSuccessfully();
doReturn(null).when(applicationEventHandler).addAndGet(any());
consumer.close();
- verify(applicationEventHandler).addAndGet(any(LeaveOnCloseEvent.class));
+ verify(applicationEventHandler).add(any(UnsubscribeEvent.class));
verify(applicationEventHandler).add(any(CommitOnCloseEvent.class));
}
@Test
- public void testPartitionRevocationOnClose() {
- MockRebalanceListener listener = new MockRebalanceListener();
- SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
- consumer = newConsumer(
+ public void testUnsubscribeOnClose() {
+ SubscriptionState subscriptions = mock(SubscriptionState.class);
+ consumer = spy(newConsumer(
mock(FetchBuffer.class),
mock(ConsumerInterceptors.class),
mock(ConsumerRebalanceListenerInvoker.class),
subscriptions,
- singletonList(new RoundRobinAssignor()),
"group-id",
- "client-id");
-
- consumer.subscribe(singleton("topic"), listener);
- subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0)));
+ "client-id"));
+ completeUnsubscribeApplicationEventSuccessfully();
consumer.close(Duration.ZERO);
- assertTrue(subscriptions.assignedPartitions().isEmpty());
- assertEquals(1, listener.revokedCount);
+ verifyUnsubscribeEvent(subscriptions);
}
@Test
public void testFailedPartitionRevocationOnClose() {
- // If rebalance listener failed to execute during close, we will skip sending leave group and proceed with
- // closing the consumer.
- ConsumerRebalanceListener listener = mock(ConsumerRebalanceListener.class);
- SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
- consumer = newConsumer(
+ // If rebalance listener failed to execute during close, we still send the leave group,
+ // and proceed with closing the consumer.
+ SubscriptionState subscriptions = mock(SubscriptionState.class);
+ consumer = spy(newConsumer(
mock(FetchBuffer.class),
new ConsumerInterceptors<>(Collections.emptyList()),
mock(ConsumerRebalanceListenerInvoker.class),
subscriptions,
- singletonList(new RoundRobinAssignor()),
"group-id",
- "client-id");
- subscriptions.subscribe(singleton("topic"), Optional.of(listener));
- TopicPartition tp = new TopicPartition("topic", 0);
- subscriptions.assignFromSubscribed(singleton(tp));
- doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp)));
+ "client-id"));
+ doThrow(new KafkaException()).when(consumer).processBackgroundEvents(any(), any());
assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO));
- verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseEvent.class));
- verify(listener).onPartitionsRevoked(eq(singleton(tp)));
- assertEquals(emptySet(), subscriptions.assignedPartitions());
- }
-
- @Test
- public void testCompleteQuietly() {
- AtomicReference exception = new AtomicReference<>();
- CompletableFuture future = CompletableFuture.completedFuture(null);
- consumer = newConsumer();
- assertDoesNotThrow(() -> consumer.completeQuietly(() -> {
- future.get(0, TimeUnit.MILLISECONDS);
- }, "test", exception));
- assertNull(exception.get());
-
- assertDoesNotThrow(() -> consumer.completeQuietly(() -> {
- throw new KafkaException("Test exception");
- }, "test", exception));
- assertInstanceOf(KafkaException.class, exception.get());
+ verifyUnsubscribeEvent(subscriptions);
+ // Close operation should carry on even if the unsubscribe fails
+ verify(applicationEventHandler).close(any(Duration.class));
}
@Test
@@ -844,13 +1007,12 @@ public void testAutoCommitSyncEnabled() {
mock(ConsumerInterceptors.class),
mock(ConsumerRebalanceListenerInvoker.class),
subscriptions,
- singletonList(new RoundRobinAssignor()),
"group-id",
"client-id");
consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class));
subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0)));
subscriptions.seek(new TopicPartition("topic", 0), 100);
- consumer.autoCommitSync(time.timer(100));
+ consumer.commitSyncAllConsumed(time.timer(100));
verify(applicationEventHandler).add(any(SyncCommitEvent.class));
}
@@ -862,7 +1024,6 @@ public void testAutoCommitSyncDisabled() {
mock(ConsumerInterceptors.class),
mock(ConsumerRebalanceListenerInvoker.class),
subscriptions,
- singletonList(new RoundRobinAssignor()),
"group-id",
"client-id");
consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class));
@@ -1135,7 +1296,7 @@ public void testNoWakeupInCloseCommit() {
}
return null;
}).when(applicationEventHandler).add(any());
-
+ completeUnsubscribeApplicationEventSuccessfully();
consumer.close(Duration.ZERO);
// A commit was triggered and not completed exceptionally by the wakeup
@@ -1173,6 +1334,7 @@ public void testCloseAwaitPendingAsyncCommitComplete() {
completeCommitAsyncApplicationEventSuccessfully();
consumer.commitAsync(cb);
+ completeUnsubscribeApplicationEventSuccessfully();
assertDoesNotThrow(() -> consumer.close(Duration.ofMillis(10)));
assertEquals(1, cb.invoked);
}
@@ -1187,6 +1349,7 @@ public void testInterceptorAutoCommitOnClose() {
consumer = newConsumer(props);
assertEquals(1, MockConsumerInterceptor.INIT_COUNT.get());
completeCommitSyncApplicationEventSuccessfully();
+ completeUnsubscribeApplicationEventSuccessfully();
consumer.close(Duration.ZERO);
@@ -1624,6 +1787,18 @@ public void testGroupRemoteAssignorUsedInConsumerProtocol() {
assertFalse(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG));
}
+ @Test
+ public void testPartitionAssignmentStrategyUnusedInAsyncConsumer() {
+ final Properties props = requiredConsumerConfig();
+ props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroup1");
+ props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT));
+ props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "CooperativeStickyAssignor");
+ final ConsumerConfig config = new ConsumerConfig(props);
+ consumer = newConsumer(config);
+
+ assertTrue(config.unused().contains(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG));
+ }
+
@Test
public void testGroupIdNull() {
final Properties props = requiredConsumerConfig();
@@ -1666,7 +1841,6 @@ public void testEnsurePollEventSentOnConsumerPoll() {
new ConsumerInterceptors<>(Collections.emptyList()),
mock(ConsumerRebalanceListenerInvoker.class),
subscriptions,
- singletonList(new RoundRobinAssignor()),
"group-id",
"client-id");
final TopicPartition tp = new TopicPartition("topic", 0);
@@ -1713,13 +1887,13 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co
if (committedOffsetsEnabled) {
// Verify there was an FetchCommittedOffsets event and no ResetPositions event
verify(applicationEventHandler, atLeast(1))
- .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class));
+ .add(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class));
verify(applicationEventHandler, never())
.addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class));
} else {
// Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions
verify(applicationEventHandler, never())
- .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class));
+ .add(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class));
verify(applicationEventHandler, atLeast(1))
.addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class));
}
@@ -1738,7 +1912,7 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions,
verify(applicationEventHandler, atLeast(1))
.addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class));
verify(applicationEventHandler, atLeast(1))
- .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class));
+ .add(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class));
verify(applicationEventHandler, atLeast(1))
.addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class));
}
@@ -1880,6 +2054,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() {
@Test
void testReaperInvokedInClose() {
consumer = newConsumer();
+ completeUnsubscribeApplicationEventSuccessfully();
consumer.close();
verify(backgroundEventReaper).reap(backgroundEventQueue);
}
@@ -1901,6 +2076,18 @@ void testReaperInvokedInPoll() {
verify(backgroundEventReaper).reap(time.milliseconds());
}
+ private void verifyUnsubscribeEvent(SubscriptionState subscriptions) {
+ // Check that an unsubscribe event was generated, and that the consumer waited for it to
+ // complete processing background events.
+ verify(applicationEventHandler).add(any(UnsubscribeEvent.class));
+ verify(consumer).processBackgroundEvents(any(), any());
+
+ // The consumer should not clear the assignment in the app thread. The unsubscribe
+ // event is the one responsible for updating the assignment in the background when it
+ // completes.
+ verify(subscriptions, never()).assignFromSubscribed(any());
+ }
+
private Map mockTopicPartitionOffset() {
final TopicPartition t0 = new TopicPartition("t0", 2);
final TopicPartition t1 = new TopicPartition("t0", 3);
@@ -1964,6 +2151,12 @@ private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Ma
doReturn(committedOffsets)
.when(applicationEventHandler)
.addAndGet(any(FetchCommittedOffsetsEvent.class));
+
+ doAnswer(invocation -> {
+ FetchCommittedOffsetsEvent event = invocation.getArgument(0);
+ event.future().complete(committedOffsets);
+ return null;
+ }).when(applicationEventHandler).add(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class));
}
private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) {
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java
index b1db0297a1..8e61d61cbc 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java
@@ -193,11 +193,11 @@ public void testPollEnsureCorrectInflightRequestBufferSize() {
offsets2.put(new TopicPartition("test", 4), new OffsetAndMetadata(20L));
// Add the requests to the CommitRequestManager and store their futures
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
- commitManager.commitSync(offsets1, expirationTimeMs);
- commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)), expirationTimeMs);
- commitManager.commitSync(offsets2, expirationTimeMs);
- commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 1)), expirationTimeMs);
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
+ commitManager.commitSync(offsets1, deadlineMs);
+ commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)), deadlineMs);
+ commitManager.commitSync(offsets2, deadlineMs);
+ commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 1)), deadlineMs);
// Poll the CommitRequestManager and verify that the inflightOffsetFetches size is correct
NetworkClientDelegate.PollResult result = commitManager.poll(time.milliseconds());
@@ -287,8 +287,8 @@ public void testCommitSyncRetriedAfterExpectedRetriableException(Errors error) {
Map offsets = Collections.singletonMap(
new TopicPartition("topic", 1),
new OffsetAndMetadata(0));
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
- CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs);
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
+ CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried(commitRequestManager, error, commitResult);
// We expect that request should have been retried on this sync commit.
@@ -307,8 +307,8 @@ public void testCommitSyncFailsWithExpectedException(Errors commitError,
new OffsetAndMetadata(0));
// Send sync offset commit that fails and verify it propagates the expected exception.
- long expirationTimeMs = time.milliseconds() + retryBackoffMs;
- CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs);
+ long deadlineMs = time.milliseconds() + retryBackoffMs;
+ CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
completeOffsetCommitRequestWithError(commitRequestManager, commitError);
assertFutureThrows(commitResult, expectedException);
}
@@ -332,8 +332,8 @@ public void testCommitSyncFailsWithCommitFailedExceptionIfUnknownMemberId() {
Map offsets = Collections.singletonMap(
new TopicPartition("topic", 1),
new OffsetAndMetadata(0));
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
- CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs);
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
+ CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
completeOffsetCommitRequestWithError(commitRequestManager, Errors.UNKNOWN_MEMBER_ID);
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
@@ -594,7 +594,7 @@ public void testOffsetFetchRequestEnsureDuplicatedRequestSucceed() {
@ParameterizedTest
@MethodSource("offsetFetchExceptionSupplier")
- public void testOffsetFetchRequestErroredRequests(final Errors error, final boolean isRetriable) {
+ public void testOffsetFetchRequestErroredRequests(final Errors error) {
CommitRequestManager commitRequestManager = create(true, 100);
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
@@ -606,7 +606,7 @@ public void testOffsetFetchRequestErroredRequests(final Errors error, final bool
1,
error);
// we only want to make sure to purge the outbound buffer for non-retriables, so retriable will be re-queued.
- if (isRetriable)
+ if (isRetriableOnOffsetFetch(error))
testRetriable(commitRequestManager, futures);
else {
testNonRetriable(futures);
@@ -614,15 +614,49 @@ public void testOffsetFetchRequestErroredRequests(final Errors error, final bool
}
}
+ @ParameterizedTest
+ @MethodSource("offsetFetchExceptionSupplier")
+ public void testOffsetFetchRequestTimeoutRequests(final Errors error) {
+ CommitRequestManager commitRequestManager = create(true, 100);
+ when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
+
+ Set partitions = new HashSet<>();
+ partitions.add(new TopicPartition("t1", 0));
+ List>> futures = sendAndVerifyDuplicatedOffsetFetchRequests(
+ commitRequestManager,
+ partitions,
+ 1,
+ error);
+
+ if (isRetriableOnOffsetFetch(error)) {
+ futures.forEach(f -> assertFalse(f.isDone()));
+
+ // Insert a long enough sleep to force a timeout of the operation. Invoke poll() again so that each
+ // OffsetFetchRequestState is evaluated via isExpired().
+ time.sleep(defaultApiTimeoutMs);
+ assertFalse(commitRequestManager.pendingRequests.unsentOffsetFetches.isEmpty());
+ commitRequestManager.poll(time.milliseconds());
+ futures.forEach(f -> assertFutureThrows(f, TimeoutException.class));
+ assertTrue(commitRequestManager.pendingRequests.unsentOffsetFetches.isEmpty());
+ } else {
+ futures.forEach(f -> assertFutureThrows(f, KafkaException.class));
+ assertEmptyPendingRequests(commitRequestManager);
+ }
+ }
+
+ private boolean isRetriableOnOffsetFetch(Errors error) {
+ return error == Errors.NOT_COORDINATOR || error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.COORDINATOR_NOT_AVAILABLE;
+ }
+
@Test
public void testSuccessfulOffsetFetch() {
CommitRequestManager commitManager = create(false, 100);
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
CompletableFuture> fetchResult =
commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)),
- expirationTimeMs);
+ deadlineMs);
// Send fetch request
NetworkClientDelegate.PollResult result = commitManager.poll(time.milliseconds());
@@ -667,8 +701,8 @@ public void testOffsetFetchMarksCoordinatorUnknownOnRetriableCoordinatorErrors(E
Set partitions = new HashSet<>();
partitions.add(new TopicPartition("t1", 0));
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
- CompletableFuture> result = commitRequestManager.fetchOffsets(partitions, expirationTimeMs);
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
+ CompletableFuture> result = commitRequestManager.fetchOffsets(partitions, deadlineMs);
completeOffsetFetchRequestWithError(commitRequestManager, partitions, error);
@@ -694,8 +728,8 @@ public void testOffsetFetchMarksCoordinatorUnknownOnCoordinatorDisconnectedAndRe
Set partitions = new HashSet<>();
partitions.add(new TopicPartition("t1", 0));
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
- CompletableFuture> result = commitRequestManager.fetchOffsets(partitions, expirationTimeMs);
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
+ CompletableFuture> result = commitRequestManager.fetchOffsets(partitions, deadlineMs);
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
assertEquals(1, res.unsentRequests.size());
@@ -748,8 +782,8 @@ public void testOffsetCommitSyncTimeoutNotReturnedOnPollAndFails() {
new OffsetAndMetadata(0));
// Send sync offset commit request that fails with retriable error.
- long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2;
- CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs);
+ long deadlineMs = time.milliseconds() + retryBackoffMs * 2;
+ CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
completeOffsetCommitRequestWithError(commitRequestManager, Errors.REQUEST_TIMED_OUT);
// Request retried after backoff, and fails with retriable again. Should not complete yet
@@ -770,8 +804,9 @@ public void testOffsetCommitSyncTimeoutNotReturnedOnPollAndFails() {
* Sync commit requests that fail with an expected retriable error should be retried
* while there is time. When time expires, they should fail with a TimeoutException.
*/
- @Test
- public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExpires() {
+ @ParameterizedTest
+ @MethodSource("offsetCommitExceptionSupplier")
+ public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExpires(final Errors error) {
CommitRequestManager commitRequestManager = create(false, 100);
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
@@ -780,17 +815,21 @@ public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExp
new OffsetAndMetadata(0));
// Send offset commit request that fails with retriable error.
- long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2;
- CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs);
- completeOffsetCommitRequestWithError(commitRequestManager, Errors.COORDINATOR_NOT_AVAILABLE);
+ long deadlineMs = time.milliseconds() + retryBackoffMs * 2;
+ CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs);
+ completeOffsetCommitRequestWithError(commitRequestManager, error);
// Sleep to expire the request timeout. Request should fail on the next poll with a
// TimeoutException.
- time.sleep(expirationTimeMs);
+ time.sleep(deadlineMs);
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
assertEquals(0, res.unsentRequests.size());
assertTrue(commitResult.isDone());
- assertFutureThrows(commitResult, TimeoutException.class);
+
+ if (error.exception() instanceof RetriableException)
+ assertFutureThrows(commitResult, TimeoutException.class);
+ else
+ assertFutureThrows(commitResult, KafkaException.class);
}
/**
@@ -829,8 +868,8 @@ public void testEnsureBackoffRetryOnOffsetCommitRequestTimeout() {
Map offsets = Collections.singletonMap(new TopicPartition("topic", 1),
new OffsetAndMetadata(0));
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
- commitRequestManager.commitSync(offsets, expirationTimeMs);
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
+ commitRequestManager.commitSync(offsets, deadlineMs);
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
assertEquals(1, res.unsentRequests.size());
res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException());
@@ -911,8 +950,8 @@ public void testSyncOffsetFetchFailsWithStaleEpochAndRetriesWithNewEpoch() {
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
// Send request that is expected to fail with invalid epoch.
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
- commitRequestManager.fetchOffsets(partitions, expirationTimeMs);
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
+ commitRequestManager.fetchOffsets(partitions, deadlineMs);
// Mock member has new a valid epoch.
int newEpoch = 8;
@@ -950,9 +989,9 @@ public void testSyncOffsetFetchFailsWithStaleEpochAndNotRetriedIfMemberNotInGrou
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode));
// Send request that is expected to fail with invalid epoch.
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
CompletableFuture> requestResult =
- commitRequestManager.fetchOffsets(partitions, expirationTimeMs);
+ commitRequestManager.fetchOffsets(partitions, deadlineMs);
// Mock member not having a valid epoch anymore (left/failed/fenced).
commitRequestManager.onMemberEpochUpdated(Optional.empty(), Optional.empty());
@@ -983,10 +1022,10 @@ public void testAutoCommitSyncBeforeRevocationRetriesOnRetriableAndStaleEpoch(Er
TopicPartition tp = new TopicPartition("topic", 1);
subscriptionState.assignFromUser(singleton(tp));
subscriptionState.seek(tp, 5);
- long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2;
+ long deadlineMs = time.milliseconds() + retryBackoffMs * 2;
// Send commit request expected to be retried on STALE_MEMBER_EPOCH error while it does not expire
- commitRequestManager.maybeAutoCommitSyncBeforeRevocation(expirationTimeMs);
+ commitRequestManager.maybeAutoCommitSyncBeforeRevocation(deadlineMs);
int newEpoch = 8;
String memberId = "member1";
@@ -1094,7 +1133,7 @@ private void testNonRetriable(final List offsetCommitExceptionSupplier() {
return Stream.of(
@@ -1113,25 +1152,27 @@ private static Stream offsetCommitExceptionSupplier() {
Arguments.of(Errors.UNKNOWN_MEMBER_ID));
}
- // Supplies (error, isRetriable)
+ /**
+ * @return {@link Errors} that could be received in {@link ApiKeys#OFFSET_FETCH} responses.
+ */
private static Stream offsetFetchExceptionSupplier() {
- // fetchCommit is only retrying on a subset of RetriableErrors
return Stream.of(
- Arguments.of(Errors.NOT_COORDINATOR, true),
- Arguments.of(Errors.COORDINATOR_LOAD_IN_PROGRESS, true),
- Arguments.of(Errors.UNKNOWN_SERVER_ERROR, false),
- Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, false),
- Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE, false),
- Arguments.of(Errors.INVALID_COMMIT_OFFSET_SIZE, false),
- Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION, false),
- Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE, true),
- Arguments.of(Errors.REQUEST_TIMED_OUT, false),
- Arguments.of(Errors.FENCED_INSTANCE_ID, false),
- Arguments.of(Errors.TOPIC_AUTHORIZATION_FAILED, false),
- Arguments.of(Errors.UNKNOWN_MEMBER_ID, false),
+ Arguments.of(Errors.NOT_COORDINATOR),
+ Arguments.of(Errors.COORDINATOR_LOAD_IN_PROGRESS),
+ Arguments.of(Errors.UNKNOWN_SERVER_ERROR),
+ Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED),
+ Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE),
+ Arguments.of(Errors.INVALID_COMMIT_OFFSET_SIZE),
+ Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION),
+ Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE),
+ Arguments.of(Errors.REQUEST_TIMED_OUT),
+ Arguments.of(Errors.FENCED_INSTANCE_ID),
+ Arguments.of(Errors.TOPIC_AUTHORIZATION_FAILED),
+ Arguments.of(Errors.UNKNOWN_MEMBER_ID),
// Adding STALE_MEMBER_EPOCH as non-retriable here because it is only retried if a new
// member epoch is received. Tested separately.
- Arguments.of(Errors.STALE_MEMBER_EPOCH, false));
+ Arguments.of(Errors.STALE_MEMBER_EPOCH),
+ Arguments.of(Errors.UNSTABLE_OFFSET_COMMIT));
}
/**
@@ -1155,9 +1196,9 @@ public void testOffsetFetchRequestPartitionDataError(final Errors error, final b
TopicPartition tp2 = new TopicPartition("t2", 3);
partitions.add(tp1);
partitions.add(tp2);
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
CompletableFuture> future =
- commitRequestManager.fetchOffsets(partitions, expirationTimeMs);
+ commitRequestManager.fetchOffsets(partitions, deadlineMs);
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
assertEquals(1, res.unsentRequests.size());
@@ -1215,9 +1256,9 @@ private List>> sendAndV
int numRequest,
final Errors error) {
List>> futures = new ArrayList<>();
- long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs;
+ long deadlineMs = time.milliseconds() + defaultApiTimeoutMs;
for (int i = 0; i < numRequest; i++) {
- futures.add(commitRequestManager.fetchOffsets(partitions, expirationTimeMs));
+ futures.add(commitRequestManager.fetchOffsets(partitions, deadlineMs));
}
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java
index 8c3f97dd64..b6780fd427 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java
@@ -339,6 +339,27 @@ void testRunOnceInvokesReaper() {
verify(applicationEventReaper).reap(any(Long.class));
}
+ @Test
+ void testSendUnsentRequest() {
+ String groupId = "group-id";
+ NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest(
+ new FindCoordinatorRequest.Builder(
+ new FindCoordinatorRequestData()
+ .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id())
+ .setKey(groupId)),
+ Optional.empty());
+
+ networkClient.add(request);
+ assertTrue(networkClient.hasAnyPendingRequests());
+ assertFalse(networkClient.unsentRequests().isEmpty());
+ assertFalse(client.hasInFlightRequests());
+ consumerNetworkThread.cleanup();
+
+ assertTrue(networkClient.unsentRequests().isEmpty());
+ assertFalse(client.hasInFlightRequests());
+ assertFalse(networkClient.hasAnyPendingRequests());
+ }
+
private void prepareOffsetCommitRequest(final Map expectedOffsets,
final Errors error,
final boolean disconnected) {
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java
index 9f6fd4a764..dabd697b89 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java
@@ -174,7 +174,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA
backgroundEventHandler,
logContext));
- this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, config));
+ this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, time, config));
if (groupInfo.isPresent()) {
GroupInformation gi = groupInfo.get();
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java
index eaabcb8f81..204f87e1c1 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java
@@ -21,6 +21,7 @@
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.KafkaClient;
import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.clients.MockClient;
import org.apache.kafka.clients.NetworkClient;
import org.apache.kafka.clients.NodeApiVersions;
@@ -369,7 +370,7 @@ public void testFetcherCloseClosesFetchSessionsInBroker() {
// NOTE: by design the FetchRequestManager doesn't perform network I/O internally. That means that calling
// the close() method with a Timer will NOT send out the close session requests on close. The network
// I/O logic is handled inside ConsumerNetworkThread.runAtClose, so we need to run that logic here.
- ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate, timer);
+ ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate);
// the network is polled during the last state of clean up.
networkClientDelegate.poll(time.timer(1));
// validate that closing the fetcher has sent a request with final epoch. 2 requests are sent, one for the
@@ -1909,7 +1910,8 @@ public void testQuotaMetrics() {
Node node = cluster.nodes().get(0);
NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE,
1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000,
- time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext());
+ time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext(),
+ MetadataRecoveryStrategy.NONE);
ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse(
400, ApiMessageType.ListenerType.ZK_BROKER);
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
index 1eac870993..d0167e9b98 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
@@ -21,6 +21,7 @@
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.FetchSessionHandler;
import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.clients.MockClient;
import org.apache.kafka.clients.NetworkClient;
import org.apache.kafka.clients.NodeApiVersions;
@@ -1905,7 +1906,8 @@ public void testQuotaMetrics() {
Node node = cluster.nodes().get(0);
NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE,
1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000,
- time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext());
+ time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext(),
+ MetadataRecoveryStrategy.NONE);
ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse(
400, ApiMessageType.ListenerType.ZK_BROKER);
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java
index 8334fb2360..f63dd55754 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java
@@ -277,7 +277,7 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() {
result = heartbeatRequestManager.poll(time.milliseconds());
assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " +
"previous one is in-flight");
-
+
time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS);
result = heartbeatRequestManager.poll(time.milliseconds());
assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent when the " +
@@ -752,6 +752,25 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() {
assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING");
}
+ @ParameterizedTest
+ @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT)
+ public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short version) {
+ mockStableMember();
+ time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS);
+ NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
+ assertEquals(1, result.unsentRequests.size());
+ result = heartbeatRequestManager.poll(time.milliseconds());
+ assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a previous one is in-flight");
+
+ membershipManager.leaveGroup();
+
+ ConsumerGroupHeartbeatRequest heartbeatToLeave = getHeartbeatRequest(heartbeatRequestManager, version);
+ assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, heartbeatToLeave.data().memberEpoch());
+
+ NetworkClientDelegate.PollResult pollAgain = heartbeatRequestManager.poll(time.milliseconds());
+ assertEquals(0, pollAgain.unsentRequests.size());
+ }
+
private void assertHeartbeat(HeartbeatRequestManager hrm, int nextPollMs) {
NetworkClientDelegate.PollResult pollResult = hrm.poll(time.milliseconds());
assertEquals(1, pollResult.unsentRequests.size());
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java
index cef608a902..44de312087 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java
@@ -365,6 +365,8 @@ public void testFencingWhenStateIsPrepareLeaving() {
// because member is already out of the group in the broker).
completeCallback(callbackEvent, membershipManager);
assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state());
+ assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, membershipManager.memberEpoch());
+ verify(membershipManager).notifyEpochChange(Optional.empty(), Optional.empty());
assertTrue(membershipManager.shouldSkipHeartbeat());
}
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java
index 70f33bfdf4..7d53ca385b 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java
@@ -43,6 +43,7 @@
import static org.apache.kafka.clients.consumer.ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG;
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -140,6 +141,34 @@ public void testEnsureTimerSetOnAdd() {
assertEquals(REQUEST_TIMEOUT_MS, ncd.unsentRequests().poll().timer().timeoutMs());
}
+ @Test
+ public void testHasAnyPendingRequests() throws Exception {
+ try (NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate()) {
+ NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest();
+ networkClientDelegate.add(unsentRequest);
+
+ // unsent
+ assertTrue(networkClientDelegate.hasAnyPendingRequests());
+ assertFalse(networkClientDelegate.unsentRequests().isEmpty());
+ assertFalse(client.hasInFlightRequests());
+
+ networkClientDelegate.poll(0, time.milliseconds());
+
+ // in-flight
+ assertTrue(networkClientDelegate.hasAnyPendingRequests());
+ assertTrue(networkClientDelegate.unsentRequests().isEmpty());
+ assertTrue(client.hasInFlightRequests());
+
+ client.respond(FindCoordinatorResponse.prepareResponse(Errors.NONE, GROUP_ID, mockNode()));
+ networkClientDelegate.poll(0, time.milliseconds());
+
+ // get response
+ assertFalse(networkClientDelegate.hasAnyPendingRequests());
+ assertTrue(networkClientDelegate.unsentRequests().isEmpty());
+ assertFalse(client.hasInFlightRequests());
+ }
+ }
+
public NetworkClientDelegate newNetworkClientDelegate() {
LogContext logContext = new LogContext();
Properties properties = new Properties();
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TimedRequestStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TimedRequestStateTest.java
new file mode 100644
index 0000000000..ddde3ae84d
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TimedRequestStateTest.java
@@ -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 org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TimedRequestStateTest {
+
+ private final static long DEFAULT_TIMEOUT_MS = 30000;
+ private final Time time = new MockTime();
+
+ @Test
+ public void testIsExpired() {
+ TimedRequestState state = new TimedRequestState(
+ new LogContext(),
+ this.getClass().getSimpleName(),
+ 100,
+ 1000,
+ time.timer(DEFAULT_TIMEOUT_MS)
+ );
+ assertFalse(state.isExpired());
+ time.sleep(DEFAULT_TIMEOUT_MS);
+ assertTrue(state.isExpired());
+ }
+
+ @Test
+ public void testRemainingMs() {
+ TimedRequestState state = new TimedRequestState(
+ new LogContext(),
+ this.getClass().getSimpleName(),
+ 100,
+ 1000,
+ time.timer(DEFAULT_TIMEOUT_MS)
+ );
+ assertEquals(DEFAULT_TIMEOUT_MS, state.remainingMs());
+ time.sleep(DEFAULT_TIMEOUT_MS);
+ assertEquals(0, state.remainingMs());
+ }
+
+ @Test
+ public void testDeadlineTimer() {
+ long deadlineMs = time.milliseconds() + DEFAULT_TIMEOUT_MS;
+ Timer timer = TimedRequestState.deadlineTimer(time, deadlineMs);
+ assertEquals(DEFAULT_TIMEOUT_MS, timer.remainingMs());
+ timer.sleep(DEFAULT_TIMEOUT_MS);
+ assertEquals(0, timer.remainingMs());
+ }
+
+ @Test
+ public void testAllowOverdueDeadlineTimer() {
+ long deadlineMs = time.milliseconds() - DEFAULT_TIMEOUT_MS;
+ Timer timer = TimedRequestState.deadlineTimer(time, deadlineMs);
+ assertEquals(0, timer.remainingMs());
+ }
+
+ @Test
+ public void testToStringUpdatesTimer() {
+ TimedRequestState state = new TimedRequestState(
+ new LogContext(),
+ this.getClass().getSimpleName(),
+ 100,
+ 1000,
+ time.timer(DEFAULT_TIMEOUT_MS)
+ );
+
+ assertToString(state, DEFAULT_TIMEOUT_MS);
+ time.sleep(DEFAULT_TIMEOUT_MS);
+ assertToString(state, 0);
+ }
+
+ private void assertToString(TimedRequestState state, long timerMs) {
+ assertTrue(state.toString().contains("remainingMs=" + timerMs + "}"));
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java
index 3f2b2c3d98..56eff5b4f4 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java
@@ -74,6 +74,7 @@ public void setup() {
props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(
new LogContext(),
+ time,
new ConsumerConfig(props)));
}
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java
index 451743ae2a..bb021fa8b9 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java
@@ -36,14 +36,10 @@
import java.util.List;
import java.util.Optional;
import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletableFuture;
-import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs;
-import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
public class ApplicationEventProcessorTest {
private final Time time = new MockTime(1);
@@ -90,16 +86,6 @@ public void testPrepClosingCommitEvents() {
verify(commitRequestManager).signalClose();
}
- @Test
- public void testPrepClosingLeaveGroupEvent() {
- LeaveOnCloseEvent event = new LeaveOnCloseEvent(calculateDeadlineMs(time, 100));
- when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager);
- when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null));
- processor.process(event);
- verify(membershipManager).leaveGroup();
- assertTrue(event.future().isDone());
- }
-
private List mockCommitResults() {
return Collections.singletonList(mock(NetworkClientDelegate.UnsentRequest.class));
}
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
index 7d4aa5e3a8..17119a2529 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
@@ -18,6 +18,7 @@
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.LeastLoadedNode;
import org.apache.kafka.clients.MockClient;
import org.apache.kafka.clients.NodeApiVersions;
import org.apache.kafka.clients.consumer.ConsumerConfig;
@@ -735,8 +736,8 @@ private static KafkaProducer producerWithOverrideNewSender(Map configs = new HashMap<>();
+ configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass);
+ configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass);
+ final ProducerConfig producerConfig = new ProducerConfig(configs);
+ assertEquals(MetadataRecoveryStrategy.NONE.name, producerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG));
+ }
+
+ @Test
+ public void testInvalidMetadataRecoveryStrategy() {
+ Map configs = new HashMap<>();
+ configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass);
+ configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass);
+ configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc");
+ ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs));
+ assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG));
+ }
+
@Test
public void testCaseInsensitiveSecurityProtocol() {
final String saslSslLowerCase = SecurityProtocol.SASL_SSL.name.toLowerCase(Locale.ROOT);
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
index 5c1088987e..cfeefc0ae5 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
@@ -19,7 +19,9 @@
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.ClientRequest;
import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.LeastLoadedNode;
import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.clients.MetadataSnapshot;
import org.apache.kafka.clients.MockClient;
import org.apache.kafka.clients.NetworkClient;
@@ -299,7 +301,8 @@ public void testQuotaMetrics() {
Node node = cluster.nodes().get(0);
NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE,
1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000,
- time, true, new ApiVersions(), throttleTimeSensor, logContext);
+ time, true, new ApiVersions(), throttleTimeSensor, logContext,
+ MetadataRecoveryStrategy.NONE);
ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse(
400, ApiMessageType.ListenerType.ZK_BROKER);
@@ -3797,12 +3800,12 @@ private void createMockClientWithMaxFlightOneMetadataPending() {
client = new MockClient(time, metadata) {
volatile boolean canSendMore = true;
@Override
- public Node leastLoadedNode(long now) {
+ public LeastLoadedNode leastLoadedNode(long now) {
for (Node node : metadata.fetch().nodes()) {
if (isReady(node, now) && canSendMore)
- return node;
+ return new LeastLoadedNode(node, true);
}
- return null;
+ return new LeastLoadedNode(null, false);
}
@Override
@@ -3821,7 +3824,7 @@ public List poll(long timeoutMs, long now) {
while (!client.ready(node, time.milliseconds()))
client.poll(0, time.milliseconds());
client.send(request, time.milliseconds());
- while (client.leastLoadedNode(time.milliseconds()) != null)
+ while (client.leastLoadedNode(time.milliseconds()).node() != null)
client.poll(0, time.milliseconds());
}
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
index 08e2cebc31..bd38ccabd9 100755
--- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
@@ -21,6 +21,8 @@
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.function.Executable;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
import org.mockito.stubbing.OngoingStubbing;
import java.io.Closeable;
@@ -106,31 +108,35 @@ public void testMurmur2() {
}
}
- @Test
- public void testGetHost() {
- // valid
+ @ParameterizedTest
+ @CsvSource(value = {"PLAINTEXT", "SASL_PLAINTEXT", "SSL", "SASL_SSL"})
+ public void testGetHostValid(String protocol) {
+ assertEquals("mydomain.com", getHost(protocol + "://mydomain.com:8080"));
+ assertEquals("MyDomain.com", getHost(protocol + "://MyDomain.com:8080"));
+ assertEquals("My_Domain.com", getHost(protocol + "://My_Domain.com:8080"));
+ assertEquals("::1", getHost(protocol + "://[::1]:1234"));
+ assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost(protocol + "://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678"));
+ assertEquals("2001:DB8:85A3:8D3:1319:8A2E:370:7348", getHost(protocol + "://[2001:DB8:85A3:8D3:1319:8A2E:370:7348]:5678"));
+ assertEquals("fe80::b1da:69ca:57f7:63d8%3", getHost(protocol + "://[fe80::b1da:69ca:57f7:63d8%3]:5678"));
assertEquals("127.0.0.1", getHost("127.0.0.1:8000"));
- assertEquals("mydomain.com", getHost("PLAINTEXT://mydomain.com:8080"));
- assertEquals("MyDomain.com", getHost("PLAINTEXT://MyDomain.com:8080"));
- assertEquals("My_Domain.com", getHost("PLAINTEXT://My_Domain.com:8080"));
assertEquals("::1", getHost("[::1]:1234"));
- assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678"));
- assertEquals("2001:DB8:85A3:8D3:1319:8A2E:370:7348", getHost("PLAINTEXT://[2001:DB8:85A3:8D3:1319:8A2E:370:7348]:5678"));
- assertEquals("fe80::b1da:69ca:57f7:63d8%3", getHost("PLAINTEXT://[fe80::b1da:69ca:57f7:63d8%3]:5678"));
+ }
- // invalid
- assertNull(getHost("PLAINTEXT://mydo)main.com:8080"));
- assertNull(getHost("PLAINTEXT://mydo(main.com:8080"));
- assertNull(getHost("PLAINTEXT://mydo()main.com:8080"));
- assertNull(getHost("PLAINTEXT://mydo(main).com:8080"));
+ @ParameterizedTest
+ @CsvSource(value = {"PLAINTEXT", "SASL_PLAINTEXT", "SSL", "SASL_SSL"})
+ public void testGetHostInvalid(String protocol) {
+ assertNull(getHost(protocol + "://mydo)main.com:8080"));
+ assertNull(getHost(protocol + "://mydo(main.com:8080"));
+ assertNull(getHost(protocol + "://mydo()main.com:8080"));
+ assertNull(getHost(protocol + "://mydo(main).com:8080"));
+ assertNull(getHost(protocol + "://[2001:db)8:85a3:8d3:1319:8a2e:370:7348]:5678"));
+ assertNull(getHost(protocol + "://[2001:db(8:85a3:8d3:1319:8a2e:370:7348]:5678"));
+ assertNull(getHost(protocol + "://[2001:db()8:85a3:8d3:1319:8a2e:370:7348]:5678"));
+ assertNull(getHost(protocol + "://[2001:db(8:85a3:)8d3:1319:8a2e:370:7348]:5678"));
assertNull(getHost("ho)st:9092"));
assertNull(getHost("ho(st:9092"));
assertNull(getHost("ho()st:9092"));
assertNull(getHost("ho(st):9092"));
- assertNull(getHost("PLAINTEXT://[2001:db)8:85a3:8d3:1319:8a2e:370:7348]:5678"));
- assertNull(getHost("PLAINTEXT://[2001:db(8:85a3:8d3:1319:8a2e:370:7348]:5678"));
- assertNull(getHost("PLAINTEXT://[2001:db()8:85a3:8d3:1319:8a2e:370:7348]:5678"));
- assertNull(getHost("PLAINTEXT://[2001:db(8:85a3:)8d3:1319:8a2e:370:7348]:5678"));
}
@Test
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java
index 2a6e71c381..cf5f01502c 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java
@@ -213,11 +213,15 @@ public static void validateValue(Schema schema, Object value) {
validateValue(null, schema, value);
}
- public static void validateValue(String name, Schema schema, Object value) {
+ public static void validateValue(String field, Schema schema, Object value) {
+ validateValue(schema, value, field == null ? "value" : "field: \"" + field + "\"");
+ }
+
+ private static void validateValue(Schema schema, Object value, String location) {
if (value == null) {
if (!schema.isOptional())
- throw new DataException("Invalid value: null used for required field: \"" + name
- + "\", schema type: " + schema.type());
+ throw new DataException("Invalid value: null used for required " + location
+ + ", schema type: " + schema.type());
return;
}
@@ -236,8 +240,8 @@ public static void validateValue(String name, Schema schema, Object value) {
exceptionMessage.append(" \"").append(schema.name()).append("\"");
}
exceptionMessage.append(" with type ").append(schema.type()).append(": ").append(value.getClass());
- if (name != null) {
- exceptionMessage.append(" for field: \"").append(name).append("\"");
+ if (location != null) {
+ exceptionMessage.append(" for ").append(location);
}
throw new DataException(exceptionMessage.toString());
}
@@ -251,19 +255,33 @@ public static void validateValue(String name, Schema schema, Object value) {
break;
case ARRAY:
List> array = (List>) value;
- for (Object entry : array)
- validateValue(schema.valueSchema(), entry);
+ String entryLocation = "element of array " + location;
+ Schema arrayValueSchema = assertSchemaNotNull(schema.valueSchema(), entryLocation);
+ for (Object entry : array) {
+ validateValue(arrayValueSchema, entry, entryLocation);
+ }
break;
case MAP:
Map, ?> map = (Map, ?>) value;
+ String keyLocation = "key of map " + location;
+ String valueLocation = "value of map " + location;
+ Schema mapKeySchema = assertSchemaNotNull(schema.keySchema(), keyLocation);
+ Schema mapValueSchema = assertSchemaNotNull(schema.valueSchema(), valueLocation);
for (Map.Entry, ?> entry : map.entrySet()) {
- validateValue(schema.keySchema(), entry.getKey());
- validateValue(schema.valueSchema(), entry.getValue());
+ validateValue(mapKeySchema, entry.getKey(), keyLocation);
+ validateValue(mapValueSchema, entry.getValue(), valueLocation);
}
break;
}
}
+ private static Schema assertSchemaNotNull(Schema schema, String location) {
+ if (schema == null) {
+ throw new DataException("No schema defined for " + location);
+ }
+ return schema;
+ }
+
private static List> expectedClassesFor(Schema schema) {
List> expectedClasses = LOGICAL_TYPE_CLASSES.get(schema.name());
if (expectedClasses == null)
diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java
index 25e6db3469..43c2342fe3 100644
--- a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java
+++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java
@@ -330,4 +330,144 @@ public void testEmptyStruct() {
new Struct(emptyStruct);
}
+ private void assertInvalidValueForSchema(String fieldName, Schema schema, Object value, String message) {
+ Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, schema, value));
+ assertEquals(message, e.getMessage());
+ }
+
+ @Test
+ public void testValidateFieldWithInvalidValueType() {
+ String fieldName = "field";
+ assertInvalidValueForSchema(fieldName, new FakeSchema(), new Object(),
+ "Invalid Java object for schema \"fake\" with type null: class java.lang.Object for field: \"field\"");
+ assertInvalidValueForSchema(null, Schema.INT8_SCHEMA, new Object(),
+ "Invalid Java object for schema with type INT8: class java.lang.Object for value");
+ assertInvalidValueForSchema(fieldName, Schema.INT8_SCHEMA, new Object(),
+ "Invalid Java object for schema with type INT8: class java.lang.Object for field: \"field\"");
+ }
+
+ @Test
+ public void testValidateFieldWithInvalidValueMismatchTimestamp() {
+ long longValue = 1000L;
+ String fieldName = "field";
+
+ ConnectSchema.validateValue(fieldName, Schema.INT64_SCHEMA, longValue);
+
+ assertInvalidValueForSchema(fieldName, Timestamp.SCHEMA, longValue,
+ "Invalid Java object for schema \"org.apache.kafka.connect.data.Timestamp\" " +
+ "with type INT64: class java.lang.Long for field: \"field\"");
+ }
+
+ @Test
+ public void testValidateList() {
+ String fieldName = "field";
+
+ // Optional element schema
+ Schema optionalStrings = SchemaBuilder.array(Schema.OPTIONAL_STRING_SCHEMA);
+ ConnectSchema.validateValue(fieldName, optionalStrings, Collections.emptyList());
+ ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonList("hello"));
+ ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonList(null));
+ ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList("hello", "world"));
+ ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList("hello", null));
+ ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList(null, "world"));
+ assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonList(true),
+ "Invalid Java object for schema with type STRING: class java.lang.Boolean for element of array field: \"field\"");
+
+ // Required element schema
+ Schema requiredStrings = SchemaBuilder.array(Schema.STRING_SCHEMA);
+ ConnectSchema.validateValue(fieldName, requiredStrings, Collections.emptyList());
+ ConnectSchema.validateValue(fieldName, requiredStrings, Collections.singletonList("hello"));
+ assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonList(null),
+ "Invalid value: null used for required element of array field: \"field\", schema type: STRING");
+ ConnectSchema.validateValue(fieldName, requiredStrings, Arrays.asList("hello", "world"));
+ assertInvalidValueForSchema(fieldName, requiredStrings, Arrays.asList("hello", null),
+ "Invalid value: null used for required element of array field: \"field\", schema type: STRING");
+ assertInvalidValueForSchema(fieldName, requiredStrings, Arrays.asList(null, "world"),
+ "Invalid value: null used for required element of array field: \"field\", schema type: STRING");
+ assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonList(true),
+ "Invalid Java object for schema with type STRING: class java.lang.Boolean for element of array field: \"field\"");
+
+ // Null element schema
+ Schema nullElements = SchemaBuilder.type(Schema.Type.ARRAY);
+ assertInvalidValueForSchema(fieldName, nullElements, Collections.emptyList(),
+ "No schema defined for element of array field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList("hello"),
+ "No schema defined for element of array field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList(null),
+ "No schema defined for element of array field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList("hello", "world"),
+ "No schema defined for element of array field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList("hello", null),
+ "No schema defined for element of array field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList(null, "world"),
+ "No schema defined for element of array field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList(true),
+ "No schema defined for element of array field: \"field\"");
+ }
+
+ @Test
+ public void testValidateMap() {
+ String fieldName = "field";
+
+ // Optional element schema
+ Schema optionalStrings = SchemaBuilder.map(Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA);
+ ConnectSchema.validateValue(fieldName, optionalStrings, Collections.emptyMap());
+ ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap("key", "value"));
+ ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap("key", null));
+ ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap(null, "value"));
+ ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap(null, null));
+ assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonMap("key", true),
+ "Invalid Java object for schema with type STRING: class java.lang.Boolean for value of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonMap(true, "value"),
+ "Invalid Java object for schema with type STRING: class java.lang.Boolean for key of map field: \"field\"");
+
+ // Required element schema
+ Schema requiredStrings = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA);
+ ConnectSchema.validateValue(fieldName, requiredStrings, Collections.emptyMap());
+ ConnectSchema.validateValue(fieldName, requiredStrings, Collections.singletonMap("key", "value"));
+ assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap("key", null),
+ "Invalid value: null used for required value of map field: \"field\", schema type: STRING");
+ assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(null, "value"),
+ "Invalid value: null used for required key of map field: \"field\", schema type: STRING");
+ assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(null, null),
+ "Invalid value: null used for required key of map field: \"field\", schema type: STRING");
+ assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap("key", true),
+ "Invalid Java object for schema with type STRING: class java.lang.Boolean for value of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(true, "value"),
+ "Invalid Java object for schema with type STRING: class java.lang.Boolean for key of map field: \"field\"");
+
+ // Null key schema
+ Schema nullKeys = SchemaBuilder.type(Schema.Type.MAP);
+ assertInvalidValueForSchema(fieldName, nullKeys, Collections.emptyMap(),
+ "No schema defined for key of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", "value"),
+ "No schema defined for key of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", null),
+ "No schema defined for key of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(null, "value"),
+ "No schema defined for key of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(null, null),
+ "No schema defined for key of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", true),
+ "No schema defined for key of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(true, "value"),
+ "No schema defined for key of map field: \"field\"");
+
+ // Null value schema
+ Schema nullValues = SchemaBuilder.mapWithNullValues(Schema.OPTIONAL_STRING_SCHEMA);
+ assertInvalidValueForSchema(fieldName, nullValues, Collections.emptyMap(),
+ "No schema defined for value of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", "value"),
+ "No schema defined for value of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", null),
+ "No schema defined for value of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(null, "value"),
+ "No schema defined for value of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(null, null),
+ "No schema defined for value of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", true),
+ "No schema defined for value of map field: \"field\"");
+ assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(true, "value"),
+ "No schema defined for value of map field: \"field\"");
+ }
}
diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java
index 55ccc81bed..6dee26ca83 100644
--- a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java
+++ b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java
@@ -304,39 +304,6 @@ public void testValidateStructWithNullValue() {
e.getMessage());
}
- @Test
- public void testValidateFieldWithInvalidValueType() {
- String fieldName = "field";
- FakeSchema fakeSchema = new FakeSchema();
-
- Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName,
- fakeSchema, new Object()));
- assertEquals("Invalid Java object for schema \"fake\" with type null: class java.lang.Object for field: \"field\"",
- e.getMessage());
-
- e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName,
- Schema.INT8_SCHEMA, new Object()));
- assertEquals("Invalid Java object for schema with type INT8: class java.lang.Object for field: \"field\"",
- e.getMessage());
-
- e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(Schema.INT8_SCHEMA, new Object()));
- assertEquals("Invalid Java object for schema with type INT8: class java.lang.Object", e.getMessage());
- }
-
- @Test
- public void testValidateFieldWithInvalidValueMismatchTimestamp() {
- String fieldName = "field";
- long longValue = 1000L;
-
- // Does not throw
- ConnectSchema.validateValue(fieldName, Schema.INT64_SCHEMA, longValue);
-
- Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName,
- Timestamp.SCHEMA, longValue));
- assertEquals("Invalid Java object for schema \"org.apache.kafka.connect.data.Timestamp\" " +
- "with type INT64: class java.lang.Long for field: \"field\"", e.getMessage());
- }
-
@Test
public void testPutNullField() {
final String fieldName = "fieldName";
diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java
index b623aaf14e..b77eeaf066 100644
--- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java
+++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java
@@ -241,15 +241,15 @@ public JsonConverter() {
/**
* Creates a JsonConvert initializing serializer and deserializer.
*
- * @param enableModules permits to enable/disable the registration of additional Jackson modules.
+ * @param enableAfterburner permits to enable/disable the registration of Jackson Afterburner module.
*
* NOTE: This is visible only for testing
*/
- public JsonConverter(boolean enableModules) {
+ public JsonConverter(boolean enableAfterburner) {
serializer = new JsonSerializer(
mkSet(),
JSON_NODE_FACTORY,
- enableModules
+ enableAfterburner
);
deserializer = new JsonDeserializer(
@@ -259,7 +259,7 @@ public JsonConverter(boolean enableModules) {
DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS
),
JSON_NODE_FACTORY,
- enableModules
+ enableAfterburner
);
}
diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java
index d63ae7808c..40d4631dfb 100644
--- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java
+++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java
@@ -16,13 +16,15 @@
*/
package org.apache.kafka.connect.json;
+import org.apache.kafka.common.errors.SerializationException;
+import org.apache.kafka.common.serialization.Deserializer;
+
import com.fasterxml.jackson.core.json.JsonReadFeature;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
-import org.apache.kafka.common.errors.SerializationException;
-import org.apache.kafka.common.serialization.Deserializer;
+import com.fasterxml.jackson.module.afterburner.AfterburnerModule;
import java.util.Collections;
import java.util.Set;
@@ -51,13 +53,13 @@ public JsonDeserializer() {
JsonDeserializer(
final Set deserializationFeatures,
final JsonNodeFactory jsonNodeFactory,
- final boolean enableModules
+ final boolean enableAfterburner
) {
objectMapper.enable(JsonReadFeature.ALLOW_LEADING_ZEROS_FOR_NUMBERS.mappedFeature());
deserializationFeatures.forEach(objectMapper::enable);
objectMapper.setNodeFactory(jsonNodeFactory);
- if (enableModules) {
- objectMapper.findAndRegisterModules();
+ if (enableAfterburner) {
+ objectMapper.registerModule(new AfterburnerModule());
}
}
diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java
index f8b13fbb6a..64c4414148 100644
--- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java
+++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java
@@ -16,12 +16,14 @@
*/
package org.apache.kafka.connect.json;
+import org.apache.kafka.common.errors.SerializationException;
+import org.apache.kafka.common.serialization.Serializer;
+
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
-import org.apache.kafka.common.errors.SerializationException;
-import org.apache.kafka.common.serialization.Serializer;
+import com.fasterxml.jackson.module.afterburner.AfterburnerModule;
import java.util.Collections;
import java.util.Set;
@@ -50,12 +52,12 @@ public JsonSerializer() {
JsonSerializer(
final Set serializationFeatures,
final JsonNodeFactory jsonNodeFactory,
- final boolean enableModules
+ final boolean enableAfterburner
) {
serializationFeatures.forEach(objectMapper::enable);
objectMapper.setNodeFactory(jsonNodeFactory);
- if (enableModules) {
- objectMapper.findAndRegisterModules();
+ if (enableAfterburner) {
+ objectMapper.registerModule(new AfterburnerModule());
}
}
diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java
index 4fbd282d11..5398f21c62 100644
--- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java
+++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java
@@ -219,16 +219,12 @@ public void startClusters(Map additionalMM2Config) throws Except
.build();
primary.start();
- primary.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS,
- "Workers of " + PRIMARY_CLUSTER_ALIAS + "-connect-cluster did not start in time.");
waitForTopicCreated(primary, "mm2-status.backup.internal");
waitForTopicCreated(primary, "mm2-offsets.backup.internal");
waitForTopicCreated(primary, "mm2-configs.backup.internal");
backup.start();
- backup.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS,
- "Workers of " + BACKUP_CLUSTER_ALIAS + "-connect-cluster did not start in time.");
primaryProducer = initializeProducer(primary);
backupProducer = initializeProducer(backup);
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
index 2a27103079..e50019676a 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
@@ -124,7 +124,7 @@
*/
public abstract class AbstractHerder implements Herder, TaskStatus.Listener, ConnectorStatus.Listener {
- private final Logger log = LoggerFactory.getLogger(AbstractHerder.class);
+ private static final Logger log = LoggerFactory.getLogger(AbstractHerder.class);
private final String workerId;
protected final Worker worker;
@@ -1039,21 +1039,30 @@ public static List> reverseTransform(String connName,
return result;
}
- public boolean taskConfigsChanged(ClusterConfigState configState, String connName, List> taskProps) {
+ public static boolean taskConfigsChanged(ClusterConfigState configState, String connName, List> rawTaskProps) {
int currentNumTasks = configState.taskCount(connName);
boolean result = false;
- if (taskProps.size() != currentNumTasks) {
- log.debug("Connector {} task count changed from {} to {}", connName, currentNumTasks, taskProps.size());
+ if (rawTaskProps.size() != currentNumTasks) {
+ log.debug("Connector {} task count changed from {} to {}", connName, currentNumTasks, rawTaskProps.size());
result = true;
- } else {
+ }
+ if (!result) {
for (int index = 0; index < currentNumTasks; index++) {
ConnectorTaskId taskId = new ConnectorTaskId(connName, index);
- if (!taskProps.get(index).equals(configState.taskConfig(taskId))) {
+ if (!rawTaskProps.get(index).equals(configState.rawTaskConfig(taskId))) {
log.debug("Connector {} has change in configuration for task {}-{}", connName, connName, index);
result = true;
}
}
}
+ if (!result) {
+ Map appliedConnectorConfig = configState.appliedConnectorConfig(connName);
+ Map currentConnectorConfig = configState.connectorConfig(connName);
+ if (!Objects.equals(appliedConnectorConfig, currentConnectorConfig)) {
+ log.debug("Forcing task restart for connector {} as its configuration appears to be updated", connName);
+ result = true;
+ }
+ }
if (result) {
log.debug("Reconfiguring connector {}: writing new updated configurations for tasks", connName);
} else {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
index 14826e982d..1880fa512d 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java
@@ -17,6 +17,7 @@
package org.apache.kafka.connect.runtime.distributed;
import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigException;
@@ -96,6 +97,10 @@ public class DistributedConfig extends WorkerConfig {
public static final String REBALANCE_TIMEOUT_MS_CONFIG = CommonClientConfigs.REBALANCE_TIMEOUT_MS_CONFIG;
private static final String REBALANCE_TIMEOUT_MS_DOC = CommonClientConfigs.REBALANCE_TIMEOUT_MS_DOC;
+ public static final String METADATA_RECOVERY_STRATEGY_CONFIG = CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG;
+ private static final String METADATA_RECOVERY_STRATEGY_DOC = CommonClientConfigs.METADATA_RECOVERY_STRATEGY_DOC;
+ public static final String DEFAULT_METADATA_RECOVERY_STRATEGY = CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY;
+
/**
* worker.sync.timeout.ms
*/
@@ -512,7 +517,14 @@ private static ConfigDef config(Crypto crypto) {
(name, value) -> validateVerificationAlgorithms(crypto, name, (List) value),
() -> "A list of one or more MAC algorithms, each supported by the worker JVM"),
ConfigDef.Importance.LOW,
- INTER_WORKER_VERIFICATION_ALGORITHMS_DOC);
+ INTER_WORKER_VERIFICATION_ALGORITHMS_DOC)
+ .define(METADATA_RECOVERY_STRATEGY_CONFIG,
+ ConfigDef.Type.STRING,
+ DEFAULT_METADATA_RECOVERY_STRATEGY,
+ ConfigDef.CaseInsensitiveValidString
+ .in(Utils.enumOptions(MetadataRecoveryStrategy.class)),
+ ConfigDef.Importance.LOW,
+ METADATA_RECOVERY_STRATEGY_DOC);
}
private final ExactlyOnceSourceSupport exactlyOnceSourceSupport;
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
index cdffbb8787..ab46ee536a 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
@@ -2229,11 +2229,11 @@ private void reconfigureConnector(final String connName, final Callback cb
}
private void publishConnectorTaskConfigs(String connName, List> taskProps, Callback cb) {
- if (!taskConfigsChanged(configState, connName, taskProps)) {
+ List> rawTaskProps = reverseTransform(connName, configState, taskProps);
+ if (!taskConfigsChanged(configState, connName, rawTaskProps)) {
return;
}
- List> rawTaskProps = reverseTransform(connName, configState, taskProps);
if (isLeader()) {
writeTaskConfigs(connName, rawTaskProps);
cb.onCompletion(null, null);
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
index 2c3537ea67..2ea83daf04 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
@@ -20,6 +20,7 @@
import org.apache.kafka.clients.ClientUtils;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.clients.NetworkClient;
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient;
import org.apache.kafka.clients.GroupRebalanceConfig;
@@ -119,7 +120,9 @@ public WorkerGroupMember(DistributedConfig config,
time,
true,
new ApiVersions(),
- logContext);
+ logContext,
+ MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG))
+ );
this.client = new ConsumerNetworkClient(
logContext,
netClient,
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java
index da8e235e42..9d338936db 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java
@@ -47,7 +47,7 @@ public Map config() {
return config;
}
- @JsonProperty
+ @JsonProperty("initial_state")
public InitialState initialState() {
return initialState;
}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java
index e773eeefd5..2768d910d4 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java
@@ -519,10 +519,10 @@ private synchronized void updateConnectorTasks(String connName) {
}
List> newTaskConfigs = recomputeTaskConfigs(connName);
+ List> rawTaskConfigs = reverseTransform(connName, configState, newTaskConfigs);
- if (taskConfigsChanged(configState, connName, newTaskConfigs)) {
+ if (taskConfigsChanged(configState, connName, rawTaskConfigs)) {
removeConnectorTasks(connName);
- List> rawTaskConfigs = reverseTransform(connName, configState, newTaskConfigs);
configBackingStore.putTaskConfigs(connName, rawTaskConfigs);
createConnectorTasks(connName);
}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/AppliedConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/AppliedConnectorConfig.java
new file mode 100644
index 0000000000..22f20e4b4a
--- /dev/null
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/AppliedConnectorConfig.java
@@ -0,0 +1,66 @@
+/*
+ * 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.connect.storage;
+
+import org.apache.kafka.connect.runtime.WorkerConfigTransformer;
+
+import java.util.Map;
+
+/**
+ * Wrapper class for a connector configuration that has been used to generate task configurations
+ * Supports lazy {@link WorkerConfigTransformer#transform(Map) transformation}.
+ */
+public class AppliedConnectorConfig {
+
+ private final Map rawConfig;
+ private Map transformedConfig;
+
+ /**
+ * Create a new applied config that has not yet undergone
+ * {@link WorkerConfigTransformer#transform(Map) transformation}.
+ * @param rawConfig the non-transformed connector configuration; may be null
+ */
+ public AppliedConnectorConfig(Map rawConfig) {
+ this.rawConfig = rawConfig;
+ }
+
+ /**
+ * If necessary, {@link WorkerConfigTransformer#transform(Map) transform} the raw
+ * connector config, then return the result. Transformed configurations are cached and
+ * returned in all subsequent calls.
+ *
+ * This method is thread-safe: different threads may invoke it at any time and the same
+ * transformed config should always be returned, with transformation still only ever
+ * taking place once before its results are cached.
+ * @param configTransformer the transformer to use, if no transformed connector
+ * config has been cached yet; may be null
+ * @return the possibly-cached, transformed, connector config; may be null
+ */
+ public synchronized Map transformedConfig(WorkerConfigTransformer configTransformer) {
+ if (transformedConfig != null || rawConfig == null)
+ return transformedConfig;
+
+ if (configTransformer != null) {
+ transformedConfig = configTransformer.transform(rawConfig);
+ } else {
+ transformedConfig = rawConfig;
+ }
+
+ return transformedConfig;
+ }
+
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ClusterConfigState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ClusterConfigState.java
index 1025373042..df5b00926f 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ClusterConfigState.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ClusterConfigState.java
@@ -43,6 +43,7 @@ public class ClusterConfigState {
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyMap(),
+ Collections.emptyMap(),
Collections.emptySet(),
Collections.emptySet());
@@ -55,6 +56,7 @@ public class ClusterConfigState {
final Map> taskConfigs;
final Map connectorTaskCountRecords;
final Map connectorTaskConfigGenerations;
+ final Map appliedConnectorConfigs;
final Set connectorsPendingFencing;
final Set inconsistentConnectors;
@@ -66,6 +68,7 @@ public ClusterConfigState(long offset,
Map> taskConfigs,
Map connectorTaskCountRecords,
Map connectorTaskConfigGenerations,
+ Map appliedConnectorConfigs,
Set connectorsPendingFencing,
Set inconsistentConnectors) {
this(offset,
@@ -76,6 +79,7 @@ public ClusterConfigState(long offset,
taskConfigs,
connectorTaskCountRecords,
connectorTaskConfigGenerations,
+ appliedConnectorConfigs,
connectorsPendingFencing,
inconsistentConnectors,
null);
@@ -89,6 +93,7 @@ public ClusterConfigState(long offset,
Map> taskConfigs,
Map connectorTaskCountRecords,
Map connectorTaskConfigGenerations,
+ Map appliedConnectorConfigs,
Set connectorsPendingFencing,
Set inconsistentConnectors,
WorkerConfigTransformer configTransformer) {
@@ -100,6 +105,7 @@ public ClusterConfigState(long offset,
this.taskConfigs = taskConfigs;
this.connectorTaskCountRecords = connectorTaskCountRecords;
this.connectorTaskConfigGenerations = connectorTaskConfigGenerations;
+ this.appliedConnectorConfigs = appliedConnectorConfigs;
this.connectorsPendingFencing = connectorsPendingFencing;
this.inconsistentConnectors = inconsistentConnectors;
this.configTransformer = configTransformer;
@@ -158,6 +164,19 @@ public Map