supportedFeatures() {
+ return supportedFeatures;
+ }
}
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 0c795bc5206dc..1d469a66436a6 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
@@ -36,6 +36,7 @@
import java.time.Duration;
import java.util.Collection;
+import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@@ -919,12 +920,21 @@ default ListConsumerGroupsResult listConsumerGroups() {
* @param options The options to use when listing the consumer group offsets.
* @return The ListGroupOffsetsResult
*/
- ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options);
+ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) {
+ @SuppressWarnings("deprecation")
+ ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec()
+ .topicPartitions(options.topicPartitions());
+
+ // We can use the provided options with the batched API, which uses topic partitions from
+ // the group spec and ignores any topic partitions set in the options.
+ return listConsumerGroupOffsets(Collections.singletonMap(groupId, groupSpec), options);
+ }
/**
* List the consumer group offsets available in the cluster with the default options.
*
- * This is a convenience method for {@link #listConsumerGroupOffsets(String, ListConsumerGroupOffsetsOptions)} with default options.
+ * 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.
*/
@@ -932,6 +942,29 @@ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId)
return listConsumerGroupOffsets(groupId, new ListConsumerGroupOffsetsOptions());
}
+ /**
+ * List the consumer group offsets available in the cluster for the specified consumer groups.
+ *
+ * @param groupSpecs Map of consumer group ids to a spec that specifies the topic partitions of the group to list offsets for.
+ *
+ * @param options The options to use when listing the consumer group offsets.
+ * @return The ListConsumerGroupOffsetsResult
+ */
+ ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs, ListConsumerGroupOffsetsOptions options);
+
+ /**
+ * List the consumer group offsets available in the cluster for the specified groups with the default options.
+ *
+ * This is a convenience method for
+ * {@link #listConsumerGroupOffsets(Map, ListConsumerGroupOffsetsOptions)} with default options.
+ *
+ * @param groupSpecs Map of consumer group ids to a spec that specifies the topic partitions of the group to list offsets for.
+ * @return The ListConsumerGroupOffsetsResult.
+ */
+ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs) {
+ return listConsumerGroupOffsets(groupSpecs, new ListConsumerGroupOffsetsOptions());
+ }
+
/**
* Delete consumer groups from the cluster.
*
@@ -1446,6 +1479,35 @@ default DescribeFeaturesResult describeFeatures() {
*/
UpdateFeaturesResult updateFeatures(Map featureUpdates, UpdateFeaturesOptions options);
+ /**
+ * Describes the state of the metadata quorum.
+ *
+ * This is a convenience method for {@link #describeMetadataQuorum(DescribeMetadataQuorumOptions)} with default options.
+ * See the overload for more details.
+ *
+ * @return the {@link DescribeMetadataQuorumResult} containing the result
+ */
+ default DescribeMetadataQuorumResult describeMetadataQuorum() {
+ return describeMetadataQuorum(new DescribeMetadataQuorumOptions());
+ }
+
+ /**
+ * Describes the state of the metadata quorum.
+ *
+ * The following exceptions can be anticipated when calling {@code get()} on the futures obtained from
+ * the returned {@code DescribeMetadataQuorumResult}:
+ *
+ * - {@link org.apache.kafka.common.errors.ClusterAuthorizationException}
+ * If the authenticated user didn't have {@code DESCRIBE} access to the cluster.
+ * - {@link org.apache.kafka.common.errors.TimeoutException}
+ * If the request timed out before the controller could list the cluster links.
+ *
+ *
+ * @param options The {@link DescribeMetadataQuorumOptions} to use when describing the quorum.
+ * @return the {@link DescribeMetadataQuorumResult} containing the result
+ */
+ DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuorumOptions options);
+
/**
* Unregister a broker.
*
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 16feef66d4351..37af386410355 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
@@ -25,6 +25,8 @@
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.SecurityConfig;
import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Utils;
import java.util.Map;
import java.util.Set;
@@ -212,6 +214,7 @@ public class AdminClientConfig extends AbstractConfig {
.define(SECURITY_PROTOCOL_CONFIG,
Type.STRING,
DEFAULT_SECURITY_PROTOCOL,
+ in(Utils.enumOptions(SecurityProtocol.class)),
Importance.MEDIUM,
SECURITY_PROTOCOL_DOC)
.withClientSslSupport()
@@ -220,6 +223,7 @@ public class AdminClientConfig extends AbstractConfig {
@Override
protected Map postProcessParsedConfig(final Map parsedValues) {
+ CommonClientConfigs.postValidateSaslMechanismConfig(this);
return CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues);
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
index 30686c93eaeef..154fc8e65db35 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
@@ -61,8 +61,14 @@ public ConfigEntry(String name, String value) {
* @param isReadOnly whether the config is read-only and cannot be updated
* @param synonyms Synonym configs in order of precedence
*/
- ConfigEntry(String name, String value, ConfigSource source, boolean isSensitive, boolean isReadOnly,
- List synonyms, ConfigType type, String documentation) {
+ public ConfigEntry(String name,
+ String value,
+ ConfigSource source,
+ boolean isSensitive,
+ boolean isReadOnly,
+ List synonyms,
+ ConfigType type,
+ String documentation) {
Objects.requireNonNull(name, "name should not be null");
this.name = name;
this.value = value;
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/CreateDelegationTokenOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/CreateDelegationTokenOptions.java
index 6a082d499bbb4..693e8f52f63b7 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/CreateDelegationTokenOptions.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/CreateDelegationTokenOptions.java
@@ -19,6 +19,7 @@
import java.util.LinkedList;
import java.util.List;
+import java.util.Optional;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.security.auth.KafkaPrincipal;
@@ -32,6 +33,7 @@
public class CreateDelegationTokenOptions extends AbstractOptions {
private long maxLifeTimeMs = -1;
private List renewers = new LinkedList<>();
+ private KafkaPrincipal owner = null;
public CreateDelegationTokenOptions renewers(List renewers) {
this.renewers = renewers;
@@ -42,6 +44,15 @@ public List renewers() {
return renewers;
}
+ public CreateDelegationTokenOptions owner(KafkaPrincipal owner) {
+ this.owner = owner;
+ return this;
+ }
+
+ public Optional owner() {
+ return Optional.ofNullable(owner);
+ }
+
public CreateDelegationTokenOptions maxlifeTimeMs(long maxLifeTimeMs) {
this.maxLifeTimeMs = maxLifeTimeMs;
return this;
diff --git a/core/src/main/scala/kafka/common/BaseEnum.scala b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeMetadataQuorumOptions.java
similarity index 52%
rename from core/src/main/scala/kafka/common/BaseEnum.scala
rename to clients/src/main/java/org/apache/kafka/clients/admin/DescribeMetadataQuorumOptions.java
index 9c39466b7f378..8f54cc81f2110 100644
--- a/core/src/main/scala/kafka/common/BaseEnum.scala
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeMetadataQuorumOptions.java
@@ -1,12 +1,12 @@
-/**
+/*
* Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * 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,
@@ -14,13 +14,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package kafka.common
+package org.apache.kafka.clients.admin;
-/*
- * We inherit from `Product` and `Serializable` because `case` objects and classes inherit from them and if we don't
- * do it here, the compiler will infer types that unexpectedly include `Product` and `Serializable`, see
- * http://underscore.io/blog/posts/2015/06/04/more-on-sealed.html for more information.
+/**
+ * Options for {@link Admin#describeMetadataQuorum(DescribeMetadataQuorumOptions)}
*/
-trait BaseEnum extends Product with Serializable {
- def name: String
+public class DescribeMetadataQuorumOptions extends AbstractOptions {
+
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeMetadataQuorumResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeMetadataQuorumResult.java
new file mode 100644
index 0000000000000..aa9bbe84eadd3
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeMetadataQuorumResult.java
@@ -0,0 +1,38 @@
+/*
+ * 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.common.KafkaFuture;
+
+/**
+ * The result of {@link Admin#describeMetadataQuorum(DescribeMetadataQuorumOptions)}
+ */
+public class DescribeMetadataQuorumResult {
+
+ private final KafkaFuture quorumInfo;
+
+ DescribeMetadataQuorumResult(KafkaFuture quorumInfo) {
+ this.quorumInfo = quorumInfo;
+ }
+
+ /**
+ * Returns a future containing the QuorumInfo
+ */
+ public KafkaFuture quorumInfo() {
+ return quorumInfo;
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
index 38753af3fe7fc..b1dd026078dd8 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java
@@ -23,33 +23,86 @@
*/
public class FeatureUpdate {
private final short maxVersionLevel;
- private final boolean allowDowngrade;
+ private final UpgradeType upgradeType;
+
+ public enum UpgradeType {
+ UNKNOWN(0),
+ UPGRADE(1),
+ SAFE_DOWNGRADE(2),
+ UNSAFE_DOWNGRADE(3);
+
+ private final byte code;
+
+ UpgradeType(int code) {
+ this.code = (byte) code;
+ }
+
+ public byte code() {
+ return code;
+ }
+
+ public static UpgradeType fromCode(int code) {
+ if (code == 1) {
+ return UPGRADE;
+ } else if (code == 2) {
+ return SAFE_DOWNGRADE;
+ } else if (code == 3) {
+ return UNSAFE_DOWNGRADE;
+ } else {
+ return UNKNOWN;
+ }
+ }
+ }
/**
* @param maxVersionLevel the new maximum version level for the finalized feature.
- * a value < 1 is special and indicates that the update is intended to
+ * a value of zero is special and indicates that the update is intended to
* delete the finalized feature, and should be accompanied by setting
* the allowDowngrade flag to true.
* @param allowDowngrade - true, if this feature update was meant to downgrade the existing
- * maximum version level of the finalized feature.
+ * maximum version level of the finalized feature. Only "safe" downgrades are
+ * enabled with this boolean. See {@link FeatureUpdate#FeatureUpdate(short, UpgradeType)}
* - false, otherwise.
*/
+ @Deprecated
public FeatureUpdate(final short maxVersionLevel, final boolean allowDowngrade) {
- if (maxVersionLevel < 1 && !allowDowngrade) {
+ this(maxVersionLevel, allowDowngrade ? UpgradeType.SAFE_DOWNGRADE : UpgradeType.UPGRADE);
+ }
+
+ /**
+ * @param maxVersionLevel The new maximum version level for the finalized feature.
+ * a value of zero is special and indicates that the update is intended to
+ * delete the finalized feature, and should be accompanied by setting
+ * the upgradeType to safe or unsafe.
+ * @param upgradeType Indicate what kind of upgrade should be performed in this operation.
+ * - UPGRADE: upgrading the feature level
+ * - SAFE_DOWNGRADE: only downgrades which do not result in metadata loss are permitted
+ * - UNSAFE_DOWNGRADE: any downgrade, including those which may result in metadata loss, are permitted
+ */
+ public FeatureUpdate(final short maxVersionLevel, final UpgradeType upgradeType) {
+ if (maxVersionLevel == 0 && upgradeType.equals(UpgradeType.UPGRADE)) {
throw new IllegalArgumentException(String.format(
- "The allowDowngrade flag should be set when the provided maxVersionLevel:%d is < 1.",
- maxVersionLevel));
+ "The downgradeType flag should be set to SAFE or UNSAFE when the provided maxVersionLevel:%d is < 1.",
+ maxVersionLevel));
+ }
+ if (maxVersionLevel < 0) {
+ throw new IllegalArgumentException("Cannot specify a negative version level.");
}
this.maxVersionLevel = maxVersionLevel;
- this.allowDowngrade = allowDowngrade;
+ this.upgradeType = upgradeType;
}
public short maxVersionLevel() {
return maxVersionLevel;
}
+ @Deprecated
public boolean allowDowngrade() {
- return allowDowngrade;
+ return upgradeType != UpgradeType.UPGRADE;
+ }
+
+ public UpgradeType upgradeType() {
+ return upgradeType;
}
@Override
@@ -63,16 +116,16 @@ public boolean equals(Object other) {
}
final FeatureUpdate that = (FeatureUpdate) other;
- return this.maxVersionLevel == that.maxVersionLevel && this.allowDowngrade == that.allowDowngrade;
+ return this.maxVersionLevel == that.maxVersionLevel && this.upgradeType.equals(that.upgradeType);
}
@Override
public int hashCode() {
- return Objects.hash(maxVersionLevel, allowDowngrade);
+ return Objects.hash(maxVersionLevel, upgradeType);
}
@Override
public String toString() {
- return String.format("FeatureUpdate{maxVersionLevel:%d, allowDowngrade:%s}", maxVersionLevel, allowDowngrade);
+ return String.format("FeatureUpdate{maxVersionLevel:%d, downgradeType:%s}", maxVersionLevel, upgradeType);
}
}
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 03322fdcf1dc8..41eb27a1ddad8 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
@@ -135,6 +135,7 @@
import org.apache.kafka.common.message.DescribeLogDirsRequestData;
import org.apache.kafka.common.message.DescribeLogDirsRequestData.DescribableLogDirTopic;
import org.apache.kafka.common.message.DescribeLogDirsResponseData;
+import org.apache.kafka.common.message.DescribeQuorumResponseData;
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData;
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData.UserName;
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
@@ -208,12 +209,16 @@
import org.apache.kafka.common.requests.DescribeLogDirsResponse;
import org.apache.kafka.common.requests.DescribeUserScramCredentialsRequest;
import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse;
+import org.apache.kafka.common.requests.DescribeQuorumRequest;
+import org.apache.kafka.common.requests.DescribeQuorumRequest.Builder;
+import org.apache.kafka.common.requests.DescribeQuorumResponse;
import org.apache.kafka.common.requests.ElectLeadersRequest;
import org.apache.kafka.common.requests.ElectLeadersResponse;
import org.apache.kafka.common.requests.ExpireDelegationTokenRequest;
import org.apache.kafka.common.requests.ExpireDelegationTokenResponse;
import org.apache.kafka.common.requests.IncrementalAlterConfigsRequest;
import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse;
+import org.apache.kafka.common.requests.JoinGroupRequest;
import org.apache.kafka.common.requests.ListGroupsRequest;
import org.apache.kafka.common.requests.ListGroupsResponse;
import org.apache.kafka.common.requests.ListOffsetsRequest;
@@ -257,6 +262,7 @@
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
+import java.util.OptionalLong;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
@@ -268,6 +274,8 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.apache.kafka.common.internals.Topic.METADATA_TOPIC_NAME;
+import static org.apache.kafka.common.internals.Topic.METADATA_TOPIC_PARTITION;
import static org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignablePartition;
import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignablePartitionResponse;
import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignableTopicResponse;
@@ -306,9 +314,9 @@ public class KafkaAdminClient extends AdminClient {
private static final long INVALID_SHUTDOWN_TIME = -1;
/**
- * The base reason for a LeaveGroupRequest
+ * The default reason for a LeaveGroupRequest.
*/
- static final String LEAVE_GROUP_REASON = "member was removed by an admin";
+ static final String DEFAULT_LEAVE_GROUP_REASON = "member was removed by an admin";
/**
* Thread name prefix for admin client network thread
@@ -2716,7 +2724,11 @@ private static Map logDirDescriptions(DescribeLogDirs
new ReplicaInfo(p.partitionSize(), p.offsetLag(), p.isFutureKey()));
}
}
- result.put(logDirResult.logDir(), new LogDirDescription(Errors.forCode(logDirResult.errorCode()).exception(), replicaInfoMap));
+ result.put(logDirResult.logDir(), new LogDirDescription(
+ Errors.forCode(logDirResult.errorCode()).exception(),
+ replicaInfoMap,
+ logDirResult.totalBytes(),
+ logDirResult.usableBytes()));
}
return result;
}
@@ -3055,10 +3067,14 @@ public CreateDelegationTokenResult createDelegationToken(final CreateDelegationT
@Override
CreateDelegationTokenRequest.Builder createRequest(int timeoutMs) {
- return new CreateDelegationTokenRequest.Builder(
- new CreateDelegationTokenRequestData()
- .setRenewers(renewers)
- .setMaxLifetimeMs(options.maxlifeTimeMs()));
+ CreateDelegationTokenRequestData data = new CreateDelegationTokenRequestData()
+ .setRenewers(renewers)
+ .setMaxLifetimeMs(options.maxlifeTimeMs());
+ if (options.owner().isPresent()) {
+ data.setOwnerPrincipalName(options.owner().get().getName());
+ data.setOwnerPrincipalType(options.owner().get().getPrincipalType());
+ }
+ return new CreateDelegationTokenRequest.Builder(data);
}
@Override
@@ -3069,6 +3085,7 @@ void handleResponse(AbstractResponse abstractResponse) {
} else {
CreateDelegationTokenResponseData data = response.data();
TokenInformation tokenInfo = new TokenInformation(data.tokenId(), new KafkaPrincipal(data.principalType(), data.principalName()),
+ new KafkaPrincipal(data.tokenRequesterPrincipalType(), data.tokenRequesterPrincipalName()),
options.renewers(), data.issueTimestampMs(), data.maxTimestampMs(), data.expiryTimestampMs());
DelegationToken token = new DelegationToken(tokenInfo, data.hmac());
delegationTokenFuture.complete(token);
@@ -3384,13 +3401,14 @@ void handleFailure(Throwable throwable) {
}
@Override
- public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String groupId,
- final ListConsumerGroupOffsetsOptions options) {
+ public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs,
+ ListConsumerGroupOffsetsOptions options) {
SimpleAdminApiFuture> future =
- ListConsumerGroupOffsetsHandler.newFuture(groupId);
- ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, options.topicPartitions(), logContext);
+ ListConsumerGroupOffsetsHandler.newFuture(groupSpecs.keySet());
+ ListConsumerGroupOffsetsHandler handler =
+ new ListConsumerGroupOffsetsHandler(groupSpecs, options.requireStable(), logContext);
invokeDriver(handler, future, options.timeoutMs);
- return new ListConsumerGroupOffsetsResult(future.get(CoordinatorKey.byGroupId(groupId)));
+ return new ListConsumerGroupOffsetsResult(future.all());
}
@Override
@@ -3713,7 +3731,7 @@ private Integer nodeFor(ConfigResource resource) {
}
}
- private List getMembersFromGroup(String groupId) {
+ private List getMembersFromGroup(String groupId, String reason) {
Collection members;
try {
members = describeConsumerGroups(Collections.singleton(groupId)).describedGroups().get(groupId).get().members();
@@ -3723,11 +3741,15 @@ private List getMembersFromGroup(String groupId) {
List membersToRemove = new ArrayList<>();
for (final MemberDescription member : members) {
+ MemberIdentity memberIdentity = new MemberIdentity().setReason(reason);
+
if (member.groupInstanceId().isPresent()) {
- membersToRemove.add(new MemberIdentity().setGroupInstanceId(member.groupInstanceId().get()));
+ memberIdentity.setGroupInstanceId(member.groupInstanceId().get());
} else {
- membersToRemove.add(new MemberIdentity().setMemberId(member.consumerId()));
+ memberIdentity.setMemberId(member.consumerId());
}
+
+ membersToRemove.add(memberIdentity);
}
return membersToRemove;
}
@@ -3735,15 +3757,17 @@ private List getMembersFromGroup(String groupId) {
@Override
public RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroup(String groupId,
RemoveMembersFromConsumerGroupOptions options) {
+ String reason = options.reason() == null || options.reason().isEmpty() ?
+ DEFAULT_LEAVE_GROUP_REASON : JoinGroupRequest.maybeTruncateReason(options.reason());
+
List members;
if (options.removeAll()) {
- members = getMembersFromGroup(groupId);
+ members = getMembersFromGroup(groupId, reason);
} else {
- members = options.members().stream().map(MemberToRemove::toMemberIdentity).collect(Collectors.toList());
+ members = options.members().stream()
+ .map(m -> m.toMemberIdentity().setReason(reason))
+ .collect(Collectors.toList());
}
-
- String reason = options.reason() == null ? LEAVE_GROUP_REASON : LEAVE_GROUP_REASON + ": " + options.reason();
- members.forEach(member -> member.setReason(reason));
SimpleAdminApiFuture> future =
RemoveMembersFromConsumerGroupHandler.newFuture(groupId);
@@ -4259,12 +4283,13 @@ UpdateFeaturesRequest.Builder createRequest(int timeoutMs) {
new UpdateFeaturesRequestData.FeatureUpdateKey();
requestItem.setFeature(feature);
requestItem.setMaxVersionLevel(update.maxVersionLevel());
- requestItem.setAllowDowngrade(update.allowDowngrade());
+ requestItem.setUpgradeType(update.upgradeType().code());
featureUpdatesRequestData.add(requestItem);
}
return new UpdateFeaturesRequest.Builder(
new UpdateFeaturesRequestData()
.setTimeoutMs(timeoutMs)
+ .setValidateOnly(options.validateOnly())
.setFeatureUpdates(featureUpdatesRequestData));
}
@@ -4314,6 +4339,84 @@ void handleFailure(Throwable throwable) {
return new UpdateFeaturesResult(new HashMap<>(updateFutures));
}
+ @Override
+ public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuorumOptions options) {
+ NodeProvider provider = new LeastLoadedNodeProvider();
+
+ final KafkaFutureImpl future = new KafkaFutureImpl<>();
+ final long now = time.milliseconds();
+ final Call call = new Call(
+ "describeMetadataQuorum", calcDeadlineMs(now, options.timeoutMs()), provider) {
+
+ private QuorumInfo.ReplicaState translateReplicaState(DescribeQuorumResponseData.ReplicaState replica) {
+ return new QuorumInfo.ReplicaState(
+ replica.replicaId(),
+ replica.logEndOffset(),
+ replica.lastFetchTimestamp() == -1 ? OptionalLong.empty() : OptionalLong.of(replica.lastFetchTimestamp()),
+ replica.lastCaughtUpTimestamp() == -1 ? OptionalLong.empty() : OptionalLong.of(replica.lastCaughtUpTimestamp()));
+ }
+
+ private QuorumInfo createQuorumResult(final DescribeQuorumResponseData.PartitionData partition) {
+ return new QuorumInfo(
+ partition.leaderId(),
+ partition.currentVoters().stream().map(v -> translateReplicaState(v)).collect(Collectors.toList()),
+ partition.observers().stream().map(o -> translateReplicaState(o)).collect(Collectors.toList()));
+ }
+
+ @Override
+ DescribeQuorumRequest.Builder createRequest(int timeoutMs) {
+ return new Builder(DescribeQuorumRequest.singletonRequest(
+ new TopicPartition(METADATA_TOPIC_NAME, METADATA_TOPIC_PARTITION.partition())));
+ }
+
+ @Override
+ void handleResponse(AbstractResponse response) {
+ final DescribeQuorumResponse quorumResponse = (DescribeQuorumResponse) response;
+ if (quorumResponse.data().errorCode() != Errors.NONE.code()) {
+ throw Errors.forCode(quorumResponse.data().errorCode()).exception();
+ }
+ if (quorumResponse.data().topics().size() != 1) {
+ String msg = String.format("DescribeMetadataQuorum received %d topics when 1 was expected",
+ quorumResponse.data().topics().size());
+ log.debug(msg);
+ throw new UnknownServerException(msg);
+ }
+ DescribeQuorumResponseData.TopicData topic = quorumResponse.data().topics().get(0);
+ if (!topic.topicName().equals(METADATA_TOPIC_NAME)) {
+ String msg = String.format("DescribeMetadataQuorum received a topic with name %s when %s was expected",
+ topic.topicName(), METADATA_TOPIC_NAME);
+ log.debug(msg);
+ throw new UnknownServerException(msg);
+ }
+ if (topic.partitions().size() != 1) {
+ String msg = String.format("DescribeMetadataQuorum received a topic %s with %d partitions when 1 was expected",
+ topic.topicName(), topic.partitions().size());
+ log.debug(msg);
+ throw new UnknownServerException(msg);
+ }
+ DescribeQuorumResponseData.PartitionData partition = topic.partitions().get(0);
+ if (partition.partitionIndex() != METADATA_TOPIC_PARTITION.partition()) {
+ String msg = String.format("DescribeMetadataQuorum received a single partition with index %d when %d was expected",
+ partition.partitionIndex(), METADATA_TOPIC_PARTITION.partition());
+ log.debug(msg);
+ throw new UnknownServerException(msg);
+ }
+ if (partition.errorCode() != Errors.NONE.code()) {
+ throw Errors.forCode(partition.errorCode()).exception();
+ }
+ future.complete(createQuorumResult(partition));
+ }
+
+ @Override
+ void handleFailure(Throwable throwable) {
+ future.completeExceptionally(throwable);
+ }
+ };
+
+ runnable.call(call, now);
+ return new DescribeMetadataQuorumResult(future);
+ }
+
@Override
public UnregisterBrokerResult unregisterBroker(int brokerId, UnregisterBrokerOptions options) {
final KafkaFutureImpl future = new KafkaFutureImpl<>();
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java
index af738ca209fb9..44d3a407327e1 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java
@@ -23,31 +23,54 @@
import java.util.List;
/**
- * Options for {@link Admin#listConsumerGroupOffsets(String)}.
+ * Options for {@link Admin#listConsumerGroupOffsets(java.util.Map)} and {@link Admin#listConsumerGroupOffsets(String)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ListConsumerGroupOffsetsOptions extends AbstractOptions {
- private List topicPartitions = null;
+ private List topicPartitions;
+ private boolean requireStable = false;
/**
* Set the topic partitions to list as part of the result.
* {@code null} includes all topic partitions.
+ *
+ * @deprecated Since 3.3.
+ * Use {@link Admin#listConsumerGroupOffsets(java.util.Map, ListConsumerGroupOffsetsOptions)}
+ * to specify topic partitions.
*
* @param topicPartitions List of topic partitions to include
* @return This ListGroupOffsetsOptions
*/
+ @Deprecated
public ListConsumerGroupOffsetsOptions topicPartitions(List topicPartitions) {
this.topicPartitions = topicPartitions;
return this;
}
+ /**
+ * Sets an optional requireStable flag.
+ */
+ public ListConsumerGroupOffsetsOptions requireStable(final boolean requireStable) {
+ this.requireStable = requireStable;
+ return this;
+ }
+
/**
* Returns a list of topic partitions to add as part of the result.
+ *
+ * @deprecated Since 3.3.
+ * Use {@link Admin#listConsumerGroupOffsets(java.util.Map, ListConsumerGroupOffsetsOptions)}
+ * to specify topic partitions.
*/
+ @Deprecated
public List topicPartitions() {
return topicPartitions;
}
+
+ public boolean requireStable() {
+ return requireStable;
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java
index 48f4531418110..2136e33a401e1 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java
@@ -17,25 +17,32 @@
package org.apache.kafka.clients.admin;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import org.apache.kafka.clients.admin.internals.CoordinatorKey;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
-import java.util.Map;
-
/**
- * The result of the {@link Admin#listConsumerGroupOffsets(String)} call.
+ * The result of the {@link Admin#listConsumerGroupOffsets(Map)} and
+ * {@link Admin#listConsumerGroupOffsets(String)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ListConsumerGroupOffsetsResult {
- final KafkaFuture
Further, when in read_committed
the seekToEnd method will" +
- " return the LSO";
+ " return the LSO
";
public static final String DEFAULT_ISOLATION_LEVEL = IsolationLevel.READ_UNCOMMITTED.toString().toLowerCase(Locale.ROOT);
@@ -350,6 +353,7 @@ public class ConsumerConfig extends AbstractConfig {
.define(GROUP_INSTANCE_ID_CONFIG,
Type.STRING,
null,
+ new ConfigDef.NonEmptyString(),
Importance.MEDIUM,
GROUP_INSTANCE_ID_DOC)
.define(SESSION_TIMEOUT_MS_CONFIG,
@@ -451,8 +455,8 @@ public class ConsumerConfig extends AbstractConfig {
CommonClientConfigs.RETRY_BACKOFF_MS_DOC)
.define(AUTO_OFFSET_RESET_CONFIG,
Type.STRING,
- "latest",
- in("latest", "earliest", "none"),
+ OffsetResetStrategy.LATEST.toString(),
+ in(Utils.enumOptions(OffsetResetStrategy.class)),
Importance.MEDIUM,
AUTO_OFFSET_RESET_DOC)
.define(CHECK_CRCS_CONFIG,
@@ -571,6 +575,7 @@ public class ConsumerConfig extends AbstractConfig {
.define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG,
Type.STRING,
CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL,
+ in(Utils.enumOptions(SecurityProtocol.class)),
Importance.MEDIUM,
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
.withClientSslSupport()
@@ -579,6 +584,7 @@ public class ConsumerConfig extends AbstractConfig {
@Override
protected Map postProcessParsedConfig(final Map parsedValues) {
+ CommonClientConfigs.postValidateSaslMechanismConfig(this);
Map refinedConfigs = CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues);
maybeOverrideClientId(refinedConfigs);
return refinedConfigs;
@@ -601,11 +607,16 @@ private void maybeOverrideClientId(Map configs) {
protected static Map appendDeserializerToConfig(Map configs,
Deserializer> keyDeserializer,
Deserializer> valueDeserializer) {
+ // validate deserializer configuration, if the passed deserializer instance is null, the user must explicitly set a valid deserializer configuration value
Map newConfigs = new HashMap<>(configs);
if (keyDeserializer != null)
newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass());
+ else if (newConfigs.get(KEY_DESERIALIZER_CLASS_CONFIG) == null)
+ throw new ConfigException(KEY_DESERIALIZER_CLASS_CONFIG, null, "must be non-null.");
if (valueDeserializer != null)
newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass());
+ else if (newConfigs.get(VALUE_DESERIALIZER_CLASS_CONFIG) == null)
+ throw new ConfigException(VALUE_DESERIALIZER_CLASS_CONFIG, null, "must be non-null.");
return newConfigs;
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
index 0fd4ea9cf4213..6ffb772915d6a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
@@ -563,7 +563,7 @@ public class KafkaConsumer implements Consumer {
private static final long NO_CURRENT_THREAD = -1L;
private static final String JMX_PREFIX = "kafka.consumer";
static final long DEFAULT_CLOSE_TIMEOUT_MS = 30 * 1000;
- private static final String DEFAULT_REASON = "rebalance enforced by user";
+ static final String DEFAULT_REASON = "rebalance enforced by user";
// Visible for testing
final Metrics metrics;
@@ -774,8 +774,12 @@ public KafkaConsumer(Map configs,
);
// no coordinator will be constructed for the default (null) group id
- this.coordinator = !groupId.isPresent() ? null :
- new ConsumerCoordinator(groupRebalanceConfig,
+ if (!groupId.isPresent()) {
+ config.ignore(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
+ config.ignore(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED);
+ this.coordinator = null;
+ } else {
+ this.coordinator = new ConsumerCoordinator(groupRebalanceConfig,
logContext,
this.client,
assignors,
@@ -788,6 +792,7 @@ public KafkaConsumer(Map configs,
config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG),
this.interceptors,
config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED));
+ }
this.fetcher = new Fetcher<>(
logContext,
this.client,
@@ -1120,7 +1125,7 @@ public void assign(Collection partitions) {
if (coordinator != null)
this.coordinator.maybeAutoCommitOffsetsAsync(time.milliseconds());
- log.info("Subscribed to partition(s): {}", Utils.join(partitions, ", "));
+ log.info("Assigned to partition(s): {}", Utils.join(partitions, ", "));
if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
metadata.requestUpdateForNewTopics();
}
@@ -2322,7 +2327,7 @@ public void enforceRebalance(final String reason) {
if (coordinator == null) {
throw new IllegalStateException("Tried to force a rebalance but consumer does not have a group.");
}
- coordinator.requestRejoin(reason == null ? DEFAULT_REASON : DEFAULT_REASON + ": " + reason);
+ coordinator.requestRejoin(reason == null || reason.isEmpty() ? DEFAULT_REASON : reason);
} finally {
release();
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java
index 6d742b850a134..8b2297c96865e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java
@@ -16,6 +16,13 @@
*/
package org.apache.kafka.clients.consumer;
+import java.util.Locale;
+
public enum OffsetResetStrategy {
- LATEST, EARLIEST, NONE
+ LATEST, EARLIEST, NONE;
+
+ @Override
+ public String toString() {
+ return super.toString().toLowerCase(Locale.ROOT);
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
index 5b9712f34667e..d2ece9efc587c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
@@ -187,11 +187,12 @@ public AbstractCoordinator(GroupRebalanceConfig rebalanceConfig,
/**
* Invoked prior to each group join or rejoin. This is typically used to perform any
* cleanup from the previous generation (such as committing offsets for the consumer)
+ * @param timer Timer bounding how long this method can block
* @param generation The previous generation or -1 if there was none
* @param memberId The identifier of this member in the previous group or "" if there was none
* @return true If onJoinPrepare async commit succeeded, false otherwise
*/
- protected abstract boolean onJoinPrepare(int generation, String memberId);
+ protected abstract boolean onJoinPrepare(Timer timer, int generation, String memberId);
/**
* Invoked when the leader is elected. This is used by the leader to perform the assignment
@@ -426,7 +427,7 @@ boolean joinGroupIfNeeded(final Timer timer) {
// exception, in which case upon retry we should not retry onJoinPrepare either.
needsJoinPrepare = false;
// return false when onJoinPrepare is waiting for committing offset
- if (!onJoinPrepare(generation.generationId, generation.memberId)) {
+ if (!onJoinPrepare(timer, generation.generationId, generation.memberId)) {
needsJoinPrepare = true;
//should not initiateJoinGroup if needsJoinPrepare still is true
return false;
@@ -478,8 +479,12 @@ boolean joinGroupIfNeeded(final Timer timer) {
resetJoinGroupFuture();
synchronized (AbstractCoordinator.this) {
- rejoinReason = String.format("rebalance failed due to '%s' (%s)", exception.getMessage(), exception.getClass().getSimpleName());
- rejoinNeeded = true;
+ final String simpleName = exception.getClass().getSimpleName();
+ final String shortReason = String.format("rebalance failed due to %s", simpleName);
+ final String fullReason = String.format("rebalance failed due to '%s' (%s)",
+ exception.getMessage(),
+ simpleName);
+ requestRejoin(shortReason, fullReason);
}
if (exception instanceof UnknownMemberIdException ||
@@ -555,7 +560,7 @@ RequestFuture sendJoinGroupRequest() {
.setProtocolType(protocolType())
.setProtocols(metadata())
.setRebalanceTimeoutMs(this.rebalanceConfig.rebalanceTimeoutMs)
- .setReason(this.rejoinReason)
+ .setReason(JoinGroupRequest.maybeTruncateReason(this.rejoinReason))
);
log.debug("Sending JoinGroup ({}) to coordinator {}", requestBuilder, this.coordinator);
@@ -807,6 +812,9 @@ public void handle(SyncGroupResponse syncResponse,
} else if (error == Errors.REBALANCE_IN_PROGRESS) {
log.info("SyncGroup failed: The group began another rebalance. Need to re-join the group. " +
"Sent generation was {}", sentGeneration);
+ // consumer didn't get assignment in this generation, so we need to reset generation
+ // to avoid joinGroup with out-of-data ownedPartitions in cooperative rebalance
+ resetStateOnResponseError(ApiKeys.SYNC_GROUP, error, false);
future.raise(error);
} else if (error == Errors.FENCED_INSTANCE_ID) {
// for sync-group request, even if the generation has changed we would not expect the instance id
@@ -939,7 +947,7 @@ protected synchronized void markCoordinatorUnknown(String cause) {
protected synchronized void markCoordinatorUnknown(boolean isDisconnected, String cause) {
if (this.coordinator != null) {
- log.info("Group coordinator {} is unavailable or invalid due to cause: {}."
+ log.info("Group coordinator {} is unavailable or invalid due to cause: {}. "
+ "isDisconnected: {}. Rediscovery will be attempted.", this.coordinator,
cause, isDisconnected);
Node oldCoordinator = this.coordinator;
@@ -1022,15 +1030,28 @@ synchronized void resetGenerationOnLeaveGroup() {
resetStateAndRejoin("consumer pro-actively leaving the group", true);
}
- public synchronized void requestRejoinIfNecessary(final String reason) {
+ public synchronized void requestRejoinIfNecessary(final String shortReason,
+ final String fullReason) {
if (!this.rejoinNeeded) {
- requestRejoin(reason);
+ requestRejoin(shortReason, fullReason);
}
}
- public synchronized void requestRejoin(final String reason) {
- log.info("Request joining group due to: {}", reason);
- this.rejoinReason = reason;
+ public synchronized void requestRejoin(final String shortReason) {
+ requestRejoin(shortReason, shortReason);
+ }
+
+ /**
+ * Request to rejoin the group.
+ *
+ * @param shortReason This is the reason passed up to the group coordinator. It must be
+ * reasonably small.
+ * @param fullReason This is the reason logged locally.
+ */
+ public synchronized void requestRejoin(final String shortReason,
+ final String fullReason) {
+ log.info("Request joining group due to: {}", fullReason);
+ this.rejoinReason = shortReason;
this.rejoinNeeded = true;
}
@@ -1094,7 +1115,7 @@ public synchronized RequestFuture maybeLeaveGroup(String leaveReason) {
generation.memberId, coordinator, leaveReason);
LeaveGroupRequest.Builder request = new LeaveGroupRequest.Builder(
rebalanceConfig.groupId,
- Collections.singletonList(new MemberIdentity().setMemberId(generation.memberId).setReason(leaveReason))
+ Collections.singletonList(new MemberIdentity().setMemberId(generation.memberId).setReason(JoinGroupRequest.maybeTruncateReason(leaveReason)))
);
future = client.send(coordinator, request).compose(new LeaveGroupResponseHandler(generation));
@@ -1425,12 +1446,11 @@ public void run() {
// clear the future so that after the backoff, if the hb still sees coordinator unknown in
// the next iteration it will try to re-discover the coordinator in case the main thread cannot
clearFindCoordinatorFuture();
-
- // backoff properly
- AbstractCoordinator.this.wait(rebalanceConfig.retryBackoffMs);
} else {
lookupCoordinator();
}
+ // backoff properly
+ AbstractCoordinator.this.wait(rebalanceConfig.retryBackoffMs);
} else if (heartbeat.sessionTimeoutExpired(now)) {
// the session timeout has expired without seeing a successful heartbeat, so we should
// probably make sure the coordinator is still healthy.
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
index 84cf822a43a3e..9838e7dc8fe4b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -16,6 +16,7 @@
*/
package org.apache.kafka.clients.consumer.internals;
+import java.time.Duration;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.kafka.clients.GroupRebalanceConfig;
@@ -140,6 +141,12 @@ private boolean sameRequest(final Set currentRequest, final Gene
}
private final RebalanceProtocol protocol;
+ // pending commit offset request in onJoinPrepare
+ private RequestFuture autoCommitOffsetRequestFuture = null;
+ // a timer for join prepare to know when to stop.
+ // it'll set to rebalance timeout so that the member can join the group successfully
+ // even though offset commit failed.
+ private Timer joinPrepareTimer = null;
/**
* Initialize the coordination manager.
@@ -401,10 +408,10 @@ protected void onJoinComplete(int generation,
assignedPartitions.addAll(assignment.partitions());
if (!subscriptions.checkAssignmentMatchedSubscription(assignedPartitions)) {
- final String reason = String.format("received assignment %s does not match the current subscription %s; " +
+ final String fullReason = String.format("received assignment %s does not match the current subscription %s; " +
"it is likely that the subscription has changed since we joined the group, will re-join with current subscription",
assignment.partitions(), subscriptions.prettyString());
- requestRejoin(reason);
+ requestRejoin("received assignment does not match the current subscription", fullReason);
return;
}
@@ -437,9 +444,9 @@ protected void onJoinComplete(int generation,
firstException.compareAndSet(null, invokePartitionsRevoked(revokedPartitions));
// If revoked any partitions, need to re-join the group afterwards
- final String reason = String.format("need to revoke partitions %s as indicated " +
+ final String fullReason = String.format("need to revoke partitions %s as indicated " +
"by the current assignment and re-join", revokedPartitions);
- requestRejoin(reason);
+ requestRejoin("need to revoke partitions and re-join", fullReason);
}
}
@@ -548,14 +555,18 @@ public boolean poll(Timer timer, boolean waitForJoinGroup) {
}
}
} else {
- // For manually assigned partitions, if coordinator is unknown, make sure we lookup one and await metadata.
+ // For manually assigned partitions, we do not try to pro-actively lookup coordinator;
+ // instead we only try to refresh metadata when necessary.
// If connections to all nodes fail, wakeups triggered while attempting to send fetch
// requests result in polls returning immediately, causing a tight loop of polls. Without
// the wakeup, poll() with no channels would block for the timeout, delaying re-connection.
// awaitMetadataUpdate() in ensureCoordinatorReady initiates new connections with configured backoff and avoids the busy loop.
- if (coordinatorUnknownAndUnready(timer)) {
- return false;
+ if (metadata.updateRequested() && !client.hasReadyNodes(timer.currentTimeMs())) {
+ client.awaitMetadataUpdate(timer);
}
+
+ // if there is pending coordinator requests, ensure they have a chance to be transmitted.
+ client.pollNoWakeup();
}
maybeAutoCommitOffsetsAsync(timer.currentTimeMs());
@@ -735,24 +746,58 @@ private void validateCooperativeAssignment(final Map future = maybeAutoCommitOffsetsAsync();
- // return true when
- // 1. future is null, which means no commit request sent, so it is still considered completed
- // 2. offset commit completed
- // 3. offset commit failed with non-retriable exception
- if (future == null)
- onJoinPrepareAsyncCommitCompleted = true;
- else if (future.succeeded())
- onJoinPrepareAsyncCommitCompleted = true;
- else if (future.failed() && !future.isRetriable()) {
- log.error("Asynchronous auto-commit of offsets failed: {}", future.exception().getMessage());
- onJoinPrepareAsyncCommitCompleted = true;
+ // and there is no in-flight offset commit request
+ if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) {
+ autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync();
}
+ // wait for commit offset response before timer expired
+ if (autoCommitOffsetRequestFuture != null) {
+ Timer pollTimer = timer.remainingMs() < joinPrepareTimer.remainingMs() ?
+ timer : joinPrepareTimer;
+ client.poll(autoCommitOffsetRequestFuture, pollTimer);
+ joinPrepareTimer.update();
+
+ // Keep retrying/waiting the offset commit when:
+ // 1. offset commit haven't done (and joinPrepareTimer not expired)
+ // 2. failed with retryable exception (and joinPrepareTimer not expired)
+ // Otherwise, continue to revoke partitions, ex:
+ // 1. if joinPrepareTime has expired
+ // 2. if offset commit failed with no-retryable exception
+ // 3. if offset commit success
+ boolean onJoinPrepareAsyncCommitCompleted = true;
+ if (joinPrepareTimer.isExpired()) {
+ log.error("Asynchronous auto-commit of offsets failed: joinPrepare timeout. Will continue to join group");
+ } else if (!autoCommitOffsetRequestFuture.isDone()) {
+ onJoinPrepareAsyncCommitCompleted = false;
+ } else if (autoCommitOffsetRequestFuture.failed() && autoCommitOffsetRequestFuture.isRetriable()) {
+ log.debug("Asynchronous auto-commit of offsets failed with retryable error: {}. Will retry it.",
+ autoCommitOffsetRequestFuture.exception().getMessage());
+ onJoinPrepareAsyncCommitCompleted = false;
+ } else if (autoCommitOffsetRequestFuture.failed() && !autoCommitOffsetRequestFuture.isRetriable()) {
+ log.error("Asynchronous auto-commit of offsets failed: {}. Will continue to join group.",
+ autoCommitOffsetRequestFuture.exception().getMessage());
+ }
+ if (autoCommitOffsetRequestFuture.isDone()) {
+ autoCommitOffsetRequestFuture = null;
+ }
+ if (!onJoinPrepareAsyncCommitCompleted) {
+ pollTimer.sleep(Math.min(pollTimer.remainingMs(), rebalanceConfig.retryBackoffMs));
+ timer.update();
+ return false;
+ }
+ }
// the generation / member-id can possibly be reset by the heartbeat thread
// upon getting errors or heartbeat timeouts; in this case whatever is previously
@@ -804,11 +849,14 @@ else if (future.failed() && !future.isRetriable()) {
isLeader = false;
subscriptions.resetGroupSubscription();
+ joinPrepareTimer = null;
+ autoCommitOffsetRequestFuture = null;
+ timer.update();
if (exception != null) {
throw new KafkaException("User rebalance callback throws an error", exception);
}
- return onJoinPrepareAsyncCommitCompleted;
+ return true;
}
@Override
@@ -851,17 +899,17 @@ public boolean rejoinNeededOrPending() {
// we need to rejoin if we performed the assignment and metadata has changed;
// also for those owned-but-no-longer-existed partitions we should drop them as lost
if (assignmentSnapshot != null && !assignmentSnapshot.matches(metadataSnapshot)) {
- final String reason = String.format("cached metadata has changed from %s at the beginning of the rebalance to %s",
+ final String fullReason = String.format("cached metadata has changed from %s at the beginning of the rebalance to %s",
assignmentSnapshot, metadataSnapshot);
- requestRejoinIfNecessary(reason);
+ requestRejoinIfNecessary("cached metadata has changed", fullReason);
return true;
}
// we need to join if our subscription has changed since the last join
if (joinedSubscription != null && !joinedSubscription.equals(subscriptions.subscription())) {
- final String reason = String.format("subscription has changed from %s at the beginning of the rebalance to %s",
+ final String fullReason = String.format("subscription has changed from %s at the beginning of the rebalance to %s",
joinedSubscription, subscriptions.subscription());
- requestRejoinIfNecessary(reason);
+ requestRejoinIfNecessary("subscription has changed", fullReason);
return true;
}
@@ -970,7 +1018,7 @@ public void close(final Timer timer) {
// we do not need to re-enable wakeups since we are closing already
client.disableWakeups();
try {
- maybeAutoCommitOffsetsAsync();
+ maybeAutoCommitOffsetsSync(timer);
while (pendingAsyncCommits.get() > 0 && timer.notExpired()) {
ensureCoordinatorReady(timer);
client.poll(timer);
@@ -1004,7 +1052,17 @@ public RequestFuture commitOffsetsAsync(final Map offsets,
return false;
}
+ private void maybeAutoCommitOffsetsSync(Timer timer) {
+ if (autoCommitEnabled) {
+ Map allConsumedOffsets = subscriptions.allConsumed();
+ try {
+ log.debug("Sending synchronous auto-commit of offsets {}", allConsumedOffsets);
+ if (!commitOffsetsSync(allConsumedOffsets, timer))
+ log.debug("Auto-commit of offsets {} timed out before completion", allConsumedOffsets);
+ } catch (WakeupException | InterruptException e) {
+ log.debug("Auto-commit of offsets {} was interrupted before completion", allConsumedOffsets);
+ // rethrow wakeups since they are triggered by the user
+ throw e;
+ } catch (Exception e) {
+ // consistent with async auto-commit failures, we do not propagate the exception
+ log.warn("Synchronous auto-commit of offsets {} failed: {}", allConsumedOffsets, e.getMessage());
+ }
+ }
+ }
+
public void maybeAutoCommitOffsetsAsync(long now) {
if (autoCommitEnabled) {
nextAutoCommitTimer.update(now);
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
index 54f70cafd72c8..73ffd217efe21 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
@@ -1201,7 +1201,7 @@ private Map prepareFetchRequests() {
continue;
}
- // Use the preferred read replica if set, otherwise the position's leader
+ // Use the preferred read replica if set, otherwise the partition's leader
Node node = selectReadReplica(partition, leaderOpt.get(), currentTimeMs);
if (client.isUnavailable(node)) {
client.maybeThrowAuthFailure(node);
@@ -1866,12 +1866,11 @@ private void maybeUpdateAssignment(SubscriptionState subscription) {
for (TopicPartition tp : newAssignedPartitions) {
if (!this.assignedPartitions.contains(tp)) {
MetricName metricName = partitionPreferredReadReplicaMetricName(tp);
- if (metrics.metric(metricName) == null) {
- metrics.addMetric(
- metricName,
- (Gauge) (config, now) -> subscription.preferredReadReplica(tp, 0L).orElse(-1)
- );
- }
+ metrics.addMetricIfAbsent(
+ metricName,
+ null,
+ (Gauge) (config, now) -> subscription.preferredReadReplica(tp, 0L).orElse(-1)
+ );
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
index 501ffe9a88da8..f76a92462d5ae 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
@@ -96,7 +96,7 @@ public FetcherMetricsRegistry(Set tags, String metricGrpPrefix) {
"The total number of fetch requests.", tags);
this.recordsLagMax = new MetricNameTemplate("records-lag-max", groupName,
- "The maximum lag in terms of number of records for any partition in this window", tags);
+ "The maximum lag in terms of number of records for any partition in this window. NOTE: This is based on current offset and not committed offset", tags);
this.recordsLeadMin = new MetricNameTemplate("records-lead-min", groupName,
"The minimum lead in terms of number of records for any partition in this window", tags);
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
index 416468d945fcb..0225822a19466 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
@@ -16,7 +16,6 @@
*/
package org.apache.kafka.clients.consumer.internals;
-import java.util.TreeSet;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.Metadata;
import org.apache.kafka.clients.NodeApiVersions;
@@ -43,6 +42,7 @@
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
+import java.util.TreeSet;
import java.util.function.LongSupplier;
import java.util.function.Predicate;
import java.util.regex.Pattern;
@@ -617,7 +617,7 @@ public synchronized Optional preferredReadReplica(TopicPartition tp, lo
* Unset the preferred read replica. This causes the fetcher to go back to the leader for fetches.
*
* @param tp The topic partition
- * @return true if the preferred read replica was set, false otherwise.
+ * @return the removed preferred read replica if set, None otherwise.
*/
public synchronized Optional clearPreferredReadReplica(TopicPartition tp) {
return assignedState(tp).clearPreferredReadReplica();
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index f739336b85692..2d5c8994b4df9 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -26,6 +26,7 @@
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
import org.apache.kafka.clients.producer.internals.BufferPool;
+import org.apache.kafka.clients.producer.internals.BuiltInPartitioner;
import org.apache.kafka.clients.producer.internals.KafkaProducerMetrics;
import org.apache.kafka.clients.producer.internals.ProducerInterceptors;
import org.apache.kafka.clients.producer.internals.ProducerMetadata;
@@ -254,6 +255,7 @@ public class KafkaProducer implements Producer {
private final Serializer valueSerializer;
private final ProducerConfig producerConfig;
private final long maxBlockTimeMs;
+ private final boolean partitionerIgnoreKeys;
private final ProducerInterceptors interceptors;
private final ApiVersions apiVersions;
private final TransactionManager transactionManager;
@@ -316,6 +318,23 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali
this(Utils.propsToMap(properties), keySerializer, valueSerializer);
}
+ /**
+ * Check if partitioner is deprecated and log a warning if it is.
+ */
+ @SuppressWarnings("deprecation")
+ private void warnIfPartitionerDeprecated() {
+ // Using DefaultPartitioner and UniformStickyPartitioner is deprecated, see KIP-794.
+ if (partitioner instanceof org.apache.kafka.clients.producer.internals.DefaultPartitioner) {
+ log.warn("DefaultPartitioner is deprecated. Please clear " + ProducerConfig.PARTITIONER_CLASS_CONFIG
+ + " configuration setting to get the default partitioning behavior");
+ }
+ if (partitioner instanceof org.apache.kafka.clients.producer.UniformStickyPartitioner) {
+ log.warn("UniformStickyPartitioner is deprecated. Please clear " + ProducerConfig.PARTITIONER_CLASS_CONFIG
+ + " configuration setting and set " + ProducerConfig.PARTITIONER_IGNORE_KEYS_CONFIG
+ + " to 'true' to get the uniform sticky partitioning behavior");
+ }
+ }
+
// visible for testing
@SuppressWarnings("unchecked")
KafkaProducer(ProducerConfig config,
@@ -360,6 +379,8 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali
ProducerConfig.PARTITIONER_CLASS_CONFIG,
Partitioner.class,
Collections.singletonMap(ProducerConfig.CLIENT_ID_CONFIG, clientId));
+ warnIfPartitionerDeprecated();
+ this.partitionerIgnoreKeys = config.getBoolean(ProducerConfig.PARTITIONER_IGNORE_KEYS_CONFIG);
long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG);
if (keySerializer == null) {
this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
@@ -397,12 +418,20 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali
this.apiVersions = new ApiVersions();
this.transactionManager = configureTransactionState(config, logContext);
+ // There is no need to do work required for adaptive partitioning, if we use a custom partitioner.
+ boolean enableAdaptivePartitioning = partitioner == null &&
+ config.getBoolean(ProducerConfig.PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG);
+ RecordAccumulator.PartitionerConfig partitionerConfig = new RecordAccumulator.PartitionerConfig(
+ enableAdaptivePartitioning,
+ config.getLong(ProducerConfig.PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG)
+ );
this.accumulator = new RecordAccumulator(logContext,
config.getInt(ProducerConfig.BATCH_SIZE_CONFIG),
this.compressionType,
lingerMs(config),
retryBackoffMs,
deliveryTimeoutMs,
+ partitionerConfig,
metrics,
PRODUCER_METRIC_GROUP_NAME,
time,
@@ -440,6 +469,44 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali
}
}
+ // visible for testing
+ KafkaProducer(ProducerConfig config,
+ LogContext logContext,
+ Metrics metrics,
+ Serializer keySerializer,
+ Serializer valueSerializer,
+ ProducerMetadata metadata,
+ RecordAccumulator accumulator,
+ TransactionManager transactionManager,
+ Sender sender,
+ ProducerInterceptors interceptors,
+ Partitioner partitioner,
+ Time time,
+ KafkaThread ioThread) {
+ this.producerConfig = config;
+ this.time = time;
+ this.clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG);
+ this.log = logContext.logger(KafkaProducer.class);
+ this.metrics = metrics;
+ this.producerMetrics = new KafkaProducerMetrics(metrics);
+ this.partitioner = partitioner;
+ this.keySerializer = keySerializer;
+ this.valueSerializer = valueSerializer;
+ this.interceptors = interceptors;
+ this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
+ this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG);
+ this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG));
+ this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG);
+ this.partitionerIgnoreKeys = config.getBoolean(ProducerConfig.PARTITIONER_IGNORE_KEYS_CONFIG);
+ this.apiVersions = new ApiVersions();
+ this.transactionManager = transactionManager;
+ this.accumulator = accumulator;
+ this.errors = this.metrics.sensor("errors");
+ this.metadata = metadata;
+ this.sender = sender;
+ this.ioThread = ioThread;
+ }
+
// visible for testing
Sender newSender(LogContext logContext, KafkaClient kafkaClient, ProducerMetadata metadata) {
int maxInflightRequests = producerConfig.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION);
@@ -530,6 +597,9 @@ private TransactionManager configureTransactionState(ProducerConfig config,
log.info("Instantiated a transactional producer.");
else
log.info("Instantiated an idempotent producer.");
+ } else {
+ // ignore unretrieved configurations related to producer transaction
+ config.ignore(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG);
}
return transactionManager;
}
@@ -685,11 +755,15 @@ public void sendOffsetsToTransaction(Map offs
/**
* Commits the ongoing transaction. This method will flush any unsent records before actually committing the transaction.
- *
+ *
* Further, if any of the {@link #send(ProducerRecord)} calls which were part of the transaction hit irrecoverable
* errors, this method will throw the last received exception immediately and the transaction will not be committed.
* So all {@link #send(ProducerRecord)} calls in a transaction must succeed in order for this method to succeed.
- *
+ *
+ * If the transaction is committed successfully and this method returns without throwing an exception, it is guaranteed
+ * that all {@link Callback callbacks} for records in the transaction will have been invoked and completed.
+ * Note that exceptions thrown by callbacks are ignored; the producer proceeds to commit the transaction in any case.
+ *
* Note that this method will raise {@link TimeoutException} if the transaction cannot be committed before expiration
* of {@code max.block.ms}. Additionally, it will raise {@link InterruptException} if interrupted.
* It is safe to retry in either case, but it is not possible to attempt a different operation (such as abortTransaction)
@@ -768,8 +842,9 @@ public Future send(ProducerRecord record) {
* response after each one.
*
* The result of the send is a {@link RecordMetadata} specifying the partition the record was sent to, the offset
- * it was assigned and the timestamp of the record. If
- * {@link org.apache.kafka.common.record.TimestampType#CREATE_TIME CreateTime} is used by the topic, the timestamp
+ * it was assigned and the timestamp of the record. If the producer is configured with acks = 0, the {@link RecordMetadata}
+ * will have offset = -1 because the producer does not wait for the acknowledgement from the broker.
+ * If {@link org.apache.kafka.common.record.TimestampType#CREATE_TIME CreateTime} is used by the topic, the timestamp
* will be the user provided timestamp or the record send time if the user did not specify a timestamp for the
* record. If {@link org.apache.kafka.common.record.TimestampType#LOG_APPEND_TIME LogAppendTime} is used for the
* topic, the timestamp will be the Kafka broker local time when the message is appended.
@@ -881,11 +956,24 @@ private void throwIfProducerClosed() {
throw new IllegalStateException("Cannot perform operation after producer has been closed");
}
+ /**
+ * Call deprecated {@link Partitioner#onNewBatch}
+ */
+ @SuppressWarnings("deprecation")
+ private void onNewBatch(String topic, Cluster cluster, int prevPartition) {
+ assert partitioner != null;
+ partitioner.onNewBatch(topic, cluster, prevPartition);
+ }
+
/**
* Implementation of asynchronously send a record to a topic.
*/
private Future doSend(ProducerRecord record, Callback callback) {
- TopicPartition tp = null;
+ // Append callback takes care of the following:
+ // - call interceptors and user callback on completion
+ // - remember partition that is calculated in RecordAccumulator.append
+ AppendCallbacks appendCallbacks = new AppendCallbacks(callback, this.interceptors, record);
+
try {
throwIfProducerClosed();
// first make sure the metadata for the topic is available
@@ -917,8 +1005,11 @@ private Future doSend(ProducerRecord record, Callback call
" to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() +
" specified in value.serializer", cce);
}
+
+ // Try to calculate partition, but note that after this call it can be RecordMetadata.UNKNOWN_PARTITION,
+ // which means that the RecordAccumulator would pick a partition using built-in logic (which may
+ // take into account broker load, the amount of data produced to each partition, etc.).
int partition = partition(record, serializedKey, serializedValue, cluster);
- tp = new TopicPartition(record.topic(), partition);
setReadOnly(record.headers());
Header[] headers = record.headers().toArray();
@@ -927,36 +1018,38 @@ private Future doSend(ProducerRecord record, Callback call
compressionType, serializedKey, serializedValue, headers);
ensureValidRecordSize(serializedSize);
long timestamp = record.timestamp() == null ? nowMs : record.timestamp();
- if (log.isTraceEnabled()) {
- log.trace("Attempting to append record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition);
- }
- // producer callback will make sure to call both 'callback' and interceptor callback
- Callback interceptCallback = new InterceptorCallback<>(callback, this.interceptors, tp);
- if (transactionManager != null) {
- transactionManager.maybeAddPartition(tp);
- }
+ // A custom partitioner may take advantage on the onNewBatch callback.
+ boolean abortOnNewBatch = partitioner != null;
- RecordAccumulator.RecordAppendResult result = accumulator.append(tp, timestamp, serializedKey,
- serializedValue, headers, interceptCallback, remainingWaitMs, true, nowMs);
+ // Append the record to the accumulator. Note, that the actual partition may be
+ // calculated there and can be accessed via appendCallbacks.topicPartition.
+ RecordAccumulator.RecordAppendResult result = accumulator.append(record.topic(), partition, timestamp, serializedKey,
+ serializedValue, headers, appendCallbacks, remainingWaitMs, abortOnNewBatch, nowMs, cluster);
+ assert appendCallbacks.getPartition() != RecordMetadata.UNKNOWN_PARTITION;
if (result.abortForNewBatch) {
int prevPartition = partition;
- partitioner.onNewBatch(record.topic(), cluster, prevPartition);
+ onNewBatch(record.topic(), cluster, prevPartition);
partition = partition(record, serializedKey, serializedValue, cluster);
- tp = new TopicPartition(record.topic(), partition);
if (log.isTraceEnabled()) {
log.trace("Retrying append due to new batch creation for topic {} partition {}. The old partition was {}", record.topic(), partition, prevPartition);
}
- // producer callback will make sure to call both 'callback' and interceptor callback
- interceptCallback = new InterceptorCallback<>(callback, this.interceptors, tp);
+ result = accumulator.append(record.topic(), partition, timestamp, serializedKey,
+ serializedValue, headers, appendCallbacks, remainingWaitMs, false, nowMs, cluster);
+ }
- result = accumulator.append(tp, timestamp, serializedKey,
- serializedValue, headers, interceptCallback, remainingWaitMs, false, nowMs);
+ // Add the partition to the transaction (if in progress) after it has been successfully
+ // appended to the accumulator. We cannot do it before because the partition may be
+ // unknown or the initially selected partition may be changed when the batch is closed
+ // (as indicated by `abortForNewBatch`). Note that the `Sender` will refuse to dequeue
+ // batches from the accumulator until they have been added to the transaction.
+ if (transactionManager != null) {
+ transactionManager.maybeAddPartition(appendCallbacks.topicPartition());
}
if (result.batchIsFull || result.newBatchCreated) {
- log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), partition);
+ log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), appendCallbacks.getPartition());
this.sender.wakeup();
}
return result.future;
@@ -965,34 +1058,28 @@ private Future doSend(ProducerRecord record, Callback call
// for other exceptions throw directly
} catch (ApiException e) {
log.debug("Exception occurred during message send:", e);
- // producer callback will make sure to call both 'callback' and interceptor callback
- if (tp == null) {
- // set topicPartition to -1 when null
- tp = ProducerInterceptors.extractTopicPartition(record);
+ if (callback != null) {
+ TopicPartition tp = appendCallbacks.topicPartition();
+ RecordMetadata nullMetadata = new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1);
+ callback.onCompletion(nullMetadata, e);
}
-
- Callback interceptCallback = new InterceptorCallback<>(callback, this.interceptors, tp);
-
- // The onCompletion callback does expect a non-null metadata, but one will be created inside
- // the interceptor's onCompletion implementation before the user's callback is invoked.
- interceptCallback.onCompletion(null, e);
this.errors.record();
- this.interceptors.onSendError(record, tp, e);
+ this.interceptors.onSendError(record, appendCallbacks.topicPartition(), e);
if (transactionManager != null) {
transactionManager.maybeTransitionToErrorState(e);
}
return new FutureFailure(e);
} catch (InterruptedException e) {
this.errors.record();
- this.interceptors.onSendError(record, tp, e);
+ this.interceptors.onSendError(record, appendCallbacks.topicPartition(), e);
throw new InterruptException(e);
} catch (KafkaException e) {
this.errors.record();
- this.interceptors.onSendError(record, tp, e);
+ this.interceptors.onSendError(record, appendCallbacks.topicPartition(), e);
throw e;
} catch (Exception e) {
// we notify interceptor about all exceptions, since onSend is called before anything else in this method
- this.interceptors.onSendError(record, tp, e);
+ this.interceptors.onSendError(record, appendCallbacks.topicPartition(), e);
throw e;
}
}
@@ -1033,6 +1120,7 @@ private ClusterAndWaitTime waitOnMetadata(String topic, Integer partition, long
// Issue metadata requests until we have metadata for the topic and the requested partition,
// or until maxWaitTimeMs is exceeded. This is necessary in case the metadata
// is stale and the number of partitions for this topic has increased in the meantime.
+ long nowNanos = time.nanoseconds();
do {
if (partition != null) {
log.trace("Requesting metadata update for partition {} of topic {}.", partition, topic);
@@ -1064,6 +1152,8 @@ private ClusterAndWaitTime waitOnMetadata(String topic, Integer partition, long
partitionsCount = cluster.partitionCountForTopic(topic);
} while (partitionsCount == null || (partition != null && partition >= partitionsCount));
+ producerMetrics.recordMetadataWait(time.nanoseconds() - nowNanos);
+
return new ClusterAndWaitTime(cluster, elapsed);
}
@@ -1273,21 +1363,33 @@ private ClusterResourceListeners configureClusterResourceListeners(Serializer
/**
* computes partition for given record.
* if the record has partition returns the value otherwise
- * calls configured partitioner class to compute the partition.
+ * if custom partitioner is specified, call it to compute partition
+ * otherwise try to calculate partition based on key.
+ * If there is no key or key should be ignored return
+ * RecordMetadata.UNKNOWN_PARTITION to indicate any partition
+ * can be used (the partition is then calculated by built-in
+ * partitioning logic).
*/
private int partition(ProducerRecord record, byte[] serializedKey, byte[] serializedValue, Cluster cluster) {
- Integer partition = record.partition();
- if (partition != null) {
- return partition;
- }
+ if (record.partition() != null)
+ return record.partition();
- int customPartition = partitioner.partition(
+ if (partitioner != null) {
+ int customPartition = partitioner.partition(
record.topic(), record.key(), serializedKey, record.value(), serializedValue, cluster);
- if (customPartition < 0) {
- throw new IllegalArgumentException(String.format(
+ if (customPartition < 0) {
+ throw new IllegalArgumentException(String.format(
"The partitioner generated an invalid partition number: %d. Partition number should always be non-negative.", customPartition));
+ }
+ return customPartition;
+ }
+
+ if (serializedKey != null && !partitionerIgnoreKeys) {
+ // hash the keyBytes to choose a partition
+ return BuiltInPartitioner.partitionForKey(serializedKey, cluster.partitionsForTopic(record.topic()).size());
+ } else {
+ return RecordMetadata.UNKNOWN_PARTITION;
}
- return customPartition;
}
private void throwIfInvalidGroupMetadata(ConsumerGroupMetadata groupMetadata) {
@@ -1355,25 +1457,66 @@ public boolean isDone() {
}
/**
- * A callback called when producer request is complete. It in turn calls user-supplied callback (if given) and
- * notifies producer interceptors about the request completion.
+ * Callbacks that are called by the RecordAccumulator append functions:
+ * - user callback
+ * - interceptor callbacks
+ * - partition callback
*/
- private static class InterceptorCallback implements Callback {
+ private class AppendCallbacks implements RecordAccumulator.AppendCallbacks {
private final Callback userCallback;
private final ProducerInterceptors interceptors;
- private final TopicPartition tp;
+ private final String topic;
+ private final Integer recordPartition;
+ private final String recordLogString;
+ private volatile int partition = RecordMetadata.UNKNOWN_PARTITION;
+ private volatile TopicPartition topicPartition;
- private InterceptorCallback(Callback userCallback, ProducerInterceptors interceptors, TopicPartition tp) {
+ private AppendCallbacks(Callback userCallback, ProducerInterceptors interceptors, ProducerRecord record) {
this.userCallback = userCallback;
this.interceptors = interceptors;
- this.tp = tp;
+ // Extract record info as we don't want to keep a reference to the record during
+ // whole lifetime of the batch.
+ // We don't want to have an NPE here, because the interceptors would not be notified (see .doSend).
+ topic = record != null ? record.topic() : null;
+ recordPartition = record != null ? record.partition() : null;
+ recordLogString = log.isTraceEnabled() && record != null ? record.toString() : "";
}
+ @Override
public void onCompletion(RecordMetadata metadata, Exception exception) {
- metadata = metadata != null ? metadata : new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1);
+ if (metadata == null) {
+ metadata = new RecordMetadata(topicPartition(), -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1);
+ }
this.interceptors.onAcknowledgement(metadata, exception);
if (this.userCallback != null)
this.userCallback.onCompletion(metadata, exception);
}
+
+ @Override
+ public void setPartition(int partition) {
+ assert partition != RecordMetadata.UNKNOWN_PARTITION;
+ this.partition = partition;
+
+ if (log.isTraceEnabled()) {
+ // Log the message here, because we don't know the partition before that.
+ log.trace("Attempting to append record {} with callback {} to topic {} partition {}", recordLogString, userCallback, topic, partition);
+ }
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ public TopicPartition topicPartition() {
+ if (topicPartition == null && topic != null) {
+ if (partition != RecordMetadata.UNKNOWN_PARTITION)
+ topicPartition = new TopicPartition(topic, partition);
+ else if (recordPartition != null)
+ topicPartition = new TopicPartition(topic, recordPartition);
+ else
+ topicPartition = new TopicPartition(topic, RecordMetadata.UNKNOWN_PARTITION);
+ }
+ return topicPartition;
+ }
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
index 4fd540dceaa8a..3df73b20a4d6e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
@@ -18,7 +18,6 @@
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.producer.internals.DefaultPartitioner;
import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
import org.apache.kafka.clients.producer.internals.ProduceRequestResult;
import org.apache.kafka.common.Cluster;
@@ -117,10 +116,24 @@ public MockProducer(final Cluster cluster,
*
* Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)}
*/
+ @SuppressWarnings("deprecation")
public MockProducer(final boolean autoComplete,
final Serializer keySerializer,
final Serializer valueSerializer) {
- this(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer);
+ this(Cluster.empty(), autoComplete, new org.apache.kafka.clients.producer.internals.DefaultPartitioner(), keySerializer, valueSerializer);
+ }
+
+ /**
+ * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers.
+ *
+ * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(cluster, autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)}
+ */
+ @SuppressWarnings("deprecation")
+ public MockProducer(final Cluster cluster,
+ final boolean autoComplete,
+ final Serializer keySerializer,
+ final Serializer valueSerializer) {
+ this(cluster, autoComplete, new org.apache.kafka.clients.producer.internals.DefaultPartitioner(), keySerializer, valueSerializer);
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java
index 13eaa5aaea9af..eeafc73d662c3 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java
@@ -44,12 +44,16 @@ public interface Partitioner extends Configurable, Closeable {
void close();
/**
+ * Note this method is only implemented in DefatultPartitioner and UniformStickyPartitioner which
+ * are now deprecated. See KIP-794 for more info.
+ *
* Notifies the partitioner a new batch is about to be created. When using the sticky partitioner,
- * this method can change the chosen sticky partition for the new batch.
+ * this method can change the chosen sticky partition for the new batch.
* @param topic The topic name
* @param cluster The current cluster metadata
* @param prevPartition The partition previously selected for the record that triggered a new batch
*/
+ @Deprecated
default void onNewBatch(String topic, Cluster cluster, int prevPartition) {
}
}
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 afc1e55cdfdad..aff5e49cfcb1b 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,7 +18,6 @@
import org.apache.kafka.clients.ClientDnsLookup;
import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.producer.internals.DefaultPartitioner;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
@@ -26,7 +25,10 @@
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.config.SecurityConfig;
import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -91,6 +93,26 @@ public class ProducerConfig extends AbstractConfig {
+ "This linger.ms
setting defaults to 0, which means we'll immediately send out a record even the accumulated "
+ "batch size is under this batch.size
setting.";
+ /** partitioner.adaptive.partitioning.enable
*/
+ public static final String PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG = "partitioner.adaptive.partitioning.enable";
+ private static final String PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC =
+ "When set to 'true', the producer will try to adapt to broker performance and produce more messages to partitions hosted on faster brokers. "
+ + "If 'false', producer will try to distribute messages uniformly. Note: this setting has no effect if a custom partitioner is used";
+
+ /** partitioner.availability.timeout.ms
*/
+ public static final String PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG = "partitioner.availability.timeout.ms";
+ private static final String PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC =
+ "If a broker cannot process produce requests from a partition for " + PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG + "
time, "
+ + "the partitioner treats that partition as not available. If the value is 0, this logic is disabled. "
+ + "Note: this setting has no effect if a custom partitioner is used or " + PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG
+ + "
is set to 'false'";
+
+ /** partitioner.ignore.keys
*/
+ public static final String PARTITIONER_IGNORE_KEYS_CONFIG = "partitioner.ignore.keys";
+ private static final String PARTITIONER_IGNORE_KEYS_DOC = "When set to 'true' the producer won't use record keys to choose a partition. "
+ + "If 'false', producer would choose a partition based on a hash of the key when a key is present. "
+ + "Note: this setting has no effect if a custom partitioner is used.";
+
/** acks
*/
public static final String ACKS_CONFIG = "acks";
private static final String ACKS_DOC = "The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the "
@@ -214,10 +236,11 @@ public class ProducerConfig extends AbstractConfig {
/** max.in.flight.requests.per.connection
*/
public static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "max.in.flight.requests.per.connection";
private static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC = "The maximum number of unacknowledged requests the client will send on a single connection before blocking."
- + " Note that if this config is set to be greater than 1 and enable.idempotence
is set to false, there is a risk of"
- + " message re-ordering after a failed send due to retries (i.e., if retries are enabled)."
- + " Additionally, enabling idempotence requires this config value to be less than or equal to " + MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_FOR_IDEMPOTENCE + "."
- + " If conflicting configurations are set and idempotence is not explicitly enabled, idempotence is disabled.";
+ + " Note that if this configuration is set to be greater than 1 and enable.idempotence
is set to false, there is a risk of"
+ + " message reordering after a failed send due to retries (i.e., if retries are enabled); "
+ + " if retries are disabled or if enable.idempotence
is set to true, ordering will be preserved."
+ + " Additionally, enabling idempotence requires the value of this configuration to be less than or equal to " + MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_FOR_IDEMPOTENCE + "."
+ + " If conflicting configurations are set and idempotence is not explicitly enabled, idempotence is disabled. ";
/** retries
*/
public static final String RETRIES_CONFIG = CommonClientConfigs.RETRIES_CONFIG;
@@ -256,11 +279,11 @@ public class ProducerConfig extends AbstractConfig {
public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class";
private static final String PARTITIONER_CLASS_DOC = "A class to use to determine which partition to be send to when produce the records. Available options are:" +
"" +
- "org.apache.kafka.clients.producer.internals.DefaultPartitioner
: The default partitioner. " +
- "This strategy will try sticking to a partition until the batch is full, or linger.ms
is up. It works with the strategy:" +
+ "- If not set, the default partitioning logic is used. " +
+ "This strategy will try sticking to a partition until " + BATCH_SIZE_CONFIG + " bytes is produced to the partition. It works with the strategy:" +
"
" +
"- If no partition is specified but a key is present, choose a partition based on a hash of the key
" +
- "- If no partition or key is present, choose the sticky partition that changes when the batch is full, or
linger.ms
is up. " +
+ "- If no partition or key is present, choose the sticky partition that changes when " + BATCH_SIZE_CONFIG + " bytes are produced to the partition.
" +
"
" +
" " +
"org.apache.kafka.clients.producer.RoundRobinPartitioner
: This partitioning strategy is that " +
@@ -268,9 +291,6 @@ public class ProducerConfig extends AbstractConfig {
"until we run out of partitions and start over again. Note: There's a known issue that will cause uneven distribution when new batch is created. " +
"Please check KAFKA-9965 for more detail." +
" " +
- "org.apache.kafka.clients.producer.UniformStickyPartitioner
: This partitioning strategy will " +
- "try sticking to a partition(no matter if the 'key' is provided or not) until the batch is full, or linger.ms
is up." +
- " " +
"
" +
"Implementing the org.apache.kafka.clients.producer.Partitioner
interface allows you to plug in a custom partitioner.";
@@ -329,8 +349,11 @@ public class ProducerConfig extends AbstractConfig {
in("all", "-1", "0", "1"),
Importance.LOW,
ACKS_DOC)
- .define(COMPRESSION_TYPE_CONFIG, Type.STRING, "none", Importance.HIGH, COMPRESSION_TYPE_DOC)
+ .define(COMPRESSION_TYPE_CONFIG, Type.STRING, CompressionType.NONE.name, in(Utils.enumOptions(CompressionType.class)), Importance.HIGH, COMPRESSION_TYPE_DOC)
.define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC)
+ .define(PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG, Type.BOOLEAN, true, Importance.LOW, PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC)
+ .define(PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG, Type.LONG, 0, atLeast(0), Importance.LOW, PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC)
+ .define(PARTITIONER_IGNORE_KEYS_CONFIG, Type.BOOLEAN, false, Importance.MEDIUM, PARTITIONER_IGNORE_KEYS_DOC)
.define(LINGER_MS_CONFIG, Type.LONG, 0, atLeast(0), Importance.MEDIUM, LINGER_MS_DOC)
.define(DELIVERY_TIMEOUT_MS_CONFIG, Type.INT, 120 * 1000, atLeast(0), Importance.MEDIUM, DELIVERY_TIMEOUT_MS_DOC)
.define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CommonClientConfigs.CLIENT_ID_DOC)
@@ -415,7 +438,7 @@ public class ProducerConfig extends AbstractConfig {
CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC)
.define(PARTITIONER_CLASS_CONFIG,
Type.CLASS,
- DefaultPartitioner.class,
+ null,
Importance.MEDIUM, PARTITIONER_CLASS_DOC)
.define(INTERCEPTOR_CLASSES_CONFIG,
Type.LIST,
@@ -426,6 +449,7 @@ public class ProducerConfig extends AbstractConfig {
.define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG,
Type.STRING,
CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL,
+ in(Utils.enumOptions(SecurityProtocol.class)),
Importance.MEDIUM,
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
.define(SECURITY_PROVIDERS_CONFIG,
@@ -455,6 +479,7 @@ public class ProducerConfig extends AbstractConfig {
@Override
protected Map postProcessParsedConfig(final Map parsedValues) {
+ CommonClientConfigs.postValidateSaslMechanismConfig(this);
Map refinedConfigs = CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues);
postProcessAndValidateIdempotenceConfigs(refinedConfigs);
maybeOverrideClientId(refinedConfigs);
@@ -488,11 +513,11 @@ private void postProcessAndValidateIdempotenceConfigs(final Map
if (userConfiguredIdempotence) {
throw new ConfigException("Must set " + RETRIES_CONFIG + " to non-zero when using the idempotent producer.");
}
- log.info("Idempotence will be disabled because {} is set to 0.", RETRIES_CONFIG, retries);
+ log.info("Idempotence will be disabled because {} is set to 0.", RETRIES_CONFIG);
shouldDisableIdempotence = true;
}
- final short acks = Short.valueOf(acksStr);
+ final short acks = Short.parseShort(acksStr);
if (acks != (short) -1) {
if (userConfiguredIdempotence) {
throw new ConfigException("Must set " + ACKS_CONFIG + " to all in order to use the idempotent " +
@@ -537,11 +562,16 @@ private static String parseAcks(String acksString) {
static Map appendSerializerToConfig(Map configs,
Serializer> keySerializer,
Serializer> valueSerializer) {
+ // validate serializer configuration, if the passed serializer instance is null, the user must explicitly set a valid serializer configuration value
Map newConfigs = new HashMap<>(configs);
if (keySerializer != null)
newConfigs.put(KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass());
+ else if (newConfigs.get(KEY_SERIALIZER_CLASS_CONFIG) == null)
+ throw new ConfigException(KEY_SERIALIZER_CLASS_CONFIG, null, "must be non-null.");
if (valueSerializer != null)
newConfigs.put(VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass());
+ else if (newConfigs.get(VALUE_SERIALIZER_CLASS_CONFIG) == null)
+ throw new ConfigException(VALUE_SERIALIZER_CLASS_CONFIG, null, "must be non-null.");
return newConfigs;
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java
index 8f89d6faa9ab1..48caf98d44a3c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java
@@ -34,7 +34,7 @@
*
* Implement {@link org.apache.kafka.common.ClusterResourceListener} to receive cluster metadata once it's available. Please see the class documentation for ClusterResourceListener for more information.
*/
-public interface ProducerInterceptor extends Configurable {
+public interface ProducerInterceptor extends Configurable, AutoCloseable {
/**
* This is called from {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord)} and
* {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord, Callback)} methods, before key and value
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/UniformStickyPartitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/UniformStickyPartitioner.java
index be11d0b662445..6e4fe420df259 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/UniformStickyPartitioner.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/UniformStickyPartitioner.java
@@ -23,6 +23,10 @@
/**
+ * NOTE this partitioner is deprecated and shouldn't be used. To use default partitioning logic
+ * remove partitioner.class configuration setting and set partitioner.ignore.keys=true.
+ * See KIP-794 for more info.
+ *
* The partitioning strategy:
*
* - If a partition is specified in the record, use it
@@ -33,6 +37,7 @@
*
* See KIP-480 for details about sticky partitioning.
*/
+@Deprecated
public class UniformStickyPartitioner implements Partitioner {
private final StickyPartitionCache stickyPartitionCache = new StickyPartitionCache();
@@ -59,6 +64,7 @@ public void close() {}
* If a batch completed for the current sticky partition, change the sticky partition.
* Alternately, if no sticky partition has been determined, set one.
*/
+ @SuppressWarnings("deprecation")
public void onNewBatch(String topic, Cluster cluster, int prevPartition) {
stickyPartitionCache.nextPartition(topic, cluster, prevPartition);
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
index 210911ada38cf..67cf485f81a55 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
@@ -279,7 +279,8 @@ public void deallocate(ByteBuffer buffer, int size) {
}
public void deallocate(ByteBuffer buffer) {
- deallocate(buffer, buffer.capacity());
+ if (buffer != null)
+ deallocate(buffer, buffer.capacity());
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java
new file mode 100644
index 0000000000000..a5805df56b78d
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java
@@ -0,0 +1,304 @@
+/*
+ * 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.producer.internals;
+
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+
+/**
+ * Built-in default partitioner. Note, that this is just a utility class that is used directly from
+ * RecordAccumulator, it does not implement the Partitioner interface.
+ *
+ * The class keeps track of various bookkeeping information required for adaptive sticky partitioning
+ * (described in detail in KIP-794). There is one partitioner object per topic.
+ */
+public class BuiltInPartitioner {
+ private final Logger log;
+ private final String topic;
+ private final int stickyBatchSize;
+
+ private volatile PartitionLoadStats partitionLoadStats = null;
+ private final AtomicReference stickyPartitionInfo = new AtomicReference<>();
+
+ // Visible and used for testing only.
+ static volatile public Supplier mockRandom = null;
+
+ /**
+ * BuiltInPartitioner constructor.
+ *
+ * @param topic The topic
+ * @param stickyBatchSize How much to produce to partition before switch
+ */
+ public BuiltInPartitioner(LogContext logContext, String topic, int stickyBatchSize) {
+ this.log = logContext.logger(BuiltInPartitioner.class);
+ this.topic = topic;
+ this.stickyBatchSize = stickyBatchSize;
+ }
+
+ /**
+ * Calculate the next partition for the topic based on the partition load stats.
+ */
+ private int nextPartition(Cluster cluster) {
+ int random = mockRandom != null ? mockRandom.get() : Utils.toPositive(ThreadLocalRandom.current().nextInt());
+
+ // Cache volatile variable in local variable.
+ PartitionLoadStats partitionLoadStats = this.partitionLoadStats;
+ int partition;
+
+ if (partitionLoadStats == null) {
+ // We don't have stats to do adaptive partitioning (or it's disabled), just switch to the next
+ // partition based on uniform distribution.
+ List availablePartitions = cluster.availablePartitionsForTopic(topic);
+ if (availablePartitions.size() > 0) {
+ partition = availablePartitions.get(random % availablePartitions.size()).partition();
+ } else {
+ // We don't have available partitions, just pick one among all partitions.
+ List partitions = cluster.partitionsForTopic(topic);
+ partition = random % partitions.size();
+ }
+ } else {
+ // Calculate next partition based on load distribution.
+ // Note that partitions without leader are excluded from the partitionLoadStats.
+ assert partitionLoadStats.length > 0;
+
+ int[] cumulativeFrequencyTable = partitionLoadStats.cumulativeFrequencyTable;
+ int weightedRandom = random % cumulativeFrequencyTable[partitionLoadStats.length - 1];
+
+ // By construction, the cumulative frequency table is sorted, so we can use binary
+ // search to find the desired index.
+ int searchResult = Arrays.binarySearch(cumulativeFrequencyTable, 0, partitionLoadStats.length, weightedRandom);
+
+ // binarySearch results the index of the found element, or -(insertion_point) - 1
+ // (where insertion_point is the index of the first element greater than the key).
+ // We need to get the index of the first value that is strictly greater, which
+ // would be the insertion point, except if we found the element that's equal to
+ // the searched value (in this case we need to get next). For example, if we have
+ // 4 5 8
+ // and we're looking for 3, then we'd get the insertion_point = 0, and the function
+ // would return -0 - 1 = -1, by adding 1 we'd get 0. If we're looking for 4, we'd
+ // get 0, and we need the next one, so adding 1 works here as well.
+ int partitionIndex = Math.abs(searchResult + 1);
+ assert partitionIndex < partitionLoadStats.length;
+ partition = partitionLoadStats.partitionIds[partitionIndex];
+ }
+
+ log.trace("Switching to partition {} in topic {}", partition, topic);
+ return partition;
+ }
+
+ /**
+ * Test-only function. When partition load stats are defined, return the end of range for the
+ * random number.
+ */
+ public int loadStatsRangeEnd() {
+ assert partitionLoadStats != null;
+ assert partitionLoadStats.length > 0;
+ return partitionLoadStats.cumulativeFrequencyTable[partitionLoadStats.length - 1];
+ }
+
+ /**
+ * Peek currently chosen sticky partition. This method works in conjunction with {@link #isPartitionChanged}
+ * and {@link #updatePartitionInfo}. The workflow is the following:
+ *
+ * 1. peekCurrentPartitionInfo is called to know which partition to lock.
+ * 2. Lock partition's batch queue.
+ * 3. isPartitionChanged under lock to make sure that nobody raced us.
+ * 4. Append data to buffer.
+ * 5. updatePartitionInfo to update produced bytes and maybe switch partition.
+ *
+ * It's important that steps 3-5 are under partition's batch queue lock.
+ *
+ * @param cluster The cluster information (needed if there is no current partition)
+ * @return sticky partition info object
+ */
+ StickyPartitionInfo peekCurrentPartitionInfo(Cluster cluster) {
+ StickyPartitionInfo partitionInfo = stickyPartitionInfo.get();
+ if (partitionInfo != null)
+ return partitionInfo;
+
+ // We're the first to create it.
+ partitionInfo = new StickyPartitionInfo(nextPartition(cluster));
+ if (stickyPartitionInfo.compareAndSet(null, partitionInfo))
+ return partitionInfo;
+
+ // Someone has raced us.
+ return stickyPartitionInfo.get();
+ }
+
+ /**
+ * Check if partition is changed by a concurrent thread. NOTE this function needs to be called under
+ * the partition's batch queue lock.
+ *
+ * @param partitionInfo The sticky partition info object returned by peekCurrentPartitionInfo
+ * @return true if sticky partition object is changed (race condition)
+ */
+ boolean isPartitionChanged(StickyPartitionInfo partitionInfo) {
+ // partitionInfo may be null if the caller didn't use built-in partitioner.
+ return partitionInfo != null && stickyPartitionInfo.get() != partitionInfo;
+ }
+
+ /**
+ * Update partition info with the number of bytes appended and maybe switch partition.
+ * NOTE this function needs to be called under the partition's batch queue lock.
+ *
+ * @param partitionInfo The sticky partition info object returned by peekCurrentPartitionInfo
+ * @param appendedBytes The number of bytes appended to this partition
+ * @param cluster The cluster information
+ */
+ void updatePartitionInfo(StickyPartitionInfo partitionInfo, int appendedBytes, Cluster cluster) {
+ // partitionInfo may be null if the caller didn't use built-in partitioner.
+ if (partitionInfo == null)
+ return;
+
+ assert partitionInfo == stickyPartitionInfo.get();
+ int producedBytes = partitionInfo.producedBytes.addAndGet(appendedBytes);
+ if (producedBytes >= stickyBatchSize) {
+ // We've produced enough to this partition, switch to next.
+ StickyPartitionInfo newPartitionInfo = new StickyPartitionInfo(nextPartition(cluster));
+ stickyPartitionInfo.set(newPartitionInfo);
+ }
+ }
+
+ /**
+ * Update partition load stats from the queue sizes of each partition
+ * NOTE: queueSizes are modified in place to avoid allocations
+ *
+ * @param queueSizes The queue sizes, partitions without leaders are excluded
+ * @param partitionIds The partition ids for the queues, partitions without leaders are excluded
+ * @param length The logical length of the arrays (could be less): we may eliminate some partitions
+ * based on latency, but to avoid reallocation of the arrays, we just decrement
+ * logical length
+ * Visible for testing
+ */
+ public void updatePartitionLoadStats(int[] queueSizes, int[] partitionIds, int length) {
+ if (queueSizes == null) {
+ log.trace("No load stats for topic {}, not using adaptive", topic);
+ partitionLoadStats = null;
+ return;
+ }
+ assert queueSizes.length == partitionIds.length;
+ assert length <= queueSizes.length;
+
+ // The queueSizes.length represents the number of all partitions in the topic and if we have
+ // less than 2 partitions, there is no need to do adaptive logic.
+ // If partitioner.availability.timeout.ms != 0, then partitions that experience high latencies
+ // (greater than partitioner.availability.timeout.ms) may be excluded, the length represents
+ // partitions that are not excluded. If some partitions were excluded, we'd still want to
+ // go through adaptive logic, even if we have one partition.
+ // See also RecordAccumulator#partitionReady where the queueSizes are built.
+ if (length < 1 || queueSizes.length < 2) {
+ log.trace("The number of partitions is too small: available={}, all={}, not using adaptive for topic {}",
+ length, queueSizes.length, topic);
+ partitionLoadStats = null;
+ return;
+ }
+
+ // We build cumulative frequency table from the queue sizes in place. At the beginning
+ // each entry contains queue size, then we invert it (so it represents the frequency)
+ // and convert to a running sum. Then a uniformly distributed random variable
+ // in the range [0..last) would map to a partition with weighted probability.
+ // Example: suppose we have 3 partitions with the corresponding queue sizes:
+ // 0 3 1
+ // Then we can invert them by subtracting the queue size from the max queue size + 1 = 4:
+ // 4 1 3
+ // Then we can convert it into a running sum (next value adds previous value):
+ // 4 5 8
+ // Now if we get a random number in the range [0..8) and find the first value that
+ // is strictly greater than the number (e.g. for 4 it would be 5), then the index of
+ // the value is the index of the partition we're looking for. In this example
+ // random numbers 0, 1, 2, 3 would map to partition[0], 4 would map to partition[1]
+ // and 5, 6, 7 would map to partition[2].
+
+ // Calculate max queue size + 1 and check if all sizes are the same.
+ int maxSizePlus1 = queueSizes[0];
+ boolean allEqual = true;
+ for (int i = 1; i < length; i++) {
+ if (queueSizes[i] != maxSizePlus1)
+ allEqual = false;
+ if (queueSizes[i] > maxSizePlus1)
+ maxSizePlus1 = queueSizes[i];
+ }
+ ++maxSizePlus1;
+
+ if (allEqual && length == queueSizes.length) {
+ // No need to have complex probability logic when all queue sizes are the same,
+ // and we didn't exclude partitions that experience high latencies (greater than
+ // partitioner.availability.timeout.ms).
+ log.trace("All queue lengths are the same, not using adaptive for topic {}", topic);
+ partitionLoadStats = null;
+ return;
+ }
+
+ // Invert and fold the queue size, so that they become separator values in the CFT.
+ queueSizes[0] = maxSizePlus1 - queueSizes[0];
+ for (int i = 1; i < length; i++) {
+ queueSizes[i] = maxSizePlus1 - queueSizes[i] + queueSizes[i - 1];
+ }
+ log.trace("Partition load stats for topic {}: CFT={}, IDs={}, length={}",
+ topic, queueSizes, partitionIds, length);
+ partitionLoadStats = new PartitionLoadStats(queueSizes, partitionIds, length);
+ }
+
+ /**
+ * Info for the current sticky partition.
+ */
+ public static class StickyPartitionInfo {
+ private final int index;
+ private final AtomicInteger producedBytes = new AtomicInteger();
+
+ StickyPartitionInfo(int index) {
+ this.index = index;
+ }
+
+ public int partition() {
+ return index;
+ }
+ }
+
+ /*
+ * Default hashing function to choose a partition from the serialized key bytes
+ */
+ public static int partitionForKey(final byte[] serializedKey, final int numPartitions) {
+ return Utils.toPositive(Utils.murmur2(serializedKey)) % numPartitions;
+ }
+
+ /**
+ * The partition load stats for each topic that are used for adaptive partition distribution.
+ */
+ private final static class PartitionLoadStats {
+ public final int[] cumulativeFrequencyTable;
+ public final int[] partitionIds;
+ public final int length;
+ public PartitionLoadStats(int[] cumulativeFrequencyTable, int[] partitionIds, int length) {
+ assert cumulativeFrequencyTable.length == partitionIds.length;
+ assert length <= cumulativeFrequencyTable.length;
+ this.cumulativeFrequencyTable = cumulativeFrequencyTable;
+ this.partitionIds = partitionIds;
+ this.length = length;
+ }
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java
index cf765d1eee6aa..716773626c8c1 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java
@@ -18,11 +18,13 @@
import org.apache.kafka.clients.producer.Partitioner;
import org.apache.kafka.common.Cluster;
-import org.apache.kafka.common.utils.Utils;
import java.util.Map;
/**
+ * NOTE this partitioner is deprecated and shouldn't be used. To use default partitioning logic
+ * remove partitioner.class configuration setting. See KIP-794 for more info.
+ *
* The default partitioning strategy:
*
* - If a partition is specified in the record, use it
@@ -31,6 +33,7 @@
*
* See KIP-480 for details about sticky partitioning.
*/
+@Deprecated
public class DefaultPartitioner implements Partitioner {
private final StickyPartitionCache stickyPartitionCache = new StickyPartitionCache();
@@ -67,8 +70,7 @@ public int partition(String topic, Object key, byte[] keyBytes, Object value, by
if (keyBytes == null) {
return stickyPartitionCache.partition(topic, cluster);
}
- // hash the keyBytes to choose a partition
- return Utils.toPositive(Utils.murmur2(keyBytes)) % numPartitions;
+ return BuiltInPartitioner.partitionForKey(keyBytes, numPartitions);
}
public void close() {}
@@ -77,6 +79,7 @@ public void close() {}
* If a batch completed for the current sticky partition, change the sticky partition.
* Alternately, if no sticky partition has been determined, set one.
*/
+ @SuppressWarnings("deprecation")
public void onNewBatch(String topic, Cluster cluster, int prevPartition) {
stickyPartitionCache.nextPartition(topic, cluster, prevPartition);
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java
index 3c6fe2691e2a8..7d942d572cfd5 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java
@@ -34,6 +34,7 @@ public class KafkaProducerMetrics implements AutoCloseable {
private static final String TXN_COMMIT = "txn-commit";
private static final String TXN_ABORT = "txn-abort";
private static final String TOTAL_TIME_SUFFIX = "-time-ns-total";
+ private static final String METADATA_WAIT = "metadata-wait";
private final Map tags;
private final Metrics metrics;
@@ -43,6 +44,7 @@ public class KafkaProducerMetrics implements AutoCloseable {
private final Sensor sendOffsetsSensor;
private final Sensor commitTxnSensor;
private final Sensor abortTxnSensor;
+ private final Sensor metadataWaitSensor;
public KafkaProducerMetrics(Metrics metrics) {
this.metrics = metrics;
@@ -71,6 +73,10 @@ public KafkaProducerMetrics(Metrics metrics) {
TXN_ABORT,
"Total time producer has spent in abortTransaction in nanoseconds."
);
+ metadataWaitSensor = newLatencySensor(
+ METADATA_WAIT,
+ "Total time producer has spent waiting on topic metadata in nanoseconds."
+ );
}
@Override
@@ -81,6 +87,7 @@ public void close() {
removeMetric(TXN_SEND_OFFSETS);
removeMetric(TXN_COMMIT);
removeMetric(TXN_ABORT);
+ removeMetric(METADATA_WAIT);
}
public void recordFlush(long duration) {
@@ -107,6 +114,10 @@ public void recordAbortTxn(long duration) {
abortTxnSensor.record(duration);
}
+ public void recordMetadataWait(long duration) {
+ metadataWaitSensor.record(duration);
+ }
+
private Sensor newLatencySensor(String name, String description) {
Sensor sensor = metrics.sensor(name + TOTAL_TIME_SUFFIX);
sensor.add(metricName(name, description), new CumulativeSum());
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index a47c9d38ad32d..a1f684ac95cd4 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -29,19 +29,18 @@
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
+
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.utils.ProducerIdAndEpoch;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.header.Header;
-import org.apache.kafka.common.metrics.Measurable;
-import org.apache.kafka.common.metrics.MetricConfig;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.record.AbstractRecords;
import org.apache.kafka.common.record.CompressionRatioEstimator;
@@ -63,8 +62,9 @@
* The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless
* this behavior is explicitly disabled.
*/
-public final class RecordAccumulator {
+public class RecordAccumulator {
+ private final LogContext logContext;
private final Logger log;
private volatile boolean closed;
private final AtomicInteger flushesInProgress;
@@ -74,14 +74,17 @@ public final class RecordAccumulator {
private final int lingerMs;
private final long retryBackoffMs;
private final int deliveryTimeoutMs;
+ private final long partitionAvailabilityTimeoutMs; // latency threshold for marking partition temporary unavailable
+ private final boolean enableAdaptivePartitioning;
private final BufferPool free;
private final Time time;
private final ApiVersions apiVersions;
- private final ConcurrentMap> batches;
+ private final ConcurrentMap topicInfoMap = new CopyOnWriteMap<>();
+ private final ConcurrentMap nodeStats = new CopyOnWriteMap<>();
private final IncompleteBatches incomplete;
// The following variables are only accessed by the sender thread, so we don't need to protect them.
private final Set muted;
- private int drainIndex;
+ private final Map nodesDrainIndex;
private final TransactionManager transactionManager;
private long nextBatchExpiryTimeMs = Long.MAX_VALUE; // the earliest time (absolute) a batch will expire.
@@ -96,11 +99,15 @@ public final class RecordAccumulator {
* latency for potentially better throughput due to more batching (and hence fewer, larger requests).
* @param retryBackoffMs An artificial delay time to retry the produce request upon receiving an error. This avoids
* exhausting all retries in a short period of time.
+ * @param deliveryTimeoutMs An upper bound on the time to report success or failure on record delivery
+ * @param partitionerConfig Partitioner config
* @param metrics The metrics
+ * @param metricGrpName The metric group name
* @param time The time instance to use
* @param apiVersions Request API versions for current connected brokers
* @param transactionManager The shared transaction state object which tracks producer IDs, epochs, and sequence
* numbers per partition.
+ * @param bufferPool The buffer pool
*/
public RecordAccumulator(LogContext logContext,
int batchSize,
@@ -108,14 +115,15 @@ public RecordAccumulator(LogContext logContext,
int lingerMs,
long retryBackoffMs,
int deliveryTimeoutMs,
+ PartitionerConfig partitionerConfig,
Metrics metrics,
String metricGrpName,
Time time,
ApiVersions apiVersions,
TransactionManager transactionManager,
BufferPool bufferPool) {
+ this.logContext = logContext;
this.log = logContext.logger(RecordAccumulator.class);
- this.drainIndex = 0;
this.closed = false;
this.flushesInProgress = new AtomicInteger(0);
this.appendsInProgress = new AtomicInteger(0);
@@ -124,40 +132,85 @@ public RecordAccumulator(LogContext logContext,
this.lingerMs = lingerMs;
this.retryBackoffMs = retryBackoffMs;
this.deliveryTimeoutMs = deliveryTimeoutMs;
- this.batches = new CopyOnWriteMap<>();
+ this.enableAdaptivePartitioning = partitionerConfig.enableAdaptivePartitioning;
+ this.partitionAvailabilityTimeoutMs = partitionerConfig.partitionAvailabilityTimeoutMs;
this.free = bufferPool;
this.incomplete = new IncompleteBatches();
this.muted = new HashSet<>();
this.time = time;
this.apiVersions = apiVersions;
+ nodesDrainIndex = new HashMap<>();
this.transactionManager = transactionManager;
registerMetrics(metrics, metricGrpName);
}
- private void registerMetrics(Metrics metrics, String metricGrpName) {
- MetricName metricName = metrics.metricName("waiting-threads", metricGrpName, "The number of user threads blocked waiting for buffer memory to enqueue their records");
- Measurable waitingThreads = new Measurable() {
- public double measure(MetricConfig config, long now) {
- return free.queued();
- }
- };
- metrics.addMetric(metricName, waitingThreads);
+ /**
+ * Create a new record accumulator with default partitioner config
+ *
+ * @param logContext The log context used for logging
+ * @param batchSize The size to use when allocating {@link MemoryRecords} instances
+ * @param compression The compression codec for the records
+ * @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for
+ * sending. This allows time for more records to arrive. Setting a non-zero lingerMs will trade off some
+ * latency for potentially better throughput due to more batching (and hence fewer, larger requests).
+ * @param retryBackoffMs An artificial delay time to retry the produce request upon receiving an error. This avoids
+ * exhausting all retries in a short period of time.
+ * @param deliveryTimeoutMs An upper bound on the time to report success or failure on record delivery
+ * @param metrics The metrics
+ * @param metricGrpName The metric group name
+ * @param time The time instance to use
+ * @param apiVersions Request API versions for current connected brokers
+ * @param transactionManager The shared transaction state object which tracks producer IDs, epochs, and sequence
+ * numbers per partition.
+ * @param bufferPool The buffer pool
+ */
+ public RecordAccumulator(LogContext logContext,
+ int batchSize,
+ CompressionType compression,
+ int lingerMs,
+ long retryBackoffMs,
+ int deliveryTimeoutMs,
+ Metrics metrics,
+ String metricGrpName,
+ Time time,
+ ApiVersions apiVersions,
+ TransactionManager transactionManager,
+ BufferPool bufferPool) {
+ this(logContext,
+ batchSize,
+ compression,
+ lingerMs,
+ retryBackoffMs,
+ deliveryTimeoutMs,
+ new PartitionerConfig(),
+ metrics,
+ metricGrpName,
+ time,
+ apiVersions,
+ transactionManager,
+ bufferPool);
+ }
- metricName = metrics.metricName("buffer-total-bytes", metricGrpName, "The maximum amount of buffer memory the client can use (whether or not it is currently used).");
- Measurable totalBytes = new Measurable() {
- public double measure(MetricConfig config, long now) {
- return free.totalMemory();
- }
- };
- metrics.addMetric(metricName, totalBytes);
+ private void registerMetrics(Metrics metrics, String metricGrpName) {
+ metrics.addMetric(
+ metrics.metricName("waiting-threads", metricGrpName,
+ "The number of user threads blocked waiting for buffer memory to enqueue their records"),
+ (config, now) -> free.queued());
+
+ metrics.addMetric(
+ metrics.metricName("buffer-total-bytes", metricGrpName,
+ "The maximum amount of buffer memory the client can use (whether or not it is currently used)."),
+ (config, now) -> free.totalMemory());
+
+ metrics.addMetric(
+ metrics.metricName("buffer-available-bytes", metricGrpName,
+ "The total amount of buffer memory that is not being used (either unallocated or in the free list)."),
+ (config, now) -> free.availableMemory());
+ }
- metricName = metrics.metricName("buffer-available-bytes", metricGrpName, "The total amount of buffer memory that is not being used (either unallocated or in the free list).");
- Measurable availableBytes = new Measurable() {
- public double measure(MetricConfig config, long now) {
- return free.availableMemory();
- }
- };
- metrics.addMetric(metricName, availableBytes);
+ private void setPartition(AppendCallbacks callbacks, int partition) {
+ if (callbacks != null)
+ callbacks.setPartition(partition);
}
/**
@@ -166,85 +219,156 @@ public double measure(MetricConfig config, long now) {
* The append result will contain the future metadata, and flag for whether the appended batch is full or a new batch is created
*
*
- * @param tp The topic/partition to which this record is being sent
+ * @param topic The topic to which this record is being sent
+ * @param partition The partition to which this record is being sent or RecordMetadata.UNKNOWN_PARTITION
+ * if any partition could be used
* @param timestamp The timestamp of the record
* @param key The key for the record
* @param value The value for the record
* @param headers the Headers for the record
- * @param callback The user-supplied callback to execute when the request is complete
+ * @param callbacks The callbacks to execute
* @param maxTimeToBlock The maximum time in milliseconds to block for buffer memory to be available
* @param abortOnNewBatch A boolean that indicates returning before a new batch is created and
* running the partitioner's onNewBatch method before trying to append again
* @param nowMs The current time, in milliseconds
+ * @param cluster The cluster metadata
*/
- public RecordAppendResult append(TopicPartition tp,
+ public RecordAppendResult append(String topic,
+ int partition,
long timestamp,
byte[] key,
byte[] value,
Header[] headers,
- Callback callback,
+ AppendCallbacks callbacks,
long maxTimeToBlock,
boolean abortOnNewBatch,
- long nowMs) throws InterruptedException {
+ long nowMs,
+ Cluster cluster) throws InterruptedException {
+ TopicInfo topicInfo = topicInfoMap.computeIfAbsent(topic, k -> new TopicInfo(logContext, k, batchSize));
+
// We keep track of the number of appending thread to make sure we do not miss batches in
// abortIncompleteBatches().
appendsInProgress.incrementAndGet();
ByteBuffer buffer = null;
if (headers == null) headers = Record.EMPTY_HEADERS;
try {
- // check if we have an in-progress batch
- Deque dq = getOrCreateDeque(tp);
- synchronized (dq) {
- if (closed)
- throw new KafkaException("Producer closed while send in progress");
- RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callback, dq, nowMs);
- if (appendResult != null)
- return appendResult;
- }
+ // Loop to retry in case we encounter partitioner's race conditions.
+ while (true) {
+ // If the message doesn't have any partition affinity, so we pick a partition based on the broker
+ // availability and performance. Note, that here we peek current partition before we hold the
+ // deque lock, so we'll need to make sure that it's not changed while we were waiting for the
+ // deque lock.
+ final BuiltInPartitioner.StickyPartitionInfo partitionInfo;
+ final int effectivePartition;
+ if (partition == RecordMetadata.UNKNOWN_PARTITION) {
+ partitionInfo = topicInfo.builtInPartitioner.peekCurrentPartitionInfo(cluster);
+ effectivePartition = partitionInfo.partition();
+ } else {
+ partitionInfo = null;
+ effectivePartition = partition;
+ }
- // we don't have an in-progress record batch try to allocate a new batch
- if (abortOnNewBatch) {
- // Return a result that will cause another call to append.
- return new RecordAppendResult(null, false, false, true);
- }
+ // Now that we know the effective partition, let the caller know.
+ setPartition(callbacks, effectivePartition);
+
+ // check if we have an in-progress batch
+ Deque dq = topicInfo.batches.computeIfAbsent(effectivePartition, k -> new ArrayDeque<>());
+ synchronized (dq) {
+ // After taking the lock, validate that the partition hasn't changed and retry.
+ if (topicInfo.builtInPartitioner.isPartitionChanged(partitionInfo)) {
+ log.trace("Partition {} for topic {} switched by a concurrent append, retrying",
+ partitionInfo.partition(), topic);
+ continue;
+ }
+ RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callbacks, dq, nowMs);
+ if (appendResult != null) {
+ topicInfo.builtInPartitioner.updatePartitionInfo(partitionInfo, appendResult.appendedBytes, cluster);
+ return appendResult;
+ }
+ }
- byte maxUsableMagic = apiVersions.maxUsableProduceMagic();
- int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression, key, value, headers));
- log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, tp.topic(), tp.partition(), maxTimeToBlock);
- buffer = free.allocate(size, maxTimeToBlock);
+ // we don't have an in-progress record batch try to allocate a new batch
+ if (abortOnNewBatch) {
+ // Return a result that will cause another call to append.
+ return new RecordAppendResult(null, false, false, true, 0);
+ }
- // Update the current time in case the buffer allocation blocked above.
- nowMs = time.milliseconds();
- synchronized (dq) {
- // Need to check if producer is closed again after grabbing the dequeue lock.
- if (closed)
- throw new KafkaException("Producer closed while send in progress");
+ if (buffer == null) {
+ byte maxUsableMagic = apiVersions.maxUsableProduceMagic();
+ int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression, key, value, headers));
+ log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, topic, partition, maxTimeToBlock);
+ // This call may block if we exhausted buffer space.
+ buffer = free.allocate(size, maxTimeToBlock);
+ // Update the current time in case the buffer allocation blocked above.
+ // NOTE: getting time may be expensive, so calling it under a lock
+ // should be avoided.
+ nowMs = time.milliseconds();
+ }
- RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callback, dq, nowMs);
- if (appendResult != null) {
- // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often...
+ synchronized (dq) {
+ // After taking the lock, validate that the partition hasn't changed and retry.
+ if (topicInfo.builtInPartitioner.isPartitionChanged(partitionInfo)) {
+ log.trace("Partition {} for topic {} switched by a concurrent append, retrying",
+ partitionInfo.partition(), topic);
+ continue;
+ }
+ RecordAppendResult appendResult = appendNewBatch(topic, effectivePartition, dq, timestamp, key, value, headers, callbacks, buffer, nowMs);
+ // Set buffer to null, so that deallocate doesn't return it back to free pool, since it's used in the batch.
+ if (appendResult.newBatchCreated)
+ buffer = null;
+ topicInfo.builtInPartitioner.updatePartitionInfo(partitionInfo, appendResult.appendedBytes, cluster);
return appendResult;
}
-
- MemoryRecordsBuilder recordsBuilder = recordsBuilder(buffer, maxUsableMagic);
- ProducerBatch batch = new ProducerBatch(tp, recordsBuilder, nowMs);
- FutureRecordMetadata future = Objects.requireNonNull(batch.tryAppend(timestamp, key, value, headers,
- callback, nowMs));
-
- dq.addLast(batch);
- incomplete.add(batch);
-
- // Don't deallocate this buffer in the finally block as it's being used in the record batch
- buffer = null;
- return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true, false);
}
} finally {
- if (buffer != null)
- free.deallocate(buffer);
+ free.deallocate(buffer);
appendsInProgress.decrementAndGet();
}
}
+ /**
+ * Append a new batch to the queue
+ *
+ * @param topic The topic
+ * @param partition The partition (cannot be RecordMetadata.UNKNOWN_PARTITION)
+ * @param dq The queue
+ * @param timestamp The timestamp of the record
+ * @param key The key for the record
+ * @param value The value for the record
+ * @param headers the Headers for the record
+ * @param callbacks The callbacks to execute
+ * @param buffer The buffer for the new batch
+ * @param nowMs The current time, in milliseconds
+ */
+ private RecordAppendResult appendNewBatch(String topic,
+ int partition,
+ Deque dq,
+ long timestamp,
+ byte[] key,
+ byte[] value,
+ Header[] headers,
+ AppendCallbacks callbacks,
+ ByteBuffer buffer,
+ long nowMs) {
+ assert partition != RecordMetadata.UNKNOWN_PARTITION;
+
+ RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callbacks, dq, nowMs);
+ if (appendResult != null) {
+ // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often...
+ return appendResult;
+ }
+
+ MemoryRecordsBuilder recordsBuilder = recordsBuilder(buffer, apiVersions.maxUsableProduceMagic());
+ ProducerBatch batch = new ProducerBatch(new TopicPartition(topic, partition), recordsBuilder, nowMs);
+ FutureRecordMetadata future = Objects.requireNonNull(batch.tryAppend(timestamp, key, value, headers,
+ callbacks, nowMs));
+
+ dq.addLast(batch);
+ incomplete.add(batch);
+
+ return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true, false, batch.estimatedSizeInBytes());
+ }
+
private MemoryRecordsBuilder recordsBuilder(ByteBuffer buffer, byte maxUsableMagic) {
if (transactionManager != null && maxUsableMagic < RecordBatch.MAGIC_VALUE_V2) {
throw new UnsupportedVersionException("Attempting to use idempotence with a broker which does not " +
@@ -263,13 +387,18 @@ private MemoryRecordsBuilder recordsBuilder(ByteBuffer buffer, byte maxUsableMag
*/
private RecordAppendResult tryAppend(long timestamp, byte[] key, byte[] value, Header[] headers,
Callback callback, Deque deque, long nowMs) {
+ if (closed)
+ throw new KafkaException("Producer closed while send in progress");
ProducerBatch last = deque.peekLast();
if (last != null) {
+ int initialBytes = last.estimatedSizeInBytes();
FutureRecordMetadata future = last.tryAppend(timestamp, key, value, headers, callback, nowMs);
- if (future == null)
+ if (future == null) {
last.closeForRecordAppends();
- else
- return new RecordAppendResult(future, deque.size() > 1 || last.isFull(), false, false);
+ } else {
+ int appendedBytes = last.estimatedSizeInBytes() - initialBytes;
+ return new RecordAppendResult(future, deque.size() > 1 || last.isFull(), false, false, appendedBytes);
+ }
}
return null;
}
@@ -298,19 +427,20 @@ public void maybeUpdateNextBatchExpiryTime(ProducerBatch batch) {
*/
public List expiredBatches(long now) {
List expiredBatches = new ArrayList<>();
- for (Map.Entry> entry : this.batches.entrySet()) {
- // expire the batches in the order of sending
- Deque deque = entry.getValue();
- synchronized (deque) {
- while (!deque.isEmpty()) {
- ProducerBatch batch = deque.getFirst();
- if (batch.hasReachedDeliveryTimeout(deliveryTimeoutMs, now)) {
- deque.poll();
- batch.abortRecordAppends();
- expiredBatches.add(batch);
- } else {
- maybeUpdateNextBatchExpiryTime(batch);
- break;
+ for (TopicInfo topicInfo : topicInfoMap.values()) {
+ for (Deque deque : topicInfo.batches.values()) {
+ // expire the batches in the order of sending
+ synchronized (deque) {
+ while (!deque.isEmpty()) {
+ ProducerBatch batch = deque.getFirst();
+ if (batch.hasReachedDeliveryTimeout(deliveryTimeoutMs, now)) {
+ deque.poll();
+ batch.abortRecordAppends();
+ expiredBatches.add(batch);
+ } else {
+ maybeUpdateNextBatchExpiryTime(batch);
+ break;
+ }
}
}
}
@@ -378,12 +508,12 @@ public int splitAndReenqueue(ProducerBatch bigBatch) {
// producer id. We will not attempt to reorder messages if the producer id has changed, we will throw an
// IllegalStateException instead.
private void insertInSequenceOrder(Deque deque, ProducerBatch batch) {
- // When we are requeing and have enabled idempotence, the reenqueued batch must always have a sequence.
+ // When we are re-enqueueing and have enabled idempotence, the re-enqueued batch must always have a sequence.
if (batch.baseSequence() == RecordBatch.NO_SEQUENCE)
throw new IllegalStateException("Trying to re-enqueue a batch which doesn't have a sequence even " +
"though idempotency is enabled.");
- if (transactionManager.nextBatchBySequence(batch.topicPartition) == null)
+ if (!transactionManager.hasInflightBatches(batch.topicPartition))
throw new IllegalStateException("We are re-enqueueing a batch which is not tracked as part of the in flight " +
"requests. batch.topicPartition: " + batch.topicPartition + "; batch.baseSequence: " + batch.baseSequence());
@@ -420,38 +550,94 @@ private void insertInSequenceOrder(Deque deque, ProducerBatch bat
}
/**
- * Get a list of nodes whose partitions are ready to be sent, and the earliest time at which any non-sendable
- * partition will be ready; Also return the flag for whether there are any unknown leaders for the accumulated
- * partition batches.
- *
- * A destination node is ready to send data if:
- *
- * - There is at least one partition that is not backing off its send
- *
- and those partitions are not muted (to prevent reordering if
- * {@value org.apache.kafka.clients.producer.ProducerConfig#MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION}
- * is set to one)
- * - and any of the following are true
- *
- * - The record set is full
- * - The record set has sat in the accumulator for at least lingerMs milliseconds
- * - The accumulator is out of memory and threads are blocking waiting for data (in this case all partitions
- * are immediately considered ready).
- * - The accumulator has been closed
- *
- *
+ * Add the leader to the ready nodes if the batch is ready
+ *
+ * @param nowMs The current time
+ * @param exhausted 'true' is the buffer pool is exhausted
+ * @param part The partition
+ * @param leader The leader for the partition
+ * @param waitedTimeMs How long batch waited
+ * @param backingOff Is backing off
+ * @param full Is batch full
+ * @param nextReadyCheckDelayMs The delay for next check
+ * @param readyNodes The set of ready nodes (to be filled in)
+ * @return The delay for next check
*/
- public ReadyCheckResult ready(Cluster cluster, long nowMs) {
- Set readyNodes = new HashSet<>();
- long nextReadyCheckDelayMs = Long.MAX_VALUE;
- Set unknownLeaderTopics = new HashSet<>();
+ private long batchReady(long nowMs, boolean exhausted, TopicPartition part, Node leader,
+ long waitedTimeMs, boolean backingOff, boolean full,
+ long nextReadyCheckDelayMs, Set readyNodes) {
+ if (!readyNodes.contains(leader) && !isMuted(part)) {
+ long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
+ boolean expired = waitedTimeMs >= timeToWaitMs;
+ boolean transactionCompleting = transactionManager != null && transactionManager.isCompleting();
+ boolean sendable = full
+ || expired
+ || exhausted
+ || closed
+ || flushInProgress()
+ || transactionCompleting;
+ if (sendable && !backingOff) {
+ readyNodes.add(leader);
+ } else {
+ long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
+ // Note that this results in a conservative estimate since an un-sendable partition may have
+ // a leader that will later be found to have sendable data. However, this is good enough
+ // since we'll just wake up and then sleep again for the remaining time.
+ nextReadyCheckDelayMs = Math.min(timeLeftMs, nextReadyCheckDelayMs);
+ }
+ }
+ return nextReadyCheckDelayMs;
+ }
+ /**
+ * Iterate over partitions to see which one have batches ready and collect leaders of those partitions
+ * into the set of ready nodes. If partition has no leader, add the topic to the set of topics with
+ * no leader. This function also calculates stats for adaptive partitioning.
+ *
+ * @param cluster The cluster metadata
+ * @param nowMs The current time
+ * @param topic The topic
+ * @param topicInfo The topic info
+ * @param nextReadyCheckDelayMs The delay for next check
+ * @param readyNodes The set of ready nodes (to be filled in)
+ * @param unknownLeaderTopics The set of topics with no leader (to be filled in)
+ * @return The delay for next check
+ */
+ private long partitionReady(Cluster cluster, long nowMs, String topic,
+ TopicInfo topicInfo,
+ long nextReadyCheckDelayMs, Set readyNodes, Set unknownLeaderTopics) {
+ ConcurrentMap> batches = topicInfo.batches;
+ // Collect the queue sizes for available partitions to be used in adaptive partitioning.
+ int[] queueSizes = null;
+ int[] partitionIds = null;
+ if (enableAdaptivePartitioning && batches.size() >= cluster.partitionsForTopic(topic).size()) {
+ // We don't do adaptive partitioning until we scheduled at least a batch for all
+ // partitions (i.e. we have the corresponding entries in the batches map), we just
+ // do uniform. The reason is that we build queue sizes from the batches map,
+ // and if an entry is missing in the batches map, then adaptive partitioning logic
+ // won't know about it and won't switch to it.
+ queueSizes = new int[batches.size()];
+ partitionIds = new int[queueSizes.length];
+ }
+
+ int queueSizesIndex = -1;
boolean exhausted = this.free.queued() > 0;
- for (Map.Entry> entry : this.batches.entrySet()) {
+ for (Map.Entry> entry : batches.entrySet()) {
+ TopicPartition part = new TopicPartition(topic, entry.getKey());
+ // Advance queueSizesIndex so that we properly index available
+ // partitions. Do it here so that it's done for all code paths.
+ Node leader = cluster.leaderFor(part);
+ if (leader != null && queueSizes != null) {
+ ++queueSizesIndex;
+ assert queueSizesIndex < queueSizes.length;
+ partitionIds[queueSizesIndex] = part.partition();
+ }
+
Deque deque = entry.getValue();
- final ProducerBatch batch;
final long waitedTimeMs;
final boolean backingOff;
+ final int dequeSize;
final boolean full;
// This loop is especially hot with large partition counts.
@@ -463,43 +649,81 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) {
synchronized (deque) {
// Deques are often empty in this path, esp with large partition counts,
// so we exit early if we can.
- batch = deque.peekFirst();
+ ProducerBatch batch = deque.peekFirst();
if (batch == null) {
continue;
}
waitedTimeMs = batch.waitedTimeMs(nowMs);
backingOff = batch.attempts() > 0 && waitedTimeMs < retryBackoffMs;
- full = deque.size() > 1 || batch.isFull();
+ dequeSize = deque.size();
+ full = dequeSize > 1 || batch.isFull();
}
- TopicPartition part = entry.getKey();
- Node leader = cluster.leaderFor(part);
if (leader == null) {
// This is a partition for which leader is not known, but messages are available to send.
// Note that entries are currently not removed from batches when deque is empty.
unknownLeaderTopics.add(part.topic());
- } else if (!readyNodes.contains(leader) && !isMuted(part)) {
- long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
- boolean expired = waitedTimeMs >= timeToWaitMs;
- boolean transactionCompleting = transactionManager != null && transactionManager.isCompleting();
- boolean sendable = full
- || expired
- || exhausted
- || closed
- || flushInProgress()
- || transactionCompleting;
- if (sendable && !backingOff) {
- readyNodes.add(leader);
- } else {
- long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
- // Note that this results in a conservative estimate since an un-sendable partition may have
- // a leader that will later be found to have sendable data. However, this is good enough
- // since we'll just wake up and then sleep again for the remaining time.
- nextReadyCheckDelayMs = Math.min(timeLeftMs, nextReadyCheckDelayMs);
+ } else {
+ if (queueSizes != null)
+ queueSizes[queueSizesIndex] = dequeSize;
+ if (partitionAvailabilityTimeoutMs > 0) {
+ // Check if we want to exclude the partition from the list of available partitions
+ // if the broker hasn't responded for some time.
+ NodeLatencyStats nodeLatencyStats = nodeStats.get(leader.id());
+ if (nodeLatencyStats != null) {
+ // NOTE: there is no synchronization between reading metrics,
+ // so we read ready time first to avoid accidentally marking partition
+ // unavailable if we read while the metrics are being updated.
+ long readyTimeMs = nodeLatencyStats.readyTimeMs;
+ if (readyTimeMs - nodeLatencyStats.drainTimeMs > partitionAvailabilityTimeoutMs)
+ --queueSizesIndex;
+ }
}
+
+ nextReadyCheckDelayMs = batchReady(nowMs, exhausted, part, leader, waitedTimeMs, backingOff,
+ full, nextReadyCheckDelayMs, readyNodes);
}
}
+
+ // We've collected the queue sizes for partitions of this topic, now we can calculate
+ // load stats. NOTE: the stats are calculated in place, modifying the
+ // queueSizes array.
+ topicInfo.builtInPartitioner.updatePartitionLoadStats(queueSizes, partitionIds, queueSizesIndex + 1);
+ return nextReadyCheckDelayMs;
+ }
+
+ /**
+ * Get a list of nodes whose partitions are ready to be sent, and the earliest time at which any non-sendable
+ * partition will be ready; Also return the flag for whether there are any unknown leaders for the accumulated
+ * partition batches.
+ *
+ * A destination node is ready to send data if:
+ *
+ * - There is at least one partition that is not backing off its send
+ *
- and those partitions are not muted (to prevent reordering if
+ * {@value org.apache.kafka.clients.producer.ProducerConfig#MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION}
+ * is set to one)
+ * - and any of the following are true
+ *
+ * - The record set is full
+ * - The record set has sat in the accumulator for at least lingerMs milliseconds
+ * - The accumulator is out of memory and threads are blocking waiting for data (in this case all partitions
+ * are immediately considered ready).
+ * - The accumulator has been closed
+ *
+ *
+ */
+ public ReadyCheckResult ready(Cluster cluster, long nowMs) {
+ Set readyNodes = new HashSet<>();
+ long nextReadyCheckDelayMs = Long.MAX_VALUE;
+ Set unknownLeaderTopics = new HashSet<>();
+ // Go topic by topic so that we can get queue sizes for partitions in a topic and calculate
+ // cumulative frequency table (used in partitioner).
+ for (Map.Entry topicInfoEntry : this.topicInfoMap.entrySet()) {
+ final String topic = topicInfoEntry.getKey();
+ nextReadyCheckDelayMs = partitionReady(cluster, nowMs, topic, topicInfoEntry.getValue(), nextReadyCheckDelayMs, readyNodes, unknownLeaderTopics);
+ }
return new ReadyCheckResult(readyNodes, nextReadyCheckDelayMs, unknownLeaderTopics);
}
@@ -507,11 +731,12 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) {
* Check whether there are any batches which haven't been drained
*/
public boolean hasUndrained() {
- for (Map.Entry> entry : this.batches.entrySet()) {
- Deque deque = entry.getValue();
- synchronized (deque) {
- if (!deque.isEmpty())
- return true;
+ for (TopicInfo topicInfo : topicInfoMap.values()) {
+ for (Deque deque : topicInfo.batches.values()) {
+ synchronized (deque) {
+ if (!deque.isEmpty())
+ return true;
+ }
}
}
return false;
@@ -559,13 +784,14 @@ private List drainBatchesForOneNode(Cluster cluster, Node node, i
int size = 0;
List parts = cluster.partitionsForNode(node.id());
List ready = new ArrayList<>();
- /* to make starvation less likely this loop doesn't start at 0 */
+ /* to make starvation less likely each node has it's own drainIndex */
+ int drainIndex = getDrainIndex(node.idString());
int start = drainIndex = drainIndex % parts.size();
do {
PartitionInfo part = parts.get(drainIndex);
TopicPartition tp = new TopicPartition(part.topic(), part.partition());
- this.drainIndex = (this.drainIndex + 1) % parts.size();
-
+ updateDrainIndex(node.idString(), drainIndex);
+ drainIndex = (drainIndex + 1) % parts.size();
// Only proceed if the partition has no in-flight batches.
if (isMuted(tp))
continue;
@@ -638,6 +864,14 @@ private List drainBatchesForOneNode(Cluster cluster, Node node, i
return ready;
}
+ private int getDrainIndex(String idString) {
+ return nodesDrainIndex.computeIfAbsent(idString, s -> 0);
+ }
+
+ private void updateDrainIndex(String idString, int drainIndex) {
+ nodesDrainIndex.put(idString, drainIndex);
+ }
+
/**
* Drain all the data for the given nodes and collate them into a list of batches that will fit within the specified
* size on a per-node basis. This method attempts to avoid choosing the same topic-node over and over.
@@ -660,6 +894,36 @@ public Map> drain(Cluster cluster, Set nodes,
return batches;
}
+ public void updateNodeLatencyStats(Integer nodeId, long nowMs, boolean canDrain) {
+ // Don't bother with updating stats if the feature is turned off.
+ if (partitionAvailabilityTimeoutMs <= 0)
+ return;
+
+ // When the sender gets a node (returned by the ready() function) that has data to send
+ // but the node is not ready (and so we cannot drain the data), we only update the
+ // ready time, then the difference would reflect for how long a node wasn't ready
+ // to send the data. Then we can temporarily remove partitions that are handled by the
+ // node from the list of available partitions so that the partitioner wouldn't pick
+ // this partition.
+ // NOTE: there is no synchronization for metric updates, so drainTimeMs is updated
+ // first to avoid accidentally marking a partition unavailable if the reader gets
+ // values between updates.
+ NodeLatencyStats nodeLatencyStats = nodeStats.computeIfAbsent(nodeId, id -> new NodeLatencyStats(nowMs));
+ if (canDrain)
+ nodeLatencyStats.drainTimeMs = nowMs;
+ nodeLatencyStats.readyTimeMs = nowMs;
+ }
+
+ /* Visible for testing */
+ public NodeLatencyStats getNodeLatencyStats(Integer nodeId) {
+ return nodeStats.get(nodeId);
+ }
+
+ /* Visible for testing */
+ public BuiltInPartitioner getBuiltInPartitioner(String topic) {
+ return topicInfoMap.get(topic).builtInPartitioner;
+ }
+
/**
* The earliest absolute time a batch will expire (in milliseconds)
*/
@@ -667,23 +931,20 @@ public long nextExpiryTimeMs() {
return this.nextBatchExpiryTimeMs;
}
- private Deque getDeque(TopicPartition tp) {
- return batches.get(tp);
+ /* Visible for testing */
+ public Deque getDeque(TopicPartition tp) {
+ TopicInfo topicInfo = topicInfoMap.get(tp.topic());
+ if (topicInfo == null)
+ return null;
+ return topicInfo.batches.get(tp.partition());
}
/**
* Get the deque for the given topic-partition, creating it if necessary.
*/
private Deque getOrCreateDeque(TopicPartition tp) {
- Deque d = this.batches.get(tp);
- if (d != null)
- return d;
- d = new ArrayDeque<>();
- Deque previous = this.batches.putIfAbsent(tp, d);
- if (previous == null)
- return d;
- else
- return previous;
+ TopicInfo topicInfo = topicInfoMap.computeIfAbsent(tp.topic(), k -> new TopicInfo(logContext, k, batchSize));
+ return topicInfo.batches.computeIfAbsent(tp.partition(), k -> new ArrayDeque<>());
}
/**
@@ -713,11 +974,6 @@ boolean flushInProgress() {
return flushesInProgress.get() > 0;
}
- /* Visible for testing */
- Map> batches() {
- return Collections.unmodifiableMap(batches);
- }
-
/**
* Initiate the flushing of data from the accumulator...this makes all requests immediately ready
*/
@@ -771,7 +1027,7 @@ public void abortIncompleteBatches() {
// flag set. We need to do the last abort after no thread was appending in case there was a new
// batch appended by the last appending thread.
abortBatches();
- this.batches.clear();
+ this.topicInfoMap.clear();
}
/**
@@ -833,6 +1089,32 @@ public void close() {
this.free.close();
}
+ /**
+ * Partitioner config for built-in partitioner
+ */
+ public static final class PartitionerConfig {
+ private final boolean enableAdaptivePartitioning;
+ private final long partitionAvailabilityTimeoutMs;
+
+ /**
+ * Partitioner config
+ *
+ * @param enableAdaptivePartitioning If it's true, partition switching adapts to broker load, otherwise partition
+ * switching is random.
+ * @param partitionAvailabilityTimeoutMs If a broker cannot process produce requests from a partition
+ * for the specified time, the partition is treated by the partitioner as not available.
+ * If the timeout is 0, this logic is disabled.
+ */
+ public PartitionerConfig(boolean enableAdaptivePartitioning, long partitionAvailabilityTimeoutMs) {
+ this.enableAdaptivePartitioning = enableAdaptivePartitioning;
+ this.partitionAvailabilityTimeoutMs = partitionAvailabilityTimeoutMs;
+ }
+
+ public PartitionerConfig() {
+ this(false, 0);
+ }
+ }
+
/*
* Metadata about a record just appended to the record accumulator
*/
@@ -841,15 +1123,32 @@ public final static class RecordAppendResult {
public final boolean batchIsFull;
public final boolean newBatchCreated;
public final boolean abortForNewBatch;
+ public final int appendedBytes;
- public RecordAppendResult(FutureRecordMetadata future, boolean batchIsFull, boolean newBatchCreated, boolean abortForNewBatch) {
+ public RecordAppendResult(FutureRecordMetadata future,
+ boolean batchIsFull,
+ boolean newBatchCreated,
+ boolean abortForNewBatch,
+ int appendedBytes) {
this.future = future;
this.batchIsFull = batchIsFull;
this.newBatchCreated = newBatchCreated;
this.abortForNewBatch = abortForNewBatch;
+ this.appendedBytes = appendedBytes;
}
}
+ /*
+ * The callbacks passed into append
+ */
+ public interface AppendCallbacks extends Callback {
+ /**
+ * Called to set partition (when append is called, partition may not be calculated yet).
+ * @param partition The partition
+ */
+ void setPartition(int partition);
+ }
+
/*
* The set of nodes that have at least one complete record batch in the accumulator
*/
@@ -864,4 +1163,30 @@ public ReadyCheckResult(Set readyNodes, long nextReadyCheckDelayMs, Set> batches = new CopyOnWriteMap<>();
+ public final BuiltInPartitioner builtInPartitioner;
+
+ public TopicInfo(LogContext logContext, String topic, int stickyBatchSize) {
+ builtInPartitioner = new BuiltInPartitioner(logContext, topic, stickyBatchSize);
+ }
+ }
+
+ /**
+ * Node latency stats for each node that are used for adaptive partition distribution
+ * Visible for testing
+ */
+ public final static class NodeLatencyStats {
+ volatile public long readyTimeMs; // last time the node had batches ready to send
+ volatile public long drainTimeMs; // last time the node was able to drain batches
+
+ NodeLatencyStats(long nowMs) {
+ readyTimeMs = nowMs;
+ drainTimeMs = nowMs;
+ }
+ }
}
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 2f55e62912d76..55eb6c7be2f54 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
@@ -352,8 +352,16 @@ private long sendProducerData(long now) {
while (iter.hasNext()) {
Node node = iter.next();
if (!this.client.ready(node, now)) {
+ // Update just the readyTimeMs of the latency stats, so that it moves forward
+ // every time the batch is ready (then the difference between readyTimeMs and
+ // drainTimeMs would represent how long data is waiting for the node).
+ this.accumulator.updateNodeLatencyStats(node.id(), now, false);
iter.remove();
notReadyTimeout = Math.min(notReadyTimeout, this.client.pollDelayMs(node, now));
+ } else {
+ // Update both readyTimeMs and drainTimeMs, this would "reset" the node
+ // latency.
+ this.accumulator.updateNodeLatencyStats(node.id(), now, true);
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
index be881a972551d..5aab62eaf225a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
@@ -68,7 +68,6 @@
import org.apache.kafka.common.requests.TxnOffsetCommitRequest.CommittedOffset;
import org.apache.kafka.common.requests.TxnOffsetCommitResponse;
import org.apache.kafka.common.utils.LogContext;
-import org.apache.kafka.common.utils.PrimitiveRef;
import org.slf4j.Logger;
import java.util.ArrayList;
@@ -84,8 +83,6 @@
import java.util.PriorityQueue;
import java.util.Set;
import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.function.Consumer;
import java.util.function.Supplier;
/**
@@ -93,121 +90,19 @@
*/
public class TransactionManager {
private static final int NO_INFLIGHT_REQUEST_CORRELATION_ID = -1;
- private static final int NO_LAST_ACKED_SEQUENCE_NUMBER = -1;
+ static final int NO_LAST_ACKED_SEQUENCE_NUMBER = -1;
private final Logger log;
private final String transactionalId;
private final int transactionTimeoutMs;
private final ApiVersions apiVersions;
- private static class TopicPartitionBookkeeper {
-
- private final Map topicPartitions = new HashMap<>();
-
- private TopicPartitionEntry getPartition(TopicPartition topicPartition) {
- TopicPartitionEntry ent = topicPartitions.get(topicPartition);
- if (ent == null)
- throw new IllegalStateException("Trying to get the sequence number for " + topicPartition +
- ", but the sequence number was never set for this partition.");
- return ent;
- }
-
- private TopicPartitionEntry getOrCreatePartition(TopicPartition topicPartition) {
- TopicPartitionEntry ent = topicPartitions.get(topicPartition);
- if (ent == null) {
- ent = new TopicPartitionEntry();
- topicPartitions.put(topicPartition, ent);
- }
- return ent;
- }
-
- private void addPartition(TopicPartition topicPartition) {
- this.topicPartitions.putIfAbsent(topicPartition, new TopicPartitionEntry());
- }
-
- private boolean contains(TopicPartition topicPartition) {
- return topicPartitions.containsKey(topicPartition);
- }
-
- private void reset() {
- topicPartitions.clear();
- }
-
- private OptionalLong lastAckedOffset(TopicPartition topicPartition) {
- TopicPartitionEntry entry = topicPartitions.get(topicPartition);
- if (entry != null && entry.lastAckedOffset != ProduceResponse.INVALID_OFFSET)
- return OptionalLong.of(entry.lastAckedOffset);
- else
- return OptionalLong.empty();
- }
-
- private OptionalInt lastAckedSequence(TopicPartition topicPartition) {
- TopicPartitionEntry entry = topicPartitions.get(topicPartition);
- if (entry != null && entry.lastAckedSequence != NO_LAST_ACKED_SEQUENCE_NUMBER)
- return OptionalInt.of(entry.lastAckedSequence);
- else
- return OptionalInt.empty();
- }
-
- private void startSequencesAtBeginning(TopicPartition topicPartition, ProducerIdAndEpoch newProducerIdAndEpoch) {
- final PrimitiveRef.IntRef sequence = PrimitiveRef.ofInt(0);
- TopicPartitionEntry topicPartitionEntry = getPartition(topicPartition);
- topicPartitionEntry.resetSequenceNumbers(inFlightBatch -> {
- inFlightBatch.resetProducerState(newProducerIdAndEpoch, sequence.value, inFlightBatch.isTransactional());
- sequence.value += inFlightBatch.recordCount;
- });
- topicPartitionEntry.producerIdAndEpoch = newProducerIdAndEpoch;
- topicPartitionEntry.nextSequence = sequence.value;
- topicPartitionEntry.lastAckedSequence = NO_LAST_ACKED_SEQUENCE_NUMBER;
- }
- }
-
- private static class TopicPartitionEntry {
-
- // The producer id/epoch being used for a given partition.
- private ProducerIdAndEpoch producerIdAndEpoch;
-
- // The base sequence of the next batch bound for a given partition.
- private int nextSequence;
-
- // The sequence number of the last record of the last ack'd batch from the given partition. When there are no
- // in flight requests for a partition, the lastAckedSequence(topicPartition) == nextSequence(topicPartition) - 1.
- private int lastAckedSequence;
-
- // Keep track of the in flight batches bound for a partition, ordered by sequence. This helps us to ensure that
- // we continue to order batches by the sequence numbers even when the responses come back out of order during
- // leader failover. We add a batch to the queue when it is drained, and remove it when the batch completes
- // (either successfully or through a fatal failure).
- private SortedSet inflightBatchesBySequence;
-
- // We keep track of the last acknowledged offset on a per partition basis in order to disambiguate UnknownProducer
- // responses which are due to the retention period elapsing, and those which are due to actual lost data.
- private long lastAckedOffset;
-
- TopicPartitionEntry() {
- this.producerIdAndEpoch = ProducerIdAndEpoch.NONE;
- this.nextSequence = 0;
- this.lastAckedSequence = NO_LAST_ACKED_SEQUENCE_NUMBER;
- this.lastAckedOffset = ProduceResponse.INVALID_OFFSET;
- this.inflightBatchesBySequence = new TreeSet<>(Comparator.comparingInt(ProducerBatch::baseSequence));
- }
-
- void resetSequenceNumbers(Consumer resetSequence) {
- TreeSet newInflights = new TreeSet<>(Comparator.comparingInt(ProducerBatch::baseSequence));
- for (ProducerBatch inflightBatch : inflightBatchesBySequence) {
- resetSequence.accept(inflightBatch);
- newInflights.add(inflightBatch);
- }
- inflightBatchesBySequence = newInflights;
- }
- }
-
- private final TopicPartitionBookkeeper topicPartitionBookkeeper;
+ private final TxnPartitionMap txnPartitionMap;
private final Map pendingTxnOffsetCommits;
- // If a batch bound for a partition expired locally after being sent at least once, the partition has is considered
- // to have an unresolved state. We keep track fo such partitions here, and cannot assign any more sequence numbers
+ // If a batch bound for a partition expired locally after being sent at least once, the partition is considered
+ // to have an unresolved state. We keep track of such partitions here, and cannot assign any more sequence numbers
// for this partition until the unresolved state gets cleared. This may happen if other inflight batches returned
// successfully (indicating that the expired batch actually made it to the broker). If we don't get any successful
// responses for the partition once the inflight request count falls to zero, we reset the producer id and
@@ -320,7 +215,7 @@ public TransactionManager(final LogContext logContext,
this.partitionsWithUnresolvedSequences = new HashMap<>();
this.partitionsToRewriteSequences = new HashSet<>();
this.retryBackoffMs = retryBackoffMs;
- this.topicPartitionBookkeeper = new TopicPartitionBookkeeper();
+ this.txnPartitionMap = new TxnPartitionMap();
this.apiVersions = apiVersions;
}
@@ -444,7 +339,7 @@ public synchronized void maybeAddPartition(TopicPartition topicPartition) {
return;
} else {
log.debug("Begin adding new partition {} to transaction", topicPartition);
- topicPartitionBookkeeper.addPartition(topicPartition);
+ txnPartitionMap.getOrCreate(topicPartition);
newPartitionsInTransaction.add(topicPartition);
}
}
@@ -532,7 +427,7 @@ synchronized public void maybeUpdateProducerIdAndEpoch(TopicPartition topicParti
if (hasStaleProducerIdAndEpoch(topicPartition) && !hasInflightBatches(topicPartition)) {
// If the batch was on a different ID and/or epoch (due to an epoch bump) and all its in-flight batches
// have completed, reset the partition sequence so that the next batch (with the new epoch) starts from 0
- topicPartitionBookkeeper.startSequencesAtBeginning(topicPartition, this.producerIdAndEpoch);
+ txnPartitionMap.startSequencesAtBeginning(topicPartition, this.producerIdAndEpoch);
log.debug("ProducerId of partition {} set to {} with epoch {}. Reinitialize sequence at beginning.",
topicPartition, producerIdAndEpoch.producerId, producerIdAndEpoch.epoch);
}
@@ -561,12 +456,12 @@ private void resetIdempotentProducerId() {
}
private void resetSequenceForPartition(TopicPartition topicPartition) {
- topicPartitionBookkeeper.topicPartitions.remove(topicPartition);
+ txnPartitionMap.topicPartitions.remove(topicPartition);
this.partitionsWithUnresolvedSequences.remove(topicPartition);
}
private void resetSequenceNumbers() {
- topicPartitionBookkeeper.reset();
+ txnPartitionMap.reset();
this.partitionsWithUnresolvedSequences.clear();
}
@@ -585,7 +480,7 @@ private void bumpIdempotentProducerEpoch() {
// When the epoch is bumped, rewrite all in-flight sequences for the partition(s) that triggered the epoch bump
for (TopicPartition topicPartition : this.partitionsToRewriteSequences) {
- this.topicPartitionBookkeeper.startSequencesAtBeginning(topicPartition, this.producerIdAndEpoch);
+ this.txnPartitionMap.startSequencesAtBeginning(topicPartition, this.producerIdAndEpoch);
this.partitionsWithUnresolvedSequences.remove(topicPartition);
}
this.partitionsToRewriteSequences.clear();
@@ -613,27 +508,27 @@ synchronized void bumpIdempotentEpochAndResetIdIfNeeded() {
* Returns the next sequence number to be written to the given TopicPartition.
*/
synchronized Integer sequenceNumber(TopicPartition topicPartition) {
- return topicPartitionBookkeeper.getOrCreatePartition(topicPartition).nextSequence;
+ return txnPartitionMap.getOrCreate(topicPartition).nextSequence;
}
/**
* Returns the current producer id/epoch of the given TopicPartition.
*/
synchronized ProducerIdAndEpoch producerIdAndEpoch(TopicPartition topicPartition) {
- return topicPartitionBookkeeper.getOrCreatePartition(topicPartition).producerIdAndEpoch;
+ return txnPartitionMap.getOrCreate(topicPartition).producerIdAndEpoch;
}
synchronized void incrementSequenceNumber(TopicPartition topicPartition, int increment) {
Integer currentSequence = sequenceNumber(topicPartition);
currentSequence = DefaultRecordBatch.incrementSequence(currentSequence, increment);
- topicPartitionBookkeeper.getPartition(topicPartition).nextSequence = currentSequence;
+ txnPartitionMap.get(topicPartition).nextSequence = currentSequence;
}
synchronized void addInFlightBatch(ProducerBatch batch) {
if (!batch.hasSequence())
throw new IllegalStateException("Can't track batch for partition " + batch.topicPartition + " when sequence is not set.");
- topicPartitionBookkeeper.getPartition(batch.topicPartition).inflightBatchesBySequence.add(batch);
+ txnPartitionMap.get(batch.topicPartition).inflightBatchesBySequence.add(batch);
}
/**
@@ -647,7 +542,7 @@ synchronized int firstInFlightSequence(TopicPartition topicPartition) {
if (!hasInflightBatches(topicPartition))
return RecordBatch.NO_SEQUENCE;
- SortedSet inflightBatches = topicPartitionBookkeeper.getPartition(topicPartition).inflightBatchesBySequence;
+ SortedSet inflightBatches = txnPartitionMap.get(topicPartition).inflightBatchesBySequence;
if (inflightBatches.isEmpty())
return RecordBatch.NO_SEQUENCE;
else
@@ -655,20 +550,20 @@ synchronized int firstInFlightSequence(TopicPartition topicPartition) {
}
synchronized ProducerBatch nextBatchBySequence(TopicPartition topicPartition) {
- SortedSet queue = topicPartitionBookkeeper.getPartition(topicPartition).inflightBatchesBySequence;
+ SortedSet queue = txnPartitionMap.get(topicPartition).inflightBatchesBySequence;
return queue.isEmpty() ? null : queue.first();
}
synchronized void removeInFlightBatch(ProducerBatch batch) {
if (hasInflightBatches(batch.topicPartition)) {
- topicPartitionBookkeeper.getPartition(batch.topicPartition).inflightBatchesBySequence.remove(batch);
+ txnPartitionMap.get(batch.topicPartition).inflightBatchesBySequence.remove(batch);
}
}
private int maybeUpdateLastAckedSequence(TopicPartition topicPartition, int sequence) {
int lastAckedSequence = lastAckedSequence(topicPartition).orElse(NO_LAST_ACKED_SEQUENCE_NUMBER);
if (sequence > lastAckedSequence) {
- topicPartitionBookkeeper.getPartition(topicPartition).lastAckedSequence = sequence;
+ txnPartitionMap.get(topicPartition).lastAckedSequence = sequence;
return sequence;
}
@@ -676,11 +571,11 @@ private int maybeUpdateLastAckedSequence(TopicPartition topicPartition, int sequ
}
synchronized OptionalInt lastAckedSequence(TopicPartition topicPartition) {
- return topicPartitionBookkeeper.lastAckedSequence(topicPartition);
+ return txnPartitionMap.lastAckedSequence(topicPartition);
}
synchronized OptionalLong lastAckedOffset(TopicPartition topicPartition) {
- return topicPartitionBookkeeper.lastAckedOffset(topicPartition);
+ return txnPartitionMap.lastAckedOffset(topicPartition);
}
private void updateLastAckedOffset(ProduceResponse.PartitionResponse response, ProducerBatch batch) {
@@ -692,10 +587,10 @@ private void updateLastAckedOffset(ProduceResponse.PartitionResponse response, P
// response for this. This can happen only if the producer is only idempotent (not transactional) and in
// this case there will be no tracked bookkeeper entry about it, so we have to insert one.
if (!lastAckedOffset.isPresent() && !isTransactional()) {
- topicPartitionBookkeeper.addPartition(batch.topicPartition);
+ txnPartitionMap.getOrCreate(batch.topicPartition);
}
if (lastOffset > lastAckedOffset.orElse(ProduceResponse.INVALID_OFFSET)) {
- topicPartitionBookkeeper.getPartition(batch.topicPartition).lastAckedOffset = lastOffset;
+ txnPartitionMap.get(batch.topicPartition).lastAckedOffset = lastOffset;
} else {
log.trace("Partition {} keeps lastOffset at {}", batch.topicPartition, lastOffset);
}
@@ -768,7 +663,7 @@ synchronized void handleFailedBatch(ProducerBatch batch, RuntimeException except
// This method must only be called when we know that the batch is question has been unequivocally failed by the broker,
// ie. it has received a confirmed fatal status code like 'Message Too Large' or something similar.
private void adjustSequencesDueToFailedBatch(ProducerBatch batch) {
- if (!topicPartitionBookkeeper.contains(batch.topicPartition))
+ if (!txnPartitionMap.contains(batch.topicPartition))
// Sequence numbers are not being tracked for this partition. This could happen if the producer id was just
// reset due to a previous OutOfOrderSequenceException.
return;
@@ -781,7 +676,7 @@ private void adjustSequencesDueToFailedBatch(ProducerBatch batch) {
setNextSequence(batch.topicPartition, currentSequence);
- topicPartitionBookkeeper.getPartition(batch.topicPartition).resetSequenceNumbers(inFlightBatch -> {
+ txnPartitionMap.get(batch.topicPartition).resetSequenceNumbers(inFlightBatch -> {
if (inFlightBatch.baseSequence() < batch.baseSequence())
return;
@@ -790,17 +685,16 @@ private void adjustSequencesDueToFailedBatch(ProducerBatch batch) {
throw new IllegalStateException("Sequence number for batch with sequence " + inFlightBatch.baseSequence()
+ " for partition " + batch.topicPartition + " is going to become negative: " + newSequence);
- log.info("Resetting sequence number of batch with current sequence {} for partition {} to {}", inFlightBatch.baseSequence(), batch.topicPartition, newSequence);
inFlightBatch.resetProducerState(new ProducerIdAndEpoch(inFlightBatch.producerId(), inFlightBatch.producerEpoch()), newSequence, inFlightBatch.isTransactional());
});
}
synchronized boolean hasInflightBatches(TopicPartition topicPartition) {
- return !topicPartitionBookkeeper.getOrCreatePartition(topicPartition).inflightBatchesBySequence.isEmpty();
+ return !txnPartitionMap.getOrCreate(topicPartition).inflightBatchesBySequence.isEmpty();
}
synchronized boolean hasStaleProducerIdAndEpoch(TopicPartition topicPartition) {
- return !producerIdAndEpoch.equals(topicPartitionBookkeeper.getOrCreatePartition(topicPartition).producerIdAndEpoch);
+ return !producerIdAndEpoch.equals(txnPartitionMap.getOrCreate(topicPartition).producerIdAndEpoch);
}
synchronized boolean hasUnresolvedSequences() {
@@ -865,7 +759,7 @@ private boolean isNextSequence(TopicPartition topicPartition, int sequence) {
}
private void setNextSequence(TopicPartition topicPartition, int sequence) {
- topicPartitionBookkeeper.getPartition(topicPartition).nextSequence = sequence;
+ txnPartitionMap.get(topicPartition).nextSequence = sequence;
}
private boolean isNextSequenceForUnresolvedPartition(TopicPartition topicPartition, int sequence) {
@@ -1017,7 +911,7 @@ synchronized boolean canRetry(ProduceResponse.PartitionResponse response, Produc
// inflight batches to be from the beginning and retry them, so that the transaction does not need to
// be aborted. For the idempotent producer, bump the epoch to avoid reusing (sequence, epoch) pairs
if (isTransactional()) {
- topicPartitionBookkeeper.startSequencesAtBeginning(batch.topicPartition, this.producerIdAndEpoch);
+ txnPartitionMap.startSequencesAtBeginning(batch.topicPartition, this.producerIdAndEpoch);
} else {
requestEpochBumpForPartition(batch.topicPartition);
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionEntry.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionEntry.java
new file mode 100644
index 0000000000000..be79d8ee0f1ad
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionEntry.java
@@ -0,0 +1,74 @@
+/*
+ * 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.producer.internals;
+
+import java.util.Comparator;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import org.apache.kafka.common.requests.ProduceResponse;
+import org.apache.kafka.common.utils.ProducerIdAndEpoch;
+
+class TxnPartitionEntry {
+
+ // The producer id/epoch being used for a given partition.
+ ProducerIdAndEpoch producerIdAndEpoch;
+
+ // The base sequence of the next batch bound for a given partition.
+ int nextSequence;
+
+ // The sequence number of the last record of the last ack'd batch from the given partition. When there are no
+ // in flight requests for a partition, the lastAckedSequence(topicPartition) == nextSequence(topicPartition) - 1.
+ int lastAckedSequence;
+
+ // Keep track of the in flight batches bound for a partition, ordered by sequence. This helps us to ensure that
+ // we continue to order batches by the sequence numbers even when the responses come back out of order during
+ // leader failover. We add a batch to the queue when it is drained, and remove it when the batch completes
+ // (either successfully or through a fatal failure).
+ SortedSet inflightBatchesBySequence;
+
+ // We keep track of the last acknowledged offset on a per partition basis in order to disambiguate UnknownProducer
+ // responses which are due to the retention period elapsing, and those which are due to actual lost data.
+ long lastAckedOffset;
+
+ // `inflightBatchesBySequence` should only have batches with the same producer id and producer
+ // epoch, but there is an edge case where we may remove the wrong batch if the comparator
+ // only takes `baseSequence` into account.
+ // See https://github.com/apache/kafka/pull/12096#pullrequestreview-955554191 for details.
+ private static final Comparator PRODUCER_BATCH_COMPARATOR =
+ Comparator.comparingLong(ProducerBatch::producerId)
+ .thenComparingInt(ProducerBatch::producerEpoch)
+ .thenComparingInt(ProducerBatch::baseSequence);
+
+ TxnPartitionEntry() {
+ this.producerIdAndEpoch = ProducerIdAndEpoch.NONE;
+ this.nextSequence = 0;
+ this.lastAckedSequence = TransactionManager.NO_LAST_ACKED_SEQUENCE_NUMBER;
+ this.lastAckedOffset = ProduceResponse.INVALID_OFFSET;
+ this.inflightBatchesBySequence = new TreeSet<>(PRODUCER_BATCH_COMPARATOR);
+ }
+
+ void resetSequenceNumbers(Consumer resetSequence) {
+ TreeSet newInflights = new TreeSet<>(PRODUCER_BATCH_COMPARATOR);
+ for (ProducerBatch inflightBatch : inflightBatchesBySequence) {
+ resetSequence.accept(inflightBatch);
+ newInflights.add(inflightBatch);
+ }
+ inflightBatchesBySequence = newInflights;
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionMap.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionMap.java
new file mode 100644
index 0000000000000..95553119c5bf4
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionMap.java
@@ -0,0 +1,83 @@
+/*
+ * 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.producer.internals;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.OptionalLong;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.requests.ProduceResponse;
+import org.apache.kafka.common.utils.PrimitiveRef;
+import org.apache.kafka.common.utils.ProducerIdAndEpoch;
+
+class TxnPartitionMap {
+
+ final Map topicPartitions = new HashMap<>();
+
+ TxnPartitionEntry get(TopicPartition topicPartition) {
+ TxnPartitionEntry ent = topicPartitions.get(topicPartition);
+ if (ent == null) {
+ throw new IllegalStateException("Trying to get the sequence number for " + topicPartition +
+ ", but the sequence number was never set for this partition.");
+ }
+ return ent;
+ }
+
+ TxnPartitionEntry getOrCreate(TopicPartition topicPartition) {
+ return topicPartitions.computeIfAbsent(topicPartition, tp -> new TxnPartitionEntry());
+ }
+
+ boolean contains(TopicPartition topicPartition) {
+ return topicPartitions.containsKey(topicPartition);
+ }
+
+ void reset() {
+ topicPartitions.clear();
+ }
+
+ OptionalLong lastAckedOffset(TopicPartition topicPartition) {
+ TxnPartitionEntry entry = topicPartitions.get(topicPartition);
+ if (entry != null && entry.lastAckedOffset != ProduceResponse.INVALID_OFFSET) {
+ return OptionalLong.of(entry.lastAckedOffset);
+ } else {
+ return OptionalLong.empty();
+ }
+ }
+
+ OptionalInt lastAckedSequence(TopicPartition topicPartition) {
+ TxnPartitionEntry entry = topicPartitions.get(topicPartition);
+ if (entry != null && entry.lastAckedSequence != TransactionManager.NO_LAST_ACKED_SEQUENCE_NUMBER) {
+ return OptionalInt.of(entry.lastAckedSequence);
+ } else {
+ return OptionalInt.empty();
+ }
+ }
+
+ void startSequencesAtBeginning(TopicPartition topicPartition, ProducerIdAndEpoch newProducerIdAndEpoch) {
+ final PrimitiveRef.IntRef sequence = PrimitiveRef.ofInt(0);
+ TxnPartitionEntry topicPartitionEntry = get(topicPartition);
+ topicPartitionEntry.resetSequenceNumbers(inFlightBatch -> {
+ inFlightBatch.resetProducerState(newProducerIdAndEpoch, sequence.value, inFlightBatch.isTransactional());
+ sequence.value += inFlightBatch.recordCount;
+ });
+ topicPartitionEntry.producerIdAndEpoch = newProducerIdAndEpoch;
+ topicPartitionEntry.nextSequence = sequence.value;
+ topicPartitionEntry.lastAckedSequence = TransactionManager.NO_LAST_ACKED_SEQUENCE_NUMBER;
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/Uuid.java b/clients/src/main/java/org/apache/kafka/common/Uuid.java
index a639f3edd7fc6..83b8f0f0b1644 100644
--- a/clients/src/main/java/org/apache/kafka/common/Uuid.java
+++ b/clients/src/main/java/org/apache/kafka/common/Uuid.java
@@ -31,13 +31,11 @@ public class Uuid implements Comparable {
* A UUID for the metadata topic in KRaft mode. Will never be returned by the randomUuid method.
*/
public static final Uuid METADATA_TOPIC_ID = new Uuid(0L, 1L);
- private static final java.util.UUID METADATA_TOPIC_ID_INTERNAL = new java.util.UUID(0L, 1L);
/**
* A UUID that represents a null or empty UUID. Will never be returned by the randomUuid method.
*/
public static final Uuid ZERO_UUID = new Uuid(0L, 0L);
- private static final java.util.UUID ZERO_ID_INTERNAL = new java.util.UUID(0L, 0L);
private final long mostSignificantBits;
private final long leastSignificantBits;
@@ -51,15 +49,22 @@ public Uuid(long mostSigBits, long leastSigBits) {
this.leastSignificantBits = leastSigBits;
}
+ private static Uuid unsafeRandomUuid() {
+ java.util.UUID jUuid = java.util.UUID.randomUUID();
+ return new Uuid(jUuid.getMostSignificantBits(), jUuid.getLeastSignificantBits());
+ }
+
/**
* Static factory to retrieve a type 4 (pseudo randomly generated) UUID.
+ *
+ * This will not generate a UUID equal to 0, 1, or one whose string representation starts with a dash ("-")
*/
public static Uuid randomUuid() {
- java.util.UUID uuid = java.util.UUID.randomUUID();
- while (uuid.equals(METADATA_TOPIC_ID_INTERNAL) || uuid.equals(ZERO_ID_INTERNAL)) {
- uuid = java.util.UUID.randomUUID();
+ Uuid uuid = unsafeRandomUuid();
+ while (uuid.equals(METADATA_TOPIC_ID) || uuid.equals(ZERO_UUID) || uuid.toString().startsWith("-")) {
+ uuid = unsafeRandomUuid();
}
- return new Uuid(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits());
+ return uuid;
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/common/acl/AclOperation.java b/clients/src/main/java/org/apache/kafka/common/acl/AclOperation.java
index 671069775ca0e..19f4edaaa7187 100644
--- a/clients/src/main/java/org/apache/kafka/common/acl/AclOperation.java
+++ b/clients/src/main/java/org/apache/kafka/common/acl/AclOperation.java
@@ -106,7 +106,17 @@ public enum AclOperation {
/**
* IDEMPOTENT_WRITE operation.
*/
- IDEMPOTENT_WRITE((byte) 12);
+ IDEMPOTENT_WRITE((byte) 12),
+
+ /**
+ * CREATE_TOKENS operation.
+ */
+ CREATE_TOKENS((byte) 13),
+
+ /**
+ * DESCRIBE_TOKENS operation.
+ */
+ DESCRIBE_TOKENS((byte) 14);
// Note: we cannot have more than 30 ACL operations without modifying the format used
// to describe ACL operations in MetadataResponse.
diff --git a/clients/src/main/java/org/apache/kafka/common/compress/KafkaLZ4BlockInputStream.java b/clients/src/main/java/org/apache/kafka/common/compress/KafkaLZ4BlockInputStream.java
index e2fbd5ac04d98..037af8c8dc7f1 100644
--- a/clients/src/main/java/org/apache/kafka/common/compress/KafkaLZ4BlockInputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/compress/KafkaLZ4BlockInputStream.java
@@ -173,7 +173,7 @@ private void readBlock() throws IOException {
in.getInt(); // TODO: verify this content checksum
return;
} else if (blockSize > maxBlockSize) {
- throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize));
+ throw new IOException(String.format("Block size %d exceeded max: %d", blockSize, maxBlockSize));
}
if (in.remaining() < blockSize) {
diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
index 7ef46092749e7..e3fda4d9f5406 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
@@ -380,8 +380,10 @@ private void logAll() {
* Log warnings for any unused configurations
*/
public void logUnused() {
- for (String key : unused())
- log.warn("The configuration '{}' was supplied but isn't a known config.", key);
+ Set unusedkeys = unused();
+ if (!unusedkeys.isEmpty()) {
+ log.warn("These configurations '{}' were supplied but are not used yet.", unusedkeys);
+ }
}
private T getConfiguredInstance(Object klass, Class t, Map configPairs) {
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 85b0103e59ae8..1dfbebbe205a4 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
@@ -16,8 +16,6 @@
*/
package org.apache.kafka.common.config;
-import java.util.function.Function;
-import java.util.stream.Collectors;
import org.apache.kafka.common.config.types.Password;
import org.apache.kafka.common.utils.Utils;
@@ -33,8 +31,10 @@
import java.util.Map;
import java.util.Set;
import java.util.function.BiConsumer;
+import java.util.function.Function;
import java.util.function.Supplier;
import java.util.regex.Pattern;
+import java.util.stream.Collectors;
/**
* This class is used for specifying the set of expected configurations. For each configuration, you can specify
@@ -1121,6 +1121,32 @@ public String toString() {
}
}
+ public static class ListSize implements Validator {
+ final int maxSize;
+
+ private ListSize(final int maxSize) {
+ this.maxSize = maxSize;
+ }
+
+ public static ListSize atMostOfSize(final int maxSize) {
+ return new ListSize(maxSize);
+ }
+
+ @Override
+ public void ensureValid(final String name, final Object value) {
+ @SuppressWarnings("unchecked")
+ List values = (List) value;
+ if (values.size() > maxSize) {
+ throw new ConfigException(name, value, "exceeds maximum list size of [" + maxSize + "].");
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "List containing maximum of " + maxSize + " elements";
+ }
+ }
+
public static class ConfigKey {
public final String name;
public final Type type;
@@ -1217,15 +1243,16 @@ static String niceMemoryUnits(long bytes) {
break;
}
}
+ String resultFormat = " (" + value + " %s" + (value == 1 ? ")" : "s)");
switch (i) {
case 1:
- return " (" + value + " kibibyte" + (value == 1 ? ")" : "s)");
+ return String.format(resultFormat, "kibibyte");
case 2:
- return " (" + value + " mebibyte" + (value == 1 ? ")" : "s)");
+ return String.format(resultFormat, "mebibyte");
case 3:
- return " (" + value + " gibibyte" + (value == 1 ? ")" : "s)");
+ return String.format(resultFormat, "gibibyte");
case 4:
- return " (" + value + " tebibyte" + (value == 1 ? ")" : "s)");
+ return String.format(resultFormat, "tebibyte");
default:
return "";
}
diff --git a/clients/src/main/java/org/apache/kafka/common/config/SslClientAuth.java b/clients/src/main/java/org/apache/kafka/common/config/SslClientAuth.java
index 9d85b184ab9ab..75f8e3640e9fa 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/SslClientAuth.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/SslClientAuth.java
@@ -31,7 +31,7 @@ public enum SslClientAuth {
NONE;
public static final List VALUES =
- Collections.unmodifiableList(Arrays.asList(SslClientAuth.values()));
+ Collections.unmodifiableList(Arrays.asList(SslClientAuth.values()));
public static SslClientAuth forConfig(String key) {
if (key == null) {
@@ -45,4 +45,9 @@ public static SslClientAuth forConfig(String key) {
}
return null;
}
+
+ @Override
+ public String toString() {
+ return super.toString().toLowerCase(Locale.ROOT);
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
index 5061ed5cfcaab..4dfb0bd5a8dab 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
@@ -69,7 +69,7 @@ public class SslConfigs {
public static final String SSL_KEYSTORE_TYPE_CONFIG = "ssl.keystore.type";
public static final String SSL_KEYSTORE_TYPE_DOC = "The file format of the key store file. "
- + "This is optional for client.";
+ + "This is optional for client. The values currently supported by the default `ssl.engine.factory.class` are [JKS, PKCS12, PEM].";
public static final String DEFAULT_SSL_KEYSTORE_TYPE = "JKS";
public static final String SSL_KEYSTORE_KEY_CONFIG = "ssl.keystore.key";
@@ -96,10 +96,10 @@ public class SslConfigs {
public static final String SSL_KEY_PASSWORD_CONFIG = "ssl.key.password";
public static final String SSL_KEY_PASSWORD_DOC = "The password of the private key in the key store file or "
- + "the PEM key specified in `ssl.keystore.key'. This is required for clients only if two-way authentication is configured.";
+ + "the PEM key specified in `ssl.keystore.key'.";
public static final String SSL_TRUSTSTORE_TYPE_CONFIG = "ssl.truststore.type";
- public static final String SSL_TRUSTSTORE_TYPE_DOC = "The file format of the trust store file.";
+ public static final String SSL_TRUSTSTORE_TYPE_DOC = "The file format of the trust store file. The values currently supported by the default `ssl.engine.factory.class` are [JKS, PKCS12, PEM].";
public static final String DEFAULT_SSL_TRUSTSTORE_TYPE = "JKS";
public static final String SSL_TRUSTSTORE_LOCATION_CONFIG = "ssl.truststore.location";
diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
index 73439c5f29af9..5be3057b62db8 100755
--- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
@@ -99,7 +99,7 @@ public class TopicConfig {
"limit only applies to a single record in that case.";
public static final String INDEX_INTERVAL_BYTES_CONFIG = "index.interval.bytes";
- public static final String INDEX_INTERVAL_BYTES_DOCS = "This setting controls how frequently " +
+ public static final String INDEX_INTERVAL_BYTES_DOC = "This setting controls how frequently " +
"Kafka adds an index entry to its offset index. The default setting ensures that we index a " +
"message roughly every 4096 bytes. More indexing allows reads to jump closer to the exact " +
"position in the log but makes the index larger. You probably don't need to change this.";
@@ -139,11 +139,13 @@ public class TopicConfig {
public static final String CLEANUP_POLICY_CONFIG = "cleanup.policy";
public static final String CLEANUP_POLICY_COMPACT = "compact";
public static final String CLEANUP_POLICY_DELETE = "delete";
- public static final String CLEANUP_POLICY_DOC = "A string that is either \"" + CLEANUP_POLICY_DELETE +
- "\" or \"" + CLEANUP_POLICY_COMPACT + "\" or both. This string designates the retention policy to use on " +
- "old log segments. The default policy (\"delete\") will discard old segments when their retention " +
- "time or size limit has been reached. The \"compact\" setting will enable log " +
- "compaction on the topic.";
+ public static final String CLEANUP_POLICY_DOC = "This config designates the retention policy to " +
+ "use on log segments. The \"delete\" policy (which is the default) will discard old segments " +
+ "when their retention time or size limit has been reached. The \"compact\" policy will enable " +
+ "log compaction, which retains the latest value for each key. " +
+ "It is also possible to specify both policies in a comma-separated list (e.g. \"delete,compact\"). " +
+ "In this case, old segments will be discarded per the retention time and size configuration, " +
+ "while retained segments will be compacted.";
public static final String UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG = "unclean.leader.election.enable";
public static final String UNCLEAN_LEADER_ELECTION_ENABLE_DOC = "Indicates whether to enable replicas " +
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/AuthorizerNotReadyException.java b/clients/src/main/java/org/apache/kafka/common/errors/AuthorizerNotReadyException.java
new file mode 100644
index 0000000000000..1c110ef2143cc
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/AuthorizerNotReadyException.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+/**
+ * An exception that indicates that the authorizer is not ready to receive the request yet.
+ */
+public class AuthorizerNotReadyException extends RetriableException {
+ private static final long serialVersionUID = 1L;
+
+ public AuthorizerNotReadyException() {
+ super();
+ }
+}
diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationTest.java b/clients/src/main/java/org/apache/kafka/common/errors/IneligibleReplicaException.java
similarity index 80%
rename from connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationTest.java
rename to clients/src/main/java/org/apache/kafka/common/errors/IneligibleReplicaException.java
index ed82aa97ed48f..6c79add033a4e 100644
--- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationTest.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/IneligibleReplicaException.java
@@ -14,10 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.kafka.connect.mirror.integration;
+package org.apache.kafka.common.errors;
-import org.junit.jupiter.api.Tag;
-
-@Tag("integration")
-public class MirrorConnectorsIntegrationTest extends MirrorConnectorsIntegrationBaseTest {
+public class IneligibleReplicaException extends ApiException {
+ public IneligibleReplicaException(String message) {
+ super(message);
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NewLeaderElectedException.java b/clients/src/main/java/org/apache/kafka/common/errors/NewLeaderElectedException.java
new file mode 100644
index 0000000000000..20fd869df9f0e
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/NewLeaderElectedException.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+public class NewLeaderElectedException extends ApiException {
+ public NewLeaderElectedException(String message) {
+ super(message);
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/feature/Features.java b/clients/src/main/java/org/apache/kafka/common/feature/Features.java
index 4006d71947fb7..a39ee37e53dca 100644
--- a/clients/src/main/java/org/apache/kafka/common/feature/Features.java
+++ b/clients/src/main/java/org/apache/kafka/common/feature/Features.java
@@ -32,7 +32,6 @@
*
* @param is the type of version range.
* @see SupportedVersionRange
- * @see FinalizedVersionRange
*/
public class Features {
private final Map features;
@@ -57,20 +56,6 @@ public static Features supportedFeatures(Map(features);
}
- /**
- * @param features Map of feature name to FinalizedVersionRange.
- *
- * @return Returns a new Features object representing finalized features.
- */
- public static Features finalizedFeatures(Map features) {
- return new Features<>(features);
- }
-
- // Visible for testing.
- public static Features emptyFinalizedFeatures() {
- return new Features<>(new HashMap<>());
- }
-
public static Features emptySupportedFeatures() {
return new Features<>(new HashMap<>());
}
@@ -138,19 +123,6 @@ private static Features fromFeaturesMap(
entry -> converter.fromMap(entry.getValue()))));
}
- /**
- * Converts from a map to Features.
- *
- * @param featuresMap the map representation of a Features object,
- * generated using the toMap() API.
- *
- * @return the Features object
- */
- public static Features fromFinalizedFeaturesMap(
- Map> featuresMap) {
- return fromFeaturesMap(featuresMap, FinalizedVersionRange::fromMap);
- }
-
/**
* Converts from a map to Features.
*
diff --git a/clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
deleted file mode 100644
index 27e6440478644..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.kafka.common.feature;
-
-import java.util.Map;
-
-/**
- * An extended {@link BaseVersionRange} representing the min/max versions for a finalized feature.
- */
-public class FinalizedVersionRange extends BaseVersionRange {
- // Label for the min version key, that's used only to convert to/from a map.
- private static final String MIN_VERSION_LEVEL_KEY_LABEL = "min_version_level";
-
- // Label for the max version key, that's used only to convert to/from a map.
- private static final String MAX_VERSION_LEVEL_KEY_LABEL = "max_version_level";
-
- public FinalizedVersionRange(short minVersionLevel, short maxVersionLevel) {
- super(MIN_VERSION_LEVEL_KEY_LABEL, minVersionLevel, MAX_VERSION_LEVEL_KEY_LABEL, maxVersionLevel);
- }
-
- public static FinalizedVersionRange fromMap(Map versionRangeMap) {
- return new FinalizedVersionRange(
- BaseVersionRange.valueOrThrow(MIN_VERSION_LEVEL_KEY_LABEL, versionRangeMap),
- BaseVersionRange.valueOrThrow(MAX_VERSION_LEVEL_KEY_LABEL, versionRangeMap));
- }
-
- /**
- * Checks if the [min, max] version level range of this object does *NOT* fall within the
- * [min, max] range of the provided SupportedVersionRange parameter.
- *
- * @param supportedVersionRange the SupportedVersionRange to be checked
- *
- * @return - true, if the version levels are compatible
- * - false otherwise
- */
- public boolean isIncompatibleWith(SupportedVersionRange supportedVersionRange) {
- return min() < supportedVersionRange.min() || max() > supportedVersionRange.max();
- }
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
index 8993014a74b2e..a864a91762635 100644
--- a/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
+++ b/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java
@@ -41,4 +41,16 @@ public static SupportedVersionRange fromMap(Map versionRangeMap)
BaseVersionRange.valueOrThrow(MIN_VERSION_KEY_LABEL, versionRangeMap),
BaseVersionRange.valueOrThrow(MAX_VERSION_KEY_LABEL, versionRangeMap));
}
+
+ /**
+ * Checks if the version level does *NOT* fall within the [min, max] range of this SupportedVersionRange.
+ *
+ * @param version the version to be checked
+ *
+ * @return - true, if the version levels are incompatible
+ * - false otherwise
+ */
+ public boolean isIncompatibleWith(short version) {
+ return min() > version || max() < version;
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/internals/Topic.java b/clients/src/main/java/org/apache/kafka/common/internals/Topic.java
index 7a5fefb3d9f23..fbf491bab9800 100644
--- a/clients/src/main/java/org/apache/kafka/common/internals/Topic.java
+++ b/clients/src/main/java/org/apache/kafka/common/internals/Topic.java
@@ -16,6 +16,7 @@
*/
package org.apache.kafka.common.internals;
+import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.utils.Utils;
@@ -27,6 +28,8 @@ public class Topic {
public static final String GROUP_METADATA_TOPIC_NAME = "__consumer_offsets";
public static final String TRANSACTION_STATE_TOPIC_NAME = "__transaction_state";
+ public static final String METADATA_TOPIC_NAME = "__cluster_metadata";
+ public static final TopicPartition METADATA_TOPIC_PARTITION = new TopicPartition(METADATA_TOPIC_NAME, 0);
public static final String LEGAL_CHARS = "[a-zA-Z0-9._-]";
private static final Set INTERNAL_TOPICS = Collections.unmodifiableSet(
@@ -67,6 +70,17 @@ public static boolean hasCollisionChars(String topic) {
return topic.contains("_") || topic.contains(".");
}
+ /**
+ * Unify topic name with a period ('.') or underscore ('_'), this is only used to check collision and will not
+ * be used to really change topic name.
+ *
+ * @param topic A topic to unify
+ * @return A unified topic name
+ */
+ public static String unifyCollisionChars(String topic) {
+ return topic.replace('.', '_');
+ }
+
/**
* Returns true if the topicNames collide due to a period ('.') or underscore ('_') in the same position.
*
@@ -75,7 +89,7 @@ public static boolean hasCollisionChars(String topic) {
* @return true if the topics collide
*/
public static boolean hasCollision(String topicA, String topicB) {
- return topicA.replace('.', '_').equals(topicB.replace('.', '_'));
+ return unifyCollisionChars(topicA).equals(unifyCollisionChars(topicB));
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Gauge.java b/clients/src/main/java/org/apache/kafka/common/metrics/Gauge.java
index 647942b3d03b2..d71bbd853db16 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/Gauge.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Gauge.java
@@ -19,6 +19,7 @@
/**
* A gauge metric is an instantaneous reading of a particular value.
*/
+@FunctionalInterface
public interface Gauge extends MetricValueProvider {
/**
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
index 52b7794a4c10b..a7581442f5d12 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
@@ -502,6 +502,7 @@ public void addMetric(MetricName metricName, MetricConfig config, Measurable mea
*
* @param metricName The name of the metric
* @param metricValueProvider The metric value provider associated with this metric
+ * @throws IllegalArgumentException if a metric with same name already exists.
*/
public void addMetric(MetricName metricName, MetricConfig config, MetricValueProvider> metricValueProvider) {
KafkaMetric m = new KafkaMetric(new Object(),
@@ -509,7 +510,10 @@ public void addMetric(MetricName metricName, MetricConfig config, MetricValuePro
Objects.requireNonNull(metricValueProvider),
config == null ? this.config : config,
time);
- registerMetric(m);
+ KafkaMetric existingMetric = registerMetric(m);
+ if (existingMetric != null) {
+ throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one.");
+ }
}
/**
@@ -524,6 +528,26 @@ public void addMetric(MetricName metricName, MetricValueProvider> metricValueP
addMetric(metricName, null, metricValueProvider);
}
+ /**
+ * Create or get an existing metric to monitor an object that implements MetricValueProvider.
+ * This metric won't be associated with any sensor. This is a way to expose existing values as metrics.
+ * This method takes care of synchronisation while updating/accessing metrics by concurrent threads.
+ *
+ * @param metricName The name of the metric
+ * @param metricValueProvider The metric value provider associated with this metric
+ * @return Existing KafkaMetric if already registered or else a newly created one
+ */
+ public KafkaMetric addMetricIfAbsent(MetricName metricName, MetricConfig config, MetricValueProvider> metricValueProvider) {
+ KafkaMetric metric = new KafkaMetric(new Object(),
+ Objects.requireNonNull(metricName),
+ Objects.requireNonNull(metricValueProvider),
+ config == null ? this.config : config,
+ time);
+
+ KafkaMetric existingMetric = registerMetric(metric);
+ return existingMetric == null ? metric : existingMetric;
+ }
+
/**
* Remove a metric if it exists and return it. Return null otherwise. If a metric is removed, `metricRemoval`
* will be invoked for each reporter.
@@ -563,11 +587,20 @@ public synchronized void removeReporter(MetricsReporter reporter) {
}
}
- synchronized void registerMetric(KafkaMetric metric) {
+ /**
+ * Register a metric if not present or return the already existing metric with the same name.
+ * When a metric is newly registered, this method returns null
+ *
+ * @param metric The KafkaMetric to register
+ * @return the existing metric with the same name or null
+ */
+ synchronized KafkaMetric registerMetric(KafkaMetric metric) {
MetricName metricName = metric.metricName();
- if (this.metrics.containsKey(metricName))
- throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one.");
- this.metrics.put(metricName, metric);
+ KafkaMetric existingMetric = this.metrics.putIfAbsent(metricName, metric);
+ if (existingMetric != null) {
+ return existingMetric;
+ }
+ // newly added metric
for (MetricsReporter reporter : reporters) {
try {
reporter.metricChange(metric);
@@ -576,6 +609,7 @@ synchronized void registerMetric(KafkaMetric metric) {
}
}
log.trace("Registered metric named {}", metricName);
+ return null;
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
index 5ae3b8d997a64..25f3c21a31365 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
@@ -297,7 +297,10 @@ public synchronized boolean add(CompoundStat stat, MetricConfig config) {
for (NamedMeasurable m : stat.stats()) {
final KafkaMetric metric = new KafkaMetric(lock, m.name(), m.stat(), statConfig, time);
if (!metrics.containsKey(metric.metricName())) {
- registry.registerMetric(metric);
+ KafkaMetric existingMetric = registry.registerMetric(metric);
+ if (existingMetric != null) {
+ throw new IllegalArgumentException("A metric named '" + metric.metricName() + "' already exists, can't register another one.");
+ }
metrics.put(metric.metricName(), metric);
}
}
@@ -336,7 +339,10 @@ public synchronized boolean add(final MetricName metricName, final MeasurableSta
statConfig,
time
);
- registry.registerMetric(metric);
+ KafkaMetric existingMetric = registry.registerMetric(metric);
+ if (existingMetric != null) {
+ throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one.");
+ }
metrics.put(metric.metricName(), metric);
stats.add(new StatAndConfig(Objects.requireNonNull(stat), metric::config));
return true;
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
index c6b8574186a88..09b7c05c8f283 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
@@ -91,7 +91,10 @@ public long windowSize(MetricConfig config, long now) {
if (numFullWindows < minFullWindows)
totalElapsedTimeMs += (minFullWindows - numFullWindows) * config.timeWindowMs();
- return totalElapsedTimeMs;
+ // If window size is being calculated at the exact beginning of the window with no prior samples, the window size
+ // will result in a value of 0. Calculation of rate over a window is size 0 is undefined, hence, we assume the
+ // minimum window size to be at least 1ms.
+ return Math.max(totalElapsedTimeMs, 1);
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
index c3b4888339083..bd1175a8ee0a3 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
@@ -845,7 +845,7 @@ private void clear() {
boolean hasPending = false;
if (!sendFailed)
hasPending = maybeReadFromClosingChannel(channel);
- if (!hasPending || sendFailed) {
+ if (!hasPending) {
doClose(channel, true);
it.remove();
}
diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
index 893fd6a4ecedd..844c2bd2c17d7 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
@@ -71,6 +71,8 @@ private enum State {
CLOSING
}
+ private static final String TLS13 = "TLSv1.3";
+
private final String channelId;
private final SSLEngine sslEngine;
private final SelectionKey key;
@@ -449,7 +451,7 @@ private void handshakeFinished() throws IOException {
if (netWriteBuffer.hasRemaining())
key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
else {
- state = sslEngine.getSession().getProtocol().equals("TLSv1.3") ? State.POST_HANDSHAKE : State.READY;
+ state = sslEngine.getSession().getProtocol().equals(TLS13) ? State.POST_HANDSHAKE : State.READY;
key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
SSLSession session = sslEngine.getSession();
log.debug("SSL handshake completed successfully with peerHost '{}' peerPort {} peerPrincipal '{}' cipherSuite '{}'",
@@ -585,10 +587,11 @@ public int read(ByteBuffer dst) throws IOException {
throw e;
}
netReadBuffer.compact();
- // handle ssl renegotiation.
+ // reject renegotiation if TLS < 1.3, key updates for TLS 1.3 are allowed
if (unwrapResult.getHandshakeStatus() != HandshakeStatus.NOT_HANDSHAKING &&
unwrapResult.getHandshakeStatus() != HandshakeStatus.FINISHED &&
- unwrapResult.getStatus() == Status.OK) {
+ unwrapResult.getStatus() == Status.OK &&
+ !sslEngine.getSession().getProtocol().equals(TLS13)) {
log.error("Renegotiation requested, but it is not supported, channelId {}, " +
"appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {} handshakeStatus {}", channelId,
appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position(), unwrapResult.getHandshakeStatus());
@@ -706,9 +709,12 @@ public int write(ByteBuffer src) throws IOException {
SSLEngineResult wrapResult = sslEngine.wrap(src, netWriteBuffer);
netWriteBuffer.flip();
- //handle ssl renegotiation
- if (wrapResult.getHandshakeStatus() != HandshakeStatus.NOT_HANDSHAKING && wrapResult.getStatus() == Status.OK)
+ // reject renegotiation if TLS < 1.3, key updates for TLS 1.3 are allowed
+ if (wrapResult.getHandshakeStatus() != HandshakeStatus.NOT_HANDSHAKING &&
+ wrapResult.getStatus() == Status.OK &&
+ !sslEngine.getSession().getProtocol().equals(TLS13)) {
throw renegotiationException();
+ }
if (wrapResult.getStatus() == Status.OK) {
written += wrapResult.bytesConsumed();
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
index 5aa6186f43423..628c9407cc2dd 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
@@ -97,8 +97,8 @@ public enum ApiKeys {
BEGIN_QUORUM_EPOCH(ApiMessageType.BEGIN_QUORUM_EPOCH, true, RecordBatch.MAGIC_VALUE_V0, false),
END_QUORUM_EPOCH(ApiMessageType.END_QUORUM_EPOCH, true, RecordBatch.MAGIC_VALUE_V0, false),
DESCRIBE_QUORUM(ApiMessageType.DESCRIBE_QUORUM, true, RecordBatch.MAGIC_VALUE_V0, true),
- ALTER_ISR(ApiMessageType.ALTER_ISR, true),
- UPDATE_FEATURES(ApiMessageType.UPDATE_FEATURES),
+ ALTER_PARTITION(ApiMessageType.ALTER_PARTITION, true),
+ UPDATE_FEATURES(ApiMessageType.UPDATE_FEATURES, true, true),
ENVELOPE(ApiMessageType.ENVELOPE, true, RecordBatch.MAGIC_VALUE_V0, false),
FETCH_SNAPSHOT(ApiMessageType.FETCH_SNAPSHOT, false, RecordBatch.MAGIC_VALUE_V0, false),
DESCRIBE_CLUSTER(ApiMessageType.DESCRIBE_CLUSTER),
@@ -241,7 +241,7 @@ private static String toHtml() {
b.append("");
b.append("\n");
}
- b.append("\n");
+ b.append("\n");
return b.toString();
}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index f48ae6c2332b4..2ca42bafcfb86 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -52,6 +52,7 @@
import org.apache.kafka.common.errors.InconsistentTopicIdException;
import org.apache.kafka.common.errors.InconsistentVoterSetException;
import org.apache.kafka.common.errors.InconsistentClusterIdException;
+import org.apache.kafka.common.errors.IneligibleReplicaException;
import org.apache.kafka.common.errors.InvalidCommitOffsetSizeException;
import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.errors.InvalidFetchSessionEpochException;
@@ -77,6 +78,7 @@
import org.apache.kafka.common.errors.LogDirNotFoundException;
import org.apache.kafka.common.errors.MemberIdRequiredException;
import org.apache.kafka.common.errors.NetworkException;
+import org.apache.kafka.common.errors.NewLeaderElectedException;
import org.apache.kafka.common.errors.NoReassignmentInProgressException;
import org.apache.kafka.common.errors.NotControllerException;
import org.apache.kafka.common.errors.NotCoordinatorException;
@@ -364,7 +366,9 @@ public enum Errors {
INCONSISTENT_TOPIC_ID(103, "The log's topic ID did not match the topic ID in the request", InconsistentTopicIdException::new),
INCONSISTENT_CLUSTER_ID(104, "The clusterId in the request does not match that found on the server", InconsistentClusterIdException::new),
TRANSACTIONAL_ID_NOT_FOUND(105, "The transactionalId could not be found", TransactionalIdNotFoundException::new),
- FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage", FetchSessionTopicIdException::new);
+ FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage", FetchSessionTopicIdException::new),
+ INELIGIBLE_REPLICA(107, "The new ISR contains at least one ineligible replica.", IneligibleReplicaException::new),
+ NEW_LEADER_ELECTED(108, "The AlterPartition request successfully updated the partition state but the leader has changed.", NewLeaderElectedException::new);
private static final Logger log = LoggerFactory.getLogger(Errors.class);
@@ -500,7 +504,7 @@ private static String toHtml() {
b.append("");
b.append("\n");
}
- b.append("\n");
+ b.append("\n");
return b.toString();
}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java b/clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java
index 288ffd08f7187..b366ebd8ead01 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java
@@ -29,6 +29,11 @@
public final class MessageUtil {
+
+ public static final long UNSIGNED_INT_MAX = 4294967295L;
+
+ public static final int UNSIGNED_SHORT_MAX = 65535;
+
/**
* Copy a byte buffer into an array. This will not affect the buffer's
* position or mark.
@@ -87,13 +92,22 @@ public static short jsonNodeToShort(JsonNode node, String about) {
public static int jsonNodeToUnsignedShort(JsonNode node, String about) {
int value = jsonNodeToInt(node, about);
- if (value < 0 || value > 65535) {
+ if (value < 0 || value > UNSIGNED_SHORT_MAX) {
throw new RuntimeException(about + ": value " + value +
" does not fit in a 16-bit unsigned integer.");
}
return value;
}
+ public static long jsonNodeToUnsignedInt(JsonNode node, String about) {
+ long value = jsonNodeToLong(node, about);
+ if (value < 0 || value > UNSIGNED_INT_MAX) {
+ throw new RuntimeException(about + ": value " + value +
+ " does not fit in a 32-bit unsigned integer.");
+ }
+ return value;
+ }
+
public static int jsonNodeToInt(JsonNode node, String about) {
if (node.isInt()) {
return node.asInt();
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index d455b26eb2d87..a75eb0661d102 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -112,7 +112,7 @@ private static void schemaToFieldTableHtml(Schema schema, StringBuilder b) {
b.append("");
b.append("\n");
}
- b.append("\n");
+ b.append("\n");
}
public static String toHtml() {
@@ -148,7 +148,7 @@ public static String toHtml() {
Schema schema = requests[i];
// Schema
if (schema != null) {
- b.append("");
+ b.append("
");
// Version header
b.append("
");
b.append(key.name);
@@ -159,7 +159,7 @@ public static String toHtml() {
b.append("
");
schemaToFieldTableHtml(requests[i], b);
}
- b.append("\n");
+ b.append("
\n");
}
// Responses
@@ -169,7 +169,7 @@ public static String toHtml() {
Schema schema = responses[i];
// Schema
if (schema != null) {
- b.append("");
+ b.append("
");
// Version header
b.append("
");
b.append(key.name);
@@ -180,7 +180,7 @@ public static String toHtml() {
b.append("
");
schemaToFieldTableHtml(responses[i], b);
}
- b.append("\n");
+ b.append("
\n");
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java b/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java
index 9c9e461ca806a..561696827df63 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java
@@ -75,4 +75,8 @@ default Uuid readUuid() {
default int readUnsignedShort() {
return Short.toUnsignedInt(readShort());
}
+
+ default long readUnsignedInt() {
+ return Integer.toUnsignedLong(readInt());
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Writable.java b/clients/src/main/java/org/apache/kafka/common/protocol/Writable.java
index 8dbec87134257..0677340af4d68 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Writable.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Writable.java
@@ -54,4 +54,8 @@ default void writeUnsignedShort(int i) {
// ints outside the valid range of a short.
writeShort((short) i);
}
+
+ default void writeUnsignedInt(long i) {
+ writeInt((int) i);
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
index 44726f8240c0f..f030387b6fcdf 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
@@ -97,6 +97,12 @@ public Uint16(String name, String docString) {
}
}
+ public static class Uint32 extends Field {
+ public Uint32(String name, String docString) {
+ super(name, Type.UNSIGNED_INT32, docString, false, null);
+ }
+ }
+
public static class Float64 extends Field {
public Float64(String name, String docString) {
super(name, Type.FLOAT64, docString, false, null);
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
index 9b9b5e66b6654..e39a84137bc0f 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
@@ -23,6 +23,9 @@
import java.util.Arrays;
import java.util.Objects;
+import static org.apache.kafka.common.protocol.MessageUtil.UNSIGNED_INT_MAX;
+import static org.apache.kafka.common.protocol.MessageUtil.UNSIGNED_SHORT_MAX;
+
/**
* A record that can be serialized and deserialized according to a pre-defined schema
*/
@@ -97,6 +100,10 @@ public Integer get(Field.Uint16 field) {
return getInt(field.name);
}
+ public Long get(Field.Uint32 field) {
+ return getLong(field.name);
+ }
+
public Short get(Field.Int16 field) {
return getShort(field.name);
}
@@ -270,6 +277,10 @@ public Long getUnsignedInt(String name) {
return (Long) get(name);
}
+ public Long getUnsignedInt(BoundField field) {
+ return (Long) get(field);
+ }
+
public Long getLong(BoundField field) {
return (Long) get(field);
}
@@ -400,13 +411,21 @@ public Struct set(Field.Int16 def, short value) {
}
public Struct set(Field.Uint16 def, int value) {
- if (value < 0 || value > 65535) {
+ if (value < 0 || value > UNSIGNED_SHORT_MAX) {
throw new RuntimeException("Invalid value for unsigned short for " +
def.name + ": " + value);
}
return set(def.name, value);
}
+ public Struct set(Field.Uint32 def, long value) {
+ if (value < 0 || value > UNSIGNED_INT_MAX) {
+ throw new RuntimeException("Invalid value for unsigned int for " +
+ def.name + ": " + value);
+ }
+ return set(def.name, value);
+ }
+
public Struct set(Field.Float64 def, double value) {
return set(def.name, value);
}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/TaggedFields.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/TaggedFields.java
index 4e1ab0d4d5add..129f80c90ba1d 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/TaggedFields.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/TaggedFields.java
@@ -178,4 +178,11 @@ public String typeName() {
public String documentation() {
return "Represents a series of tagged fields.";
}
+
+ /**
+ * The number of tagged fields
+ */
+ public int numFields() {
+ return this.fields.size();
+ }
}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
index 46a59bd08210e..4af74dbf4cc03 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
@@ -1120,7 +1120,7 @@ private static String toHtml() {
b.append("");
b.append("\n");
}
- b.append("\n");
+ b.append("\n");
return b.toString();
}
diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
index 1b9754ffabbbf..c526929b72e96 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
@@ -190,4 +190,10 @@ else if (ZSTD.name.equals(name))
else
throw new IllegalArgumentException("Unknown compression name: " + name);
}
+
+ @Override
+ public String toString() {
+ return name;
+ }
+
}
diff --git a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java
index e74f6417febc1..66a4a14d22bed 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java
@@ -28,9 +28,9 @@
*/
public class ControlRecordUtils {
- public static final short LEADER_CHANGE_SCHEMA_HIGHEST_VERSION = new LeaderChangeMessage().highestSupportedVersion();
- public static final short SNAPSHOT_HEADER_HIGHEST_VERSION = new SnapshotHeaderRecord().highestSupportedVersion();
- public static final short SNAPSHOT_FOOTER_HIGHEST_VERSION = new SnapshotFooterRecord().highestSupportedVersion();
+ public static final short LEADER_CHANGE_SCHEMA_HIGHEST_VERSION = LeaderChangeMessage.HIGHEST_SUPPORTED_VERSION;
+ public static final short SNAPSHOT_HEADER_HIGHEST_VERSION = SnapshotHeaderRecord.HIGHEST_SUPPORTED_VERSION;
+ public static final short SNAPSHOT_FOOTER_HIGHEST_VERSION = SnapshotFooterRecord.HIGHEST_SUPPORTED_VERSION;
public static LeaderChangeMessage deserializeLeaderChangeMessage(Record record) {
ControlRecordType recordType = ControlRecordType.parse(record.key());
diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
index bd80981d84bda..bc8f32491c0a8 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
@@ -107,7 +107,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
static final int PARTITION_LEADER_EPOCH_LENGTH = 4;
static final int MAGIC_OFFSET = PARTITION_LEADER_EPOCH_OFFSET + PARTITION_LEADER_EPOCH_LENGTH;
static final int MAGIC_LENGTH = 1;
- static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
+ public static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
static final int CRC_LENGTH = 4;
static final int ATTRIBUTES_OFFSET = CRC_OFFSET + CRC_LENGTH;
static final int ATTRIBUTE_LENGTH = 2;
diff --git a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java
index 17addef74de4e..01176518457de 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java
@@ -87,7 +87,7 @@ public long writeTo(TransferableChannel channel, long previouslyWritten, int rem
// Since we have already sent at least one batch and we have committed to the fetch size, we
// send an overflow batch. The consumer will read the first few records and then fetch from the
// offset of the batch which has the unsupported compression type. At that time, we will
- // send back the UNSUPPORTED_COMPRESSION_TYPE erro which will allow the consumer to fail gracefully.
+ // send back the UNSUPPORTED_COMPRESSION_TYPE error which will allow the consumer to fail gracefully.
convertedRecords = buildOverflowBatch(remaining);
}
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 b825a937e084b..02fb7d7f12d55 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
@@ -548,7 +548,6 @@ public void append(long timestamp, ByteBuffer key, ByteBuffer value) {
* @param key The record key
* @param value The record value
* @param headers The record headers if there are any
- * @return CRC of the record or null if record-level CRC is not supported for the message format
*/
public void append(long timestamp, ByteBuffer key, ByteBuffer value, Header[] headers) {
appendWithOffset(nextSequentialOffset(), timestamp, key, value, headers);
@@ -559,7 +558,6 @@ public void append(long timestamp, ByteBuffer key, ByteBuffer value, Header[] he
* @param timestamp The record timestamp
* @param key The record key
* @param value The record value
- * @return CRC of the record or null if record-level CRC is not supported for the message format
*/
public void append(long timestamp, byte[] key, byte[] value) {
append(timestamp, wrapNullable(key), wrapNullable(value), Record.EMPTY_HEADERS);
diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatchIterator.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatchIterator.java
index 88af039847d5e..967cff80cc7a5 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatchIterator.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatchIterator.java
@@ -17,8 +17,10 @@
package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.utils.AbstractIterator;
+import java.io.EOFException;
import java.io.IOException;
class RecordBatchIterator extends AbstractIterator {
@@ -36,6 +38,8 @@ protected T makeNext() {
if (batch == null)
return allDone();
return batch;
+ } catch (EOFException e) {
+ throw new CorruptRecordException("Unexpected EOF while attempting to read the next batch", e);
} catch (IOException e) {
throw new KafkaException(e);
}
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
index 0c38e998fe247..0d96d842d6df5 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
@@ -279,8 +279,8 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion,
return EndQuorumEpochRequest.parse(buffer, apiVersion);
case DESCRIBE_QUORUM:
return DescribeQuorumRequest.parse(buffer, apiVersion);
- case ALTER_ISR:
- return AlterIsrRequest.parse(buffer, apiVersion);
+ case ALTER_PARTITION:
+ return AlterPartitionRequest.parse(buffer, apiVersion);
case UPDATE_FEATURES:
return UpdateFeaturesRequest.parse(buffer, apiVersion);
case ENVELOPE:
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
index 47f2b3c7f3099..cd99f472ebb0a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
@@ -223,8 +223,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response
return EndQuorumEpochResponse.parse(responseBuffer, version);
case DESCRIBE_QUORUM:
return DescribeQuorumResponse.parse(responseBuffer, version);
- case ALTER_ISR:
- return AlterIsrResponse.parse(responseBuffer, version);
+ case ALTER_PARTITION:
+ return AlterPartitionResponse.parse(responseBuffer, version);
case UPDATE_FEATURES:
return UpdateFeaturesResponse.parse(responseBuffer, version);
case ENVELOPE:
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java
deleted file mode 100644
index 516c2ce76aa85..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.common.requests;
-
-import org.apache.kafka.common.message.AlterIsrRequestData;
-import org.apache.kafka.common.message.AlterIsrResponseData;
-import org.apache.kafka.common.protocol.ApiKeys;
-import org.apache.kafka.common.protocol.ByteBufferAccessor;
-import org.apache.kafka.common.protocol.Errors;
-
-import java.nio.ByteBuffer;
-
-public class AlterIsrRequest extends AbstractRequest {
-
- private final AlterIsrRequestData data;
-
- public AlterIsrRequest(AlterIsrRequestData data, short apiVersion) {
- super(ApiKeys.ALTER_ISR, apiVersion);
- this.data = data;
- }
-
- @Override
- public AlterIsrRequestData data() {
- return data;
- }
-
- /**
- * Get an error response for a request with specified throttle time in the response if applicable
- */
- @Override
- public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
- return new AlterIsrResponse(new AlterIsrResponseData()
- .setThrottleTimeMs(throttleTimeMs)
- .setErrorCode(Errors.forException(e).code()));
- }
-
- public static AlterIsrRequest parse(ByteBuffer buffer, short version) {
- return new AlterIsrRequest(new AlterIsrRequestData(new ByteBufferAccessor(buffer), version), version);
- }
-
- public static class Builder extends AbstractRequest.Builder {
-
- private final AlterIsrRequestData data;
-
- public Builder(AlterIsrRequestData data) {
- super(ApiKeys.ALTER_ISR);
- this.data = data;
- }
-
- @Override
- public AlterIsrRequest build(short version) {
- return new AlterIsrRequest(data, version);
- }
-
- @Override
- public String toString() {
- return data.toString();
- }
- }
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java
new file mode 100644
index 0000000000000..2d246f21041fa
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.message.AlterPartitionRequestData;
+import org.apache.kafka.common.message.AlterPartitionResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.Errors;
+
+import java.nio.ByteBuffer;
+
+public class AlterPartitionRequest extends AbstractRequest {
+
+ private final AlterPartitionRequestData data;
+
+ public AlterPartitionRequest(AlterPartitionRequestData data, short apiVersion) {
+ super(ApiKeys.ALTER_PARTITION, apiVersion);
+ this.data = data;
+ }
+
+ @Override
+ public AlterPartitionRequestData data() {
+ return data;
+ }
+
+ /**
+ * Get an error response for a request with specified throttle time in the response if applicable
+ */
+ @Override
+ public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
+ return new AlterPartitionResponse(new AlterPartitionResponseData()
+ .setThrottleTimeMs(throttleTimeMs)
+ .setErrorCode(Errors.forException(e).code()));
+ }
+
+ public static AlterPartitionRequest parse(ByteBuffer buffer, short version) {
+ return new AlterPartitionRequest(new AlterPartitionRequestData(new ByteBufferAccessor(buffer), version), version);
+ }
+
+ public static class Builder extends AbstractRequest.Builder {
+
+ private final AlterPartitionRequestData data;
+
+ /**
+ * Constructs a builder for AlterPartitionRequest.
+ *
+ * @param data The data to be sent. Note that because the version of the
+ * request is not known at this time, it is expected that all
+ * topics have a topic id and a topic name set.
+ * @param canUseTopicIds True if version 2 and above can be used.
+ */
+ public Builder(AlterPartitionRequestData data, boolean canUseTopicIds) {
+ super(
+ ApiKeys.ALTER_PARTITION,
+ ApiKeys.ALTER_PARTITION.oldestVersion(),
+ // Version 1 is the maximum version that can be used without topic ids.
+ canUseTopicIds ? ApiKeys.ALTER_PARTITION.latestVersion() : 1
+ );
+ this.data = data;
+ }
+
+ @Override
+ public AlterPartitionRequest build(short version) {
+ return new AlterPartitionRequest(data, version);
+ }
+
+ @Override
+ public String toString() {
+ return data.toString();
+ }
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java
similarity index 75%
rename from clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java
rename to clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java
index c3106ed94cbde..d2ace4112f4c1 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java
@@ -17,7 +17,7 @@
package org.apache.kafka.common.requests;
-import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.AlterPartitionResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;
@@ -26,17 +26,17 @@
import java.util.HashMap;
import java.util.Map;
-public class AlterIsrResponse extends AbstractResponse {
+public class AlterPartitionResponse extends AbstractResponse {
- private final AlterIsrResponseData data;
+ private final AlterPartitionResponseData data;
- public AlterIsrResponse(AlterIsrResponseData data) {
- super(ApiKeys.ALTER_ISR);
+ public AlterPartitionResponse(AlterPartitionResponseData data) {
+ super(ApiKeys.ALTER_PARTITION);
this.data = data;
}
@Override
- public AlterIsrResponseData data() {
+ public AlterPartitionResponseData data() {
return data;
}
@@ -55,7 +55,7 @@ public int throttleTimeMs() {
return data.throttleTimeMs();
}
- public static AlterIsrResponse parse(ByteBuffer buffer, short version) {
- return new AlterIsrResponse(new AlterIsrResponseData(new ByteBufferAccessor(buffer), version));
+ public static AlterPartitionResponse parse(ByteBuffer buffer, short version) {
+ return new AlterPartitionResponse(new AlterPartitionResponseData(new ByteBufferAccessor(buffer), version));
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
index 1190989576380..7c98eb2679b5a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
@@ -16,10 +16,11 @@
*/
package org.apache.kafka.common.requests;
+import org.apache.kafka.clients.NodeApiVersions;
import org.apache.kafka.common.feature.Features;
-import org.apache.kafka.common.feature.FinalizedVersionRange;
import org.apache.kafka.common.feature.SupportedVersionRange;
import org.apache.kafka.common.message.ApiMessageType;
+import org.apache.kafka.common.message.ApiMessageType.ListenerType;
import org.apache.kafka.common.message.ApiVersionsResponseData;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionCollection;
@@ -33,6 +34,7 @@
import org.apache.kafka.common.record.RecordVersion;
import java.nio.ByteBuffer;
+import java.util.Collections;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
@@ -116,8 +118,33 @@ public static ApiVersionsResponse createApiVersionsResponse(
throttleTimeMs,
apiVersions,
Features.emptySupportedFeatures(),
- Features.emptyFinalizedFeatures(),
- UNKNOWN_FINALIZED_FEATURES_EPOCH
+ Collections.emptyMap(),
+ UNKNOWN_FINALIZED_FEATURES_EPOCH);
+ }
+
+ public static ApiVersionsResponse createApiVersionsResponse(
+ int throttleTimeMs,
+ RecordVersion minRecordVersion,
+ Features latestSupportedFeatures,
+ Map finalizedFeatures,
+ long finalizedFeaturesEpoch,
+ NodeApiVersions controllerApiVersions,
+ ListenerType listenerType
+ ) {
+ ApiVersionCollection apiKeys;
+ if (controllerApiVersions != null) {
+ apiKeys = intersectForwardableApis(
+ listenerType, minRecordVersion, controllerApiVersions.allSupportedApiVersions());
+ } else {
+ apiKeys = filterApis(minRecordVersion, listenerType);
+ }
+
+ return createApiVersionsResponse(
+ throttleTimeMs,
+ apiKeys,
+ latestSupportedFeatures,
+ finalizedFeatures,
+ finalizedFeaturesEpoch
);
}
@@ -125,7 +152,7 @@ public static ApiVersionsResponse createApiVersionsResponse(
int throttleTimeMs,
ApiVersionCollection apiVersions,
Features latestSupportedFeatures,
- Features finalizedFeatures,
+ Map finalizedFeatures,
long finalizedFeaturesEpoch
) {
return new ApiVersionsResponse(
@@ -205,7 +232,7 @@ private static ApiVersionsResponseData createApiVersionsResponseData(
final Errors error,
final ApiVersionCollection apiKeys,
final Features latestSupportedFeatures,
- final Features finalizedFeatures,
+ final Map finalizedFeatures,
final long finalizedFeaturesEpoch
) {
final ApiVersionsResponseData data = new ApiVersionsResponseData();
@@ -235,14 +262,14 @@ private static SupportedFeatureKeyCollection createSupportedFeatureKeys(
}
private static FinalizedFeatureKeyCollection createFinalizedFeatureKeys(
- Features finalizedFeatures) {
+ Map finalizedFeatures) {
FinalizedFeatureKeyCollection converted = new FinalizedFeatureKeyCollection();
- for (Map.Entry feature : finalizedFeatures.features().entrySet()) {
+ for (Map.Entry feature : finalizedFeatures.entrySet()) {
final FinalizedFeatureKey key = new FinalizedFeatureKey();
- final FinalizedVersionRange versionLevelRange = feature.getValue();
+ final short versionLevel = feature.getValue();
key.setName(feature.getKey());
- key.setMinVersionLevel(versionLevelRange.min());
- key.setMaxVersionLevel(versionLevelRange.max());
+ key.setMinVersionLevel(versionLevel);
+ key.setMaxVersionLevel(versionLevel);
converted.add(key);
}
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java
index 1fee1b71eb3a4..b48f84f1fa6dd 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java
@@ -45,7 +45,8 @@ public CreateDelegationTokenRequestData data() {
@Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
- return CreateDelegationTokenResponse.prepareResponse(throttleTimeMs, Errors.forException(e), KafkaPrincipal.ANONYMOUS);
+ return CreateDelegationTokenResponse.prepareResponse(version(), throttleTimeMs, Errors.forException(e),
+ KafkaPrincipal.ANONYMOUS, KafkaPrincipal.ANONYMOUS);
}
public static class Builder extends AbstractRequest.Builder {
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java
index b679a30c8dd5c..22c2e1259019b 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java
@@ -39,9 +39,11 @@ public static CreateDelegationTokenResponse parse(ByteBuffer buffer, short versi
new CreateDelegationTokenResponseData(new ByteBufferAccessor(buffer), version));
}
- public static CreateDelegationTokenResponse prepareResponse(int throttleTimeMs,
+ public static CreateDelegationTokenResponse prepareResponse(int version,
+ int throttleTimeMs,
Errors error,
KafkaPrincipal owner,
+ KafkaPrincipal tokenRequester,
long issueTimestamp,
long expiryTimestamp,
long maxTimestamp,
@@ -57,11 +59,16 @@ public static CreateDelegationTokenResponse prepareResponse(int throttleTimeMs,
.setMaxTimestampMs(maxTimestamp)
.setTokenId(tokenId)
.setHmac(hmac.array());
+ if (version > 2) {
+ data.setTokenRequesterPrincipalType(tokenRequester.getPrincipalType())
+ .setTokenRequesterPrincipalName(tokenRequester.getName());
+ }
return new CreateDelegationTokenResponse(data);
}
- public static CreateDelegationTokenResponse prepareResponse(int throttleTimeMs, Errors error, KafkaPrincipal owner) {
- return prepareResponse(throttleTimeMs, error, owner, -1, -1, -1, "", ByteBuffer.wrap(new byte[] {}));
+ public static CreateDelegationTokenResponse prepareResponse(int version, int throttleTimeMs, Errors error,
+ KafkaPrincipal owner, KafkaPrincipal requester) {
+ return prepareResponse(version, throttleTimeMs, error, owner, requester, -1, -1, -1, "", ByteBuffer.wrap(new byte[] {}));
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java
index 9bf59e844a6c3..bd3b5fd57c002 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java
@@ -71,7 +71,7 @@ public boolean ownersListEmpty() {
@Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
- return new DescribeDelegationTokenResponse(throttleTimeMs, Errors.forException(e));
+ return new DescribeDelegationTokenResponse(version(), throttleTimeMs, Errors.forException(e));
}
public static DescribeDelegationTokenRequest parse(ByteBuffer buffer, short version) {
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java
index 4a2162f53aaef..4fd1d99652661 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java
@@ -36,22 +36,29 @@ public class DescribeDelegationTokenResponse extends AbstractResponse {
private final DescribeDelegationTokenResponseData data;
- public DescribeDelegationTokenResponse(int throttleTimeMs, Errors error, List tokens) {
+ public DescribeDelegationTokenResponse(int version, int throttleTimeMs, Errors error, List tokens) {
super(ApiKeys.DESCRIBE_DELEGATION_TOKEN);
List describedDelegationTokenList = tokens
.stream()
- .map(dt -> new DescribedDelegationToken()
- .setTokenId(dt.tokenInfo().tokenId())
- .setPrincipalType(dt.tokenInfo().owner().getPrincipalType())
- .setPrincipalName(dt.tokenInfo().owner().getName())
- .setIssueTimestamp(dt.tokenInfo().issueTimestamp())
- .setMaxTimestamp(dt.tokenInfo().maxTimestamp())
- .setExpiryTimestamp(dt.tokenInfo().expiryTimestamp())
- .setHmac(dt.hmac())
- .setRenewers(dt.tokenInfo().renewers()
- .stream()
- .map(r -> new DescribedDelegationTokenRenewer().setPrincipalName(r.getName()).setPrincipalType(r.getPrincipalType()))
- .collect(Collectors.toList())))
+ .map(dt -> {
+ DescribedDelegationToken ddt = new DescribedDelegationToken()
+ .setTokenId(dt.tokenInfo().tokenId())
+ .setPrincipalType(dt.tokenInfo().owner().getPrincipalType())
+ .setPrincipalName(dt.tokenInfo().owner().getName())
+ .setIssueTimestamp(dt.tokenInfo().issueTimestamp())
+ .setMaxTimestamp(dt.tokenInfo().maxTimestamp())
+ .setExpiryTimestamp(dt.tokenInfo().expiryTimestamp())
+ .setHmac(dt.hmac())
+ .setRenewers(dt.tokenInfo().renewers()
+ .stream()
+ .map(r -> new DescribedDelegationTokenRenewer().setPrincipalName(r.getName()).setPrincipalType(r.getPrincipalType()))
+ .collect(Collectors.toList()));
+ if (version > 2) {
+ ddt.setTokenRequesterPrincipalType(dt.tokenInfo().tokenRequester().getPrincipalType())
+ .setTokenRequesterPrincipalName(dt.tokenInfo().tokenRequester().getName());
+ }
+ return ddt;
+ })
.collect(Collectors.toList());
this.data = new DescribeDelegationTokenResponseData()
@@ -60,8 +67,8 @@ public DescribeDelegationTokenResponse(int throttleTimeMs, Errors error, List());
+ public DescribeDelegationTokenResponse(int version, int throttleTimeMs, Errors error) {
+ this(version, throttleTimeMs, error, new ArrayList<>());
}
public DescribeDelegationTokenResponse(DescribeDelegationTokenResponseData data) {
@@ -99,6 +106,7 @@ public List tokens() {
.map(ddt -> new DelegationToken(new TokenInformation(
ddt.tokenId(),
new KafkaPrincipal(ddt.principalType(), ddt.principalName()),
+ new KafkaPrincipal(ddt.tokenRequesterPrincipalType(), ddt.tokenRequesterPrincipalName()),
ddt.renewers()
.stream()
.map(ddtr -> new KafkaPrincipal(ddtr.principalType(), ddtr.principalName()))
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
index 537d188ec120e..fe8aebbc4f6b8 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
@@ -31,6 +31,7 @@
public class DescribeLogDirsResponse extends AbstractResponse {
public static final long INVALID_OFFSET_LAG = -1L;
+ public static final long UNKNOWN_VOLUME_BYTES = -1L;
private final DescribeLogDirsResponseData data;
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
index 48ba022610e43..09242bfc4bf0c 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
@@ -351,8 +351,10 @@ public Map fetchData(Map topicNam
if (fetchData == null) {
synchronized (this) {
if (fetchData == null) {
- fetchData = new LinkedHashMap<>();
- short version = version();
+ // Assigning the lazy-initialized `fetchData` in the last step
+ // to avoid other threads accessing a half-initialized object.
+ final LinkedHashMap fetchDataTmp = new LinkedHashMap<>();
+ final short version = version();
data.topics().forEach(fetchTopic -> {
String name;
if (version < 13) {
@@ -362,7 +364,7 @@ public Map