From 59ba28f2e71afc4aea994da5b0d6f18e75d924b6 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 31 May 2024 21:48:55 +0200 Subject: [PATCH 001/128] KAFKA-16860; [1/2] Introduce group.version feature flag (#16120) This patch introduces the `group.version` feature flag with one version: 1) Version 1 enables the new consumer group rebalance protocol (KIP-848). Reviewers: Justine Olshan --- .../main/scala/kafka/server/KafkaApis.scala | 13 ++- .../test/java/kafka/test/ClusterConfig.java | 21 ++++- .../test/java/kafka/test/ClusterInstance.java | 5 +- .../kafka/test/ClusterTestExtensionsTest.java | 16 ++-- .../kafka/test/annotation/ClusterFeature.java | 33 +++++++ .../kafka/test/annotation/ClusterTest.java | 3 +- .../test/junit/ClusterTestExtensions.java | 7 ++ .../junit/RaftClusterInvocationContext.java | 20 +++- .../kafka/api/IntegrationTestHarness.scala | 2 +- .../kafka/server/QuorumTestHarness.scala | 11 ++- .../kafka/zk/ZkMigrationIntegrationTest.scala | 3 +- .../AbstractApiVersionsRequestTest.scala | 7 +- .../kafka/server/ApiVersionsRequestTest.scala | 4 +- .../kafka/server/BrokerFeaturesTest.scala | 3 +- .../ConsumerGroupDescribeRequestsTest.scala | 48 ++++++++-- .../ConsumerGroupHeartbeatRequestTest.scala | 75 +++++++++++---- .../ConsumerProtocolMigrationTest.scala | 48 ++++++---- .../server/DeleteGroupsRequestTest.scala | 50 ++++++---- .../unit/kafka/server/KafkaApisTest.scala | 92 ++++++++++++++++--- .../kafka/server/ListGroupsRequestTest.scala | 22 +++-- .../server/OffsetCommitRequestTest.scala | 34 ++++--- .../server/OffsetDeleteRequestTest.scala | 22 +++-- .../kafka/server/OffsetFetchRequestTest.scala | 79 +++++++++------- .../kafka/server/ReplicationQuotasTest.scala | 2 +- .../unit/kafka/tools/StorageToolTest.scala | 14 ++- .../controller/QuorumControllerTest.java | 8 +- .../kafka/controller/QuorumFeaturesTest.java | 2 +- .../apache/kafka/server/common/Features.java | 3 +- .../kafka/server/common/GroupVersion.java | 62 +++++++++++++ .../kafka/server/common/MetadataVersion.java | 5 +- .../server/common/MetadataVersionTest.java | 6 ++ .../kafka/tools/FeatureCommandTest.java | 21 ++++- .../group/ConsumerGroupCommandTest.java | 5 +- .../group/ConsumerGroupCommandTestUtils.java | 5 + 34 files changed, 574 insertions(+), 177 deletions(-) create mode 100644 core/src/test/java/kafka/test/annotation/ClusterFeature.java create mode 100644 server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 70258e8dac..b76ebff59c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -67,10 +67,10 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{ProducerIdAndEpoch, Time} import org.apache.kafka.common.{Node, TopicIdPartition, TopicPartition, Uuid} -import org.apache.kafka.coordinator.group.GroupCoordinator +import org.apache.kafka.coordinator.group.{Group, GroupCoordinator} import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.{GroupVersion, MetadataVersion} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0} import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData} @@ -3797,10 +3797,15 @@ class KafkaApis(val requestChannel: RequestChannel, ) } + private def isConsumerGroupProtocolEnabled(): Boolean = { + val version = metadataCache.features().finalizedFeatures().getOrDefault(GroupVersion.FEATURE_NAME, 0.toShort) + config.groupCoordinatorRebalanceProtocols.contains(Group.GroupType.CONSUMER) && version >= GroupVersion.GV_1.featureLevel + } + def handleConsumerGroupHeartbeat(request: RequestChannel.Request): CompletableFuture[Unit] = { val consumerGroupHeartbeatRequest = request.body[ConsumerGroupHeartbeatRequest] - if (!config.isNewGroupCoordinatorEnabled) { + if (!isConsumerGroupProtocolEnabled()) { // The API is not supported by the "old" group coordinator (the default). If the // new one is not enabled, we fail directly here. requestHelper.sendMaybeThrottle(request, consumerGroupHeartbeatRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception)) @@ -3825,7 +3830,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleConsumerGroupDescribe(request: RequestChannel.Request): CompletableFuture[Unit] = { val consumerGroupDescribeRequest = request.body[ConsumerGroupDescribeRequest] - if (!config.isNewGroupCoordinatorEnabled) { + if (!isConsumerGroupProtocolEnabled()) { // The API is not supported by the "old" group coordinator (the default). If the // new one is not enabled, we fail directly here. requestHelper.sendMaybeThrottle(request, request.body[ConsumerGroupDescribeRequest].getErrorResponse(Errors.UNSUPPORTED_VERSION.exception)) diff --git a/core/src/test/java/kafka/test/ClusterConfig.java b/core/src/test/java/kafka/test/ClusterConfig.java index c116a7d505..b72bb1a578 100644 --- a/core/src/test/java/kafka/test/ClusterConfig.java +++ b/core/src/test/java/kafka/test/ClusterConfig.java @@ -19,6 +19,7 @@ import kafka.test.annotation.Type; import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.MetadataVersion; import java.io.File; @@ -58,13 +59,15 @@ public class ClusterConfig { private final Map saslClientProperties; private final List tags; private final Map> perServerProperties; + private final Map features; @SuppressWarnings("checkstyle:ParameterNumber") private ClusterConfig(Set types, int brokers, int controllers, int disksPerBroker, boolean autoStart, SecurityProtocol securityProtocol, String listenerName, File trustStoreFile, MetadataVersion metadataVersion, Map serverProperties, Map producerProperties, Map consumerProperties, Map adminClientProperties, Map saslServerProperties, - Map saslClientProperties, Map> perServerProperties, List tags) { + Map saslClientProperties, Map> perServerProperties, List tags, + Map features) { // do fail fast. the following values are invalid for both zk and kraft modes. if (brokers < 0) throw new IllegalArgumentException("Number of brokers must be greater or equal to zero."); if (controllers < 0) throw new IllegalArgumentException("Number of controller must be greater or equal to zero."); @@ -87,6 +90,7 @@ private ClusterConfig(Set types, int brokers, int controllers, int disksPe this.saslClientProperties = Objects.requireNonNull(saslClientProperties); this.perServerProperties = Objects.requireNonNull(perServerProperties); this.tags = Objects.requireNonNull(tags); + this.features = Objects.requireNonNull(features); } public Set clusterTypes() { @@ -157,6 +161,10 @@ public List tags() { return tags; } + public Map features() { + return features; + } + public Set displayTags() { Set displayTags = new LinkedHashSet<>(tags); displayTags.add("MetadataVersion=" + metadataVersion); @@ -198,7 +206,8 @@ public static Builder builder(ClusterConfig clusterConfig) { .setSaslServerProperties(clusterConfig.saslServerProperties) .setSaslClientProperties(clusterConfig.saslClientProperties) .setPerServerProperties(clusterConfig.perServerProperties) - .setTags(clusterConfig.tags); + .setTags(clusterConfig.tags) + .setFeatures(clusterConfig.features); } public static class Builder { @@ -219,6 +228,7 @@ public static class Builder { private Map saslClientProperties = Collections.emptyMap(); private Map> perServerProperties = Collections.emptyMap(); private List tags = Collections.emptyList(); + private Map features = Collections.emptyMap(); private Builder() {} @@ -309,11 +319,16 @@ public Builder setTags(List tags) { return this; } + public Builder setFeatures(Map features) { + this.features = Collections.unmodifiableMap(features); + return this; + } + public ClusterConfig build() { return new ClusterConfig(types, brokers, controllers, disksPerBroker, autoStart, securityProtocol, listenerName, trustStoreFile, metadataVersion, serverProperties, producerProperties, consumerProperties, adminClientProperties, saslServerProperties, saslClientProperties, - perServerProperties, tags); + perServerProperties, tags, features); } } } diff --git a/core/src/test/java/kafka/test/ClusterInstance.java b/core/src/test/java/kafka/test/ClusterInstance.java index 8cb5ae3d21..03e19f39b6 100644 --- a/core/src/test/java/kafka/test/ClusterInstance.java +++ b/core/src/test/java/kafka/test/ClusterInstance.java @@ -40,7 +40,6 @@ import static org.apache.kafka.clients.consumer.GroupProtocol.CLASSIC; import static org.apache.kafka.clients.consumer.GroupProtocol.CONSUMER; import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG; -import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG; public interface ClusterInstance { @@ -159,9 +158,7 @@ default Set supportedGroupProtocols() { Set supportedGroupProtocols = new HashSet<>(); supportedGroupProtocols.add(CLASSIC); - // KafkaConfig#isNewGroupCoordinatorEnabled check both NEW_GROUP_COORDINATOR_ENABLE_CONFIG and GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG - if (serverProperties.getOrDefault(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "").equals("true") || - serverProperties.getOrDefault(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "").contains("consumer")) { + if (serverProperties.getOrDefault(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "").contains("consumer")) { supportedGroupProtocols.add(CONSUMER); } diff --git a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java index cdbb942e6d..c8b53f8b8a 100644 --- a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java +++ b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java @@ -185,13 +185,10 @@ public void testNoAutoStart() { @ClusterTest public void testDefaults(ClusterInstance clusterInstance) { - Assertions.assertEquals(MetadataVersion.IBP_3_8_IV0, clusterInstance.config().metadataVersion()); + Assertions.assertEquals(MetadataVersion.IBP_4_0_IVO, clusterInstance.config().metadataVersion()); } @ClusterTests({ - @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { - @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), - }), @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer"), }), @@ -199,10 +196,6 @@ public void testDefaults(ClusterInstance clusterInstance) { @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer"), }), - @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { - @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), - @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic"), - }), @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "false"), @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer"), @@ -217,12 +210,19 @@ public void testSupportedNewGroupProtocols(ClusterInstance clusterInstance) { } @ClusterTests({ + @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { + @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), + }), @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "false"), }), @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic"), }), + @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { + @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic"), + }), @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "false"), @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic"), diff --git a/core/src/test/java/kafka/test/annotation/ClusterFeature.java b/core/src/test/java/kafka/test/annotation/ClusterFeature.java new file mode 100644 index 0000000000..ab72f13288 --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/ClusterFeature.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.test.annotation; + +import org.apache.kafka.server.common.Features; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Documented +@Target({ElementType.ANNOTATION_TYPE}) +@Retention(RetentionPolicy.RUNTIME) +public @interface ClusterFeature { + Features feature(); + short version(); +} diff --git a/core/src/test/java/kafka/test/annotation/ClusterTest.java b/core/src/test/java/kafka/test/annotation/ClusterTest.java index 9364ce690e..bd95249b4f 100644 --- a/core/src/test/java/kafka/test/annotation/ClusterTest.java +++ b/core/src/test/java/kafka/test/annotation/ClusterTest.java @@ -40,8 +40,9 @@ AutoStart autoStart() default AutoStart.DEFAULT; SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT; String listener() default ""; - MetadataVersion metadataVersion() default MetadataVersion.IBP_3_8_IV0; + MetadataVersion metadataVersion() default MetadataVersion.IBP_4_0_IVO; ClusterConfigProperty[] serverProperties() default {}; // users can add tags that they want to display in test String[] tags() default {}; + ClusterFeature[] features() default {}; } diff --git a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java index 2290a0a99e..3e6b5a7d66 100644 --- a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java +++ b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java @@ -18,6 +18,7 @@ package kafka.test.junit; import kafka.test.ClusterConfig; +import kafka.test.annotation.ClusterFeature; import kafka.test.annotation.ClusterTest; import kafka.test.annotation.ClusterTestDefaults; import kafka.test.annotation.ClusterTests; @@ -25,6 +26,7 @@ import kafka.test.annotation.ClusterConfigProperty; import kafka.test.annotation.Type; import kafka.test.annotation.AutoStart; +import org.apache.kafka.server.common.Features; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.TestTemplateInvocationContext; import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider; @@ -172,6 +174,10 @@ private List processClusterTestInternal(Extension .filter(e -> e.id() != -1) .collect(Collectors.groupingBy(ClusterConfigProperty::id, Collectors.mapping(Function.identity(), Collectors.toMap(ClusterConfigProperty::key, ClusterConfigProperty::value, (a, b) -> b)))); + + Map features = Arrays.stream(annot.features()) + .collect(Collectors.toMap(ClusterFeature::feature, ClusterFeature::version)); + ClusterConfig config = ClusterConfig.builder() .setTypes(new HashSet<>(Arrays.asList(types))) .setBrokers(annot.brokers() == 0 ? defaults.brokers() : annot.brokers()) @@ -184,6 +190,7 @@ private List processClusterTestInternal(Extension .setSecurityProtocol(annot.securityProtocol()) .setMetadataVersion(annot.metadataVersion()) .setTags(Arrays.asList(annot.tags())) + .setFeatures(features) .build(); return Arrays.stream(types).map(type -> type.invocationContexts(context.getRequiredTestMethod().getName(), config)) diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index 9857d4c92c..3a7bac5aad 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -28,15 +28,20 @@ import kafka.testkit.TestKitNodes; import kafka.zk.EmbeddedZookeeper; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.metadata.BrokerState; +import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.extension.AfterTestExecutionCallback; import org.junit.jupiter.api.extension.BeforeTestExecutionCallback; import org.junit.jupiter.api.extension.Extension; import org.junit.jupiter.api.extension.TestTemplateInvocationContext; import scala.compat.java8.OptionConverters; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -237,8 +242,21 @@ public Map controllers() { public void format() throws Exception { if (formated.compareAndSet(false, true)) { + List records = new ArrayList<>(); + records.add( + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(clusterConfig.metadataVersion().featureLevel()), (short) 0)); + + clusterConfig.features().forEach((feature, version) -> { + records.add( + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(feature.featureName()). + setFeatureLevel(version), (short) 0)); + }); + TestKitNodes nodes = new TestKitNodes.Builder() - .setBootstrapMetadataVersion(clusterConfig.metadataVersion()) + .setBootstrapMetadata(BootstrapMetadata.fromRecords(records, "testkit")) .setCombined(isCombined) .setNumBrokerNodes(clusterConfig.numBrokers()) .setNumDisksPerBroker(clusterConfig.numDisksPerBroker()) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index e1f549c4e5..192dfa392d 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -62,7 +62,6 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { } override def generateConfigs: Seq[KafkaConfig] = { - val cfgs = TestUtils.createBrokerConfigs(brokerCount, zkConnectOrNull, interBrokerSecurityProtocol = Some(securityProtocol), trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties, logDirCount = logDirCount) configureListeners(cfgs) @@ -72,6 +71,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { } if (isNewGroupCoordinatorEnabled()) { cfgs.foreach(_.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true")) + cfgs.foreach(_.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer")) } if(isKRaftTest()) { diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index fbb59dd90f..9f787a1b16 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -37,7 +37,7 @@ import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationF import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion} import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs} import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} import org.apache.zookeeper.client.ZKClientConfig @@ -342,6 +342,15 @@ abstract class QuorumTestHarness extends Logging { setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(metadataVersion.featureLevel()), 0.toShort)) + if (isNewGroupCoordinatorEnabled()) { + metadataRecords.add(new ApiMessageAndVersion( + new FeatureLevelRecord() + .setName(Features.GROUP_VERSION.featureName) + .setFeatureLevel(Features.GROUP_VERSION.latestTesting), + 0.toShort + )) + } + optionalMetadataRecords.foreach { metadataArguments => for (record <- metadataArguments) metadataRecords.add(record) } diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index 5fd32d8f4b..c0b6d916ec 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -72,7 +72,8 @@ object ZkMigrationIntegrationTest { MetadataVersion.IBP_3_7_IV1, MetadataVersion.IBP_3_7_IV2, MetadataVersion.IBP_3_7_IV4, - MetadataVersion.IBP_3_8_IV0 + MetadataVersion.IBP_3_8_IV0, + MetadataVersion.IBP_4_0_IVO ).map { mv => val serverProperties = new util.HashMap[String, String]() serverProperties.put("inter.broker.listener.name", "EXTERNAL") diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index 30318c1ccd..b79cf12aa4 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.record.RecordVersion import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, RequestUtils} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.{GroupVersion, MetadataVersion} import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Tag @@ -69,9 +69,12 @@ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) { assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().finalizedFeatures().find(MetadataVersion.FEATURE_NAME).minVersionLevel()) assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().finalizedFeatures().find(MetadataVersion.FEATURE_NAME).maxVersionLevel()) - assertEquals(1, apiVersionsResponse.data().supportedFeatures().size()) + assertEquals(2, apiVersionsResponse.data().supportedFeatures().size()) assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), apiVersionsResponse.data().supportedFeatures().find(MetadataVersion.FEATURE_NAME).minVersion()) assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().supportedFeatures().find(MetadataVersion.FEATURE_NAME).maxVersion()) + + assertEquals(0, apiVersionsResponse.data().supportedFeatures().find(GroupVersion.FEATURE_NAME).minVersion()) + assertEquals(GroupVersion.GV_1.featureLevel(), apiVersionsResponse.data().supportedFeatures().find(GroupVersion.FEATURE_NAME).maxVersion()) } val expectedApis = if (!cluster.isKRaftTest) { ApiVersionsResponse.collectApis( diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index 116cef5146..a7415b5d50 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -47,7 +47,7 @@ object ApiVersionsRequestTest { List(ClusterConfig.defaultBuilder() .setTypes(java.util.Collections.singleton(Type.ZK)) .setServerProperties(serverProperties) - .setMetadataVersion(MetadataVersion.IBP_3_8_IV0) + .setMetadataVersion(MetadataVersion.IBP_4_0_IVO) .build()).asJava } @@ -83,7 +83,7 @@ object ApiVersionsRequestTest { class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { @ClusterTemplate("testApiVersionsRequestTemplate") - @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_3_8_IV0, serverProperties = Array( + @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_4_0_IVO, serverProperties = Array( new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "true") )) diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala index 78432b7110..9c43df2b2d 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -18,7 +18,7 @@ package kafka.server import org.apache.kafka.common.feature.{Features, SupportedVersionRange} -import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.{Features => ServerFeatures, MetadataVersion} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Test @@ -95,6 +95,7 @@ class BrokerFeaturesTest { val expectedFeatures = Map[String, Short]( MetadataVersion.FEATURE_NAME -> MetadataVersion.latestTesting().featureLevel(), + ServerFeatures.GROUP_VERSION.featureName() -> ServerFeatures.GROUP_VERSION.latestTesting(), "test_feature_1" -> 4, "test_feature_2" -> 3, "test_feature_3" -> 7) diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala index b2e4e0f85f..b1f8b8405e 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala @@ -18,7 +18,7 @@ package kafka.server import kafka.server.GroupCoordinatorBaseRequestTest import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.common.ConsumerGroupState @@ -26,6 +26,7 @@ import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData.{Assign import org.apache.kafka.common.message.{ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse} +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{Tag, Timeout} @@ -60,11 +61,46 @@ class ConsumerGroupDescribeRequestsTest(cluster: ClusterInstance) extends GroupC assertEquals(expectedResponse, consumerGroupDescribeResponse.data) } - @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) + def testConsumerGroupDescribeIsInaccessibleWhenDisabledByGroupVersion(): Unit = { + val consumerGroupDescribeRequest = new ConsumerGroupDescribeRequest.Builder( + new ConsumerGroupDescribeRequestData().setGroupIds(List("grp-1", "grp-2").asJava) + ).build(ApiKeys.CONSUMER_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)) + + val consumerGroupDescribeResponse = connectAndReceive[ConsumerGroupDescribeResponse](consumerGroupDescribeRequest) + val expectedResponse = new ConsumerGroupDescribeResponseData() + expectedResponse.groups().add( + new ConsumerGroupDescribeResponseData.DescribedGroup() + .setGroupId("grp-1") + .setErrorCode(Errors.UNSUPPORTED_VERSION.code) + ) + expectedResponse.groups.add( + new ConsumerGroupDescribeResponseData.DescribedGroup() + .setGroupId("grp-2") + .setErrorCode(Errors.UNSUPPORTED_VERSION.code) + ) + + assertEquals(expectedResponse, consumerGroupDescribeResponse.data) + } + + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testConsumerGroupDescribeWithNewGroupCoordinator(): Unit = { // Creates the __consumer_offsets topics because it won't be created automatically // in this test because it does not use FindCoordinator API. diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala index 16e0f3d0ef..1cf0cedaf2 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala @@ -17,13 +17,14 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, Type} import kafka.test.junit.ClusterTestExtensions import kafka.test.junit.RaftClusterInvocationContext.RaftClusterInstance import kafka.utils.TestUtils import org.apache.kafka.common.message.{ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse} +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNotNull} import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -36,7 +37,7 @@ import scala.jdk.CollectionConverters._ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { @ClusterTest() - def testConsumerGroupHeartbeatIsAccessibleWhenEnabled(): Unit = { + def testConsumerGroupHeartbeatIsInaccessibleWhenDisabledByStaticConfig(): Unit = { val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequest.Builder( new ConsumerGroupHeartbeatRequestData() ).build() @@ -46,11 +47,35 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { assertEquals(expectedResponse, consumerGroupHeartbeatResponse.data) } - @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) + def testConsumerGroupHeartbeatIsInaccessibleWhenDisabledByGroupVersion(): Unit = { + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequest.Builder( + new ConsumerGroupHeartbeatRequestData() + ).build() + + val consumerGroupHeartbeatResponse = connectAndReceive(consumerGroupHeartbeatRequest) + val expectedResponse = new ConsumerGroupHeartbeatResponseData().setErrorCode(Errors.UNSUPPORTED_VERSION.code) + assertEquals(expectedResponse, consumerGroupHeartbeatResponse.data) + } + + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testConsumerGroupHeartbeatIsAccessibleWhenNewGroupCoordinatorIsEnabled(): Unit = { val raftCluster = cluster.asInstanceOf[RaftClusterInstance] val admin = cluster.createAdminClient() @@ -134,11 +159,17 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { assertEquals(-1, consumerGroupHeartbeatResponse.data.memberEpoch) } - @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testRejoiningStaticMemberGetsAssignmentsBackWhenNewGroupCoordinatorIsEnabled(): Unit = { val raftCluster = cluster.asInstanceOf[RaftClusterInstance] val admin = cluster.createAdminClient() @@ -248,13 +279,19 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { assertNotEquals(oldMemberId, consumerGroupHeartbeatResponse.data.memberId) } - @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "5000"), - new ClusterConfigProperty(key = "group.consumer.min.session.timeout.ms", value = "5000") - )) + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "5000"), + new ClusterConfigProperty(key = "group.consumer.min.session.timeout.ms", value = "5000") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testStaticMemberRemovedAfterSessionTimeoutExpiryWhenNewGroupCoordinatorIsEnabled(): Unit = { val raftCluster = cluster.asInstanceOf[RaftClusterInstance] val admin = cluster.createAdminClient() diff --git a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala index 969e3e9a0d..a93cb68be5 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala @@ -17,13 +17,14 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.message.ListGroupsResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.coordinator.group.Group import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Tag import org.junit.jupiter.api.Timeout @@ -34,11 +35,16 @@ import org.junit.jupiter.api.extension.ExtendWith @ClusterTestDefaults(types = Array(Type.KRAFT)) @Tag("integration") class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testUpgradeFromEmptyClassicToConsumerGroup(): Unit = { // Creates the __consumer_offsets topics because it won't be created automatically // in this test because it does not use FindCoordinator API. @@ -103,11 +109,16 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord ) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testDowngradeFromEmptyConsumerToClassicGroup(): Unit = { // Creates the __consumer_offsets topics because it won't be created automatically // in this test because it does not use FindCoordinator API. @@ -165,11 +176,16 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord ) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testUpgradeFromSimpleGroupToConsumerGroup(): Unit = { // Creates the __consumer_offsets topics because it won't be created automatically // in this test because it does not use FindCoordinator API. diff --git a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala index 10b94c882c..49bbee71a4 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala @@ -17,11 +17,12 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.coordinator.group.classic.ClassicGroupState +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.{assertEquals, fail} import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -31,31 +32,44 @@ import org.junit.jupiter.api.extension.ExtendWith @ClusterTestDefaults(types = Array(Type.KRAFT)) @Tag("integration") class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testDeleteGroupsWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testDeleteGroups(true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testDeleteGroupsWithOldConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testDeleteGroups(false) } - @ClusterTest(types = Array(Type.ZK, Type.KRAFT, Type.CO_KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + types = Array(Type.ZK, Type.KRAFT, Type.CO_KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) def testDeleteGroupsWithOldConsumerGroupProtocolAndOldGroupCoordinator(): Unit = { testDeleteGroups(false) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 52b55ffc33..bd0ded29de 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -79,7 +79,7 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1} -import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} +import org.apache.kafka.server.common.{FinalizedFeatures, GroupVersion, MetadataVersion} import org.apache.kafka.server.config._ import org.apache.kafka.server.metrics.ClientMetricsTestUtils import org.apache.kafka.server.util.{FutureUtils, MockTime} @@ -7020,6 +7020,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupHeartbeatRequest(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest, true).build()) @@ -7029,9 +7039,10 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, consumerGroupHeartbeatRequest )).thenReturn(future) - kafkaApis = createKafkaApis(overrideProperties = Map( - GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" - )) + kafkaApis = createKafkaApis( + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true + ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) val consumerGroupHeartbeatResponse = new ConsumerGroupHeartbeatResponseData() @@ -7044,6 +7055,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupHeartbeatRequestFutureFailed(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest, true).build()) @@ -7053,9 +7074,10 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, consumerGroupHeartbeatRequest )).thenReturn(future) - kafkaApis = createKafkaApis(overrideProperties = Map( - GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" - )) + kafkaApis = createKafkaApis( + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true + ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) future.completeExceptionally(Errors.FENCED_MEMBER_EPOCH.exception) @@ -7065,6 +7087,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupHeartbeatRequestAuthorizationFailed(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest, true).build()) @@ -7074,7 +7106,8 @@ class KafkaApisTest extends Logging { .thenReturn(Seq(AuthorizationResult.DENIED).asJava) kafkaApis = createKafkaApis( authorizer = Some(authorizer), - overrideProperties = Map(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true") + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) @@ -7084,6 +7117,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupDescribe(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val groupIds = List("group-id-0", "group-id-1", "group-id-2").asJava val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() consumerGroupDescribeRequestData.groupIds.addAll(groupIds) @@ -7095,7 +7138,8 @@ class KafkaApisTest extends Logging { any[util.List[String]] )).thenReturn(future) kafkaApis = createKafkaApis( - overrideProperties = Map(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true") + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) @@ -7134,6 +7178,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupDescribeAuthorizationFailed(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() consumerGroupDescribeRequestData.groupIds.add("group-id") val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) @@ -7150,7 +7204,8 @@ class KafkaApisTest extends Logging { future.complete(List().asJava) kafkaApis = createKafkaApis( authorizer = Some(authorizer), - overrideProperties = Map(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true") + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) @@ -7160,6 +7215,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupDescribeFutureFailed(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() consumerGroupDescribeRequestData.groupIds.add("group-id") val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) @@ -7169,9 +7234,10 @@ class KafkaApisTest extends Logging { any[RequestContext], any[util.List[String]] )).thenReturn(future) - kafkaApis = createKafkaApis(overrideProperties = Map( - GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" - )) + kafkaApis = createKafkaApis( + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true + ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) future.completeExceptionally(Errors.FENCED_MEMBER_EPOCH.exception) diff --git a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala index b99239497e..3fcffa893a 100644 --- a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala @@ -17,13 +17,14 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.message.ListGroupsResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.Group +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.{assertEquals, fail} import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -33,13 +34,18 @@ import org.junit.jupiter.api.extension.ExtendWith @ClusterTestDefaults(types = Array(Type.KRAFT)) @Tag("integration") class ListGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testListGroupsWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testListGroups(true) } diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala index 7791c6c942..696e6534bf 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala @@ -17,9 +17,10 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.fail import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -30,22 +31,29 @@ import org.junit.jupiter.api.extension.ExtendWith @Tag("integration") class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testOffsetCommitWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testOffsetCommit(true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) def testOffsetCommitWithOldConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testOffsetCommit(false) } diff --git a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala index 8fabac4bda..22c1e28548 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala @@ -17,9 +17,10 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.fail import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -29,13 +30,18 @@ import org.junit.jupiter.api.extension.ExtendWith @ClusterTestDefaults(types = Array(Type.KRAFT)) @Tag("integration") class OffsetDeleteRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testOffsetDeleteWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testOffsetDelete(true) } diff --git a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala index 5f51d6cdb1..2e640bfea6 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala @@ -18,18 +18,16 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions - import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message.OffsetFetchResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} - +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.{assertEquals, fail} import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith - import scala.jdk.CollectionConverters._ @Timeout(120) @@ -38,24 +36,31 @@ import scala.jdk.CollectionConverters._ @Tag("integration") class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testSingleGroupOffsetFetchWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testSingleGroupOffsetFetch(useNewProtocol = true, requireStable = true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) def testSingleGroupOffsetFetchWithOldConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testSingleGroupOffsetFetch(useNewProtocol = false, requireStable = false) } @@ -71,13 +76,18 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB testSingleGroupOffsetFetch(useNewProtocol = false, requireStable = true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testSingleGroupAllOffsetFetchWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testSingleGroupAllOffsetFetch(useNewProtocol = true, requireStable = true) } @@ -104,13 +114,18 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB testSingleGroupAllOffsetFetch(useNewProtocol = false, requireStable = true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testMultiGroupsOffsetFetchWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testMultipleGroupsOffsetFetch(useNewProtocol = true, requireStable = true) } diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 8b4bb93a66..03623bab41 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -299,7 +299,7 @@ class ReplicationQuotasTest extends QuorumTestHarness { features.add(new BrokerRegistrationRequestData.Feature() .setName(MetadataVersion.FEATURE_NAME) .setMinSupportedVersion(MetadataVersion.IBP_3_0_IV1.featureLevel()) - .setMaxSupportedVersion(MetadataVersion.IBP_3_8_IV0.featureLevel())) + .setMaxSupportedVersion(MetadataVersion.IBP_4_0_IVO.featureLevel())) controllerServer.controller.registerBroker( ControllerRequestContextUtil.ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData() diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index fb9361fc3a..54a436f231 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -376,10 +376,18 @@ Found problem: true ) - val featureLevel = Features.TEST_VERSION.defaultValue(metadataVersion) - if (featureLevel > 0) { - assertEquals(List(generateRecord(TestFeatureVersion.FEATURE_NAME, featureLevel)), records) + val expectedRecords = new ArrayBuffer[ApiMessageAndVersion]() + + def maybeAddRecordFor(features: Features): Unit = { + val featureLevel = features.defaultValue(metadataVersion) + if (featureLevel > 0) { + expectedRecords += generateRecord(features.featureName, featureLevel) + } } + + Features.FEATURES.foreach(maybeAddRecordFor) + + assertEquals(expectedRecords, records) } @Test def testVersionDefaultNoArgs(): Unit = { diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 1b18c9648d..4498778592 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -196,7 +196,7 @@ public void testConfigurationOperations() throws Throwable { ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData(). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_8_IV0)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). setBrokerId(0). setLogDirs(Collections.singletonList(Uuid.fromString("iiaQjkRPQcuMULNII0MUeA"))). setClusterId(logEnv.clusterId())).get(); @@ -240,7 +240,7 @@ public void testDelayedConfigurationOperations() throws Throwable { ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData(). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_8_IV0)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). setBrokerId(0). setLogDirs(Collections.singletonList(Uuid.fromString("sTbzRAMnTpahIyIPNjiLhw"))). setClusterId(logEnv.clusterId())).get(); @@ -298,7 +298,7 @@ public void testFenceMultipleBrokers() throws Throwable { new BrokerRegistrationRequestData(). setBrokerId(brokerId). setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). setIncarnationId(Uuid.randomUuid()). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); @@ -717,7 +717,7 @@ public void testUnregisterBroker() throws Throwable { setBrokerId(0). setClusterId(active.clusterId()). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_8_IV0)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). setListeners(listeners)); assertEquals(5L, reply.get().epoch()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java index ecbefc69da..bdda58d01b 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java @@ -74,7 +74,7 @@ public void testDefaultFeatureMapWithUnstable() { for (Features feature : Features.PRODUCTION_FEATURES) { expectedFeatures.put(feature.featureName(), VersionRange.of( 0, - feature.defaultValue(MetadataVersion.LATEST_PRODUCTION) + feature.defaultValue(MetadataVersion.latestTesting()) )); } assertEquals(expectedFeatures, QuorumFeatures.defaultFeatureMap(true)); diff --git a/server-common/src/main/java/org/apache/kafka/server/common/Features.java b/server-common/src/main/java/org/apache/kafka/server/common/Features.java index a4f11b10ac..eda4b85560 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/Features.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/Features.java @@ -39,7 +39,8 @@ public enum Features { * * See {@link TestFeatureVersion} as an example. See {@link FeatureVersion} when implementing a new feature. */ - TEST_VERSION("test.feature.version", TestFeatureVersion.values()); + TEST_VERSION("test.feature.version", TestFeatureVersion.values()), + GROUP_VERSION("group.version", GroupVersion.values()); public static final Features[] FEATURES; public static final List PRODUCTION_FEATURES; diff --git a/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java new file mode 100644 index 0000000000..002d7ef33f --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java @@ -0,0 +1,62 @@ +/* + * 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.server.common; + +import java.util.Collections; +import java.util.Map; + +public enum GroupVersion implements FeatureVersion { + + // Version 1 enables the consumer rebalance protocol (KIP-848). + GV_1(1, MetadataVersion.IBP_4_0_IVO, Collections.emptyMap()); + + public static final String FEATURE_NAME = "group.version"; + + private final short featureLevel; + private final MetadataVersion bootstrapMetadataVersion; + private final Map dependencies; + + GroupVersion( + int featureLevel, + MetadataVersion bootstrapMetadataVersion, + Map dependencies + ) { + this.featureLevel = (short) featureLevel; + this.bootstrapMetadataVersion = bootstrapMetadataVersion; + this.dependencies = dependencies; + } + + @Override + public short featureLevel() { + return featureLevel; + } + + @Override + public String featureName() { + return FEATURE_NAME; + } + + @Override + public MetadataVersion bootstrapMetadataVersion() { + return bootstrapMetadataVersion; + } + + @Override + public Map dependencies() { + return dependencies; + } +} diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index e6abca3e72..26b67321e4 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -202,7 +202,10 @@ public enum MetadataVersion { IBP_3_7_IV4(19, "3.7", "IV4", false), // Add ELR related supports (KIP-966). - IBP_3_8_IV0(20, "3.8", "IV0", true); + IBP_3_8_IV0(20, "3.8", "IV0", true), + + // Introduce version 1 of the GroupVersion feature (KIP-848). + IBP_4_0_IVO(21, "4.0", "IV0", false); // NOTES when adding a new version: // Update the default version in @ClusterTest annotation to point to the latest version diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index bd9f2594b0..cfcdcf3afe 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -47,6 +47,7 @@ public void testKRaftFeatureLevelsAtAndAfter3_0_IV1() { } @Test + @SuppressWarnings("checkstyle:JavaNCSS") public void testFromVersionString() { assertEquals(IBP_0_8_0, MetadataVersion.fromVersionString("0.8.0")); assertEquals(IBP_0_8_0, MetadataVersion.fromVersionString("0.8.0.0")); @@ -184,6 +185,8 @@ public void testFromVersionString() { assertEquals(IBP_3_7_IV4, MetadataVersion.fromVersionString("3.7-IV4")); assertEquals(IBP_3_8_IV0, MetadataVersion.fromVersionString("3.8-IV0")); + + assertEquals(IBP_4_0_IVO, MetadataVersion.fromVersionString("4.0-IV0")); } @Test @@ -243,6 +246,8 @@ public void testShortVersion() { assertEquals("3.7", IBP_3_7_IV2.shortVersion()); assertEquals("3.7", IBP_3_7_IV3.shortVersion()); assertEquals("3.7", IBP_3_7_IV4.shortVersion()); + assertEquals("3.8", IBP_3_8_IV0.shortVersion()); + assertEquals("4.0", IBP_4_0_IVO.shortVersion()); } @Test @@ -292,6 +297,7 @@ public void testVersion() { assertEquals("3.7-IV3", IBP_3_7_IV3.version()); assertEquals("3.7-IV4", IBP_3_7_IV4.version()); assertEquals("3.8-IV0", IBP_3_8_IV0.version()); + assertEquals("4.0-IV0", IBP_4_0_IVO.version()); } @Test diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index a89782797c..b6d4cea0ed 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -19,7 +19,10 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; + import kafka.test.ClusterInstance; import kafka.test.annotation.ClusterTest; import kafka.test.annotation.Type; @@ -58,9 +61,15 @@ public void testDescribeWithKRaft(ClusterInstance cluster) { String commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe")) ); + + List features = Arrays.stream(commandOutput.split("\n")).sorted().collect(Collectors.toList()); + + assertEquals("Feature: group.version\tSupportedMinVersion: 0\t" + + "SupportedMaxVersion: 1\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(0))); + // Change expected message to reflect latest MetadataVersion (SupportedMaxVersion increases when adding a new version) assertEquals("Feature: metadata.version\tSupportedMinVersion: 3.0-IV1\t" + - "SupportedMaxVersion: 3.8-IV0\tFinalizedVersionLevel: 3.3-IV1\t", outputWithoutEpoch(commandOutput)); + "SupportedMaxVersion: 4.0-IV0\tFinalizedVersionLevel: 3.3-IV1\t", outputWithoutEpoch(features.get(1))); } @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_7_IV4) @@ -68,9 +77,15 @@ public void testDescribeWithKRaftAndBootstrapControllers(ClusterInstance cluster String commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-controller", cluster.bootstrapControllers(), "describe")) ); + + List features = Arrays.stream(commandOutput.split("\n")).sorted().collect(Collectors.toList()); + + assertEquals("Feature: group.version\tSupportedMinVersion: 0\t" + + "SupportedMaxVersion: 1\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(0))); + // Change expected message to reflect latest MetadataVersion (SupportedMaxVersion increases when adding a new version) assertEquals("Feature: metadata.version\tSupportedMinVersion: 3.0-IV1\t" + - "SupportedMaxVersion: 3.8-IV0\tFinalizedVersionLevel: 3.7-IV4\t", outputWithoutEpoch(commandOutput)); + "SupportedMaxVersion: 4.0-IV0\tFinalizedVersionLevel: 3.7-IV4\t", outputWithoutEpoch(features.get(1))); } @ClusterTest(types = {Type.ZK}, metadataVersion = MetadataVersion.IBP_3_3_IV1) @@ -129,7 +144,7 @@ public void testDowngradeMetadataVersionWithKRaft(ClusterInstance cluster) { ); // Change expected message to reflect possible MetadataVersion range 1-N (N increases when adding a new version) assertEquals("Could not disable metadata.version. Invalid update version 0 for feature " + - "metadata.version. Local controller 3000 only supports versions 1-20", commandOutput); + "metadata.version. Local controller 3000 only supports versions 1-21", commandOutput); commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(1, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java index 3f29c159ae..4fbdeec0d9 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java @@ -85,7 +85,10 @@ public Seq generateConfigs() { 0, false ).foreach(props -> { - props.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, isNewGroupCoordinatorEnabled() + ""); + if (isNewGroupCoordinatorEnabled()) { + props.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true"); + props.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer"); + } cfgs.add(KafkaConfig.fromProps(props)); return null; }); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java index dc132b6075..5ea53a1c38 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java @@ -22,6 +22,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.common.Features; +import org.apache.kafka.server.common.GroupVersion; import java.time.Duration; import java.util.ArrayList; @@ -41,6 +43,7 @@ import static kafka.test.annotation.Type.CO_KRAFT; import static kafka.test.annotation.Type.KRAFT; +import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG; import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG; import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG; import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG; @@ -60,9 +63,11 @@ static List forConsumerGroupCoordinator() { serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1"); serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1"); serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true"); + serverProperties.put(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer"); return Collections.singletonList(ClusterConfig.defaultBuilder() .setTypes(Stream.of(KRAFT, CO_KRAFT).collect(Collectors.toSet())) + .setFeatures(Collections.singletonMap(Features.GROUP_VERSION, GroupVersion.GV_1.featureLevel())) .setServerProperties(serverProperties) .setTags(Collections.singletonList("consumerGroupCoordinator")) .build()); From 5257451646acd89d9bc6c19506ce7d823744dafd Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 31 May 2024 21:49:26 +0200 Subject: [PATCH 002/128] KAFKA-16860; [2/2] Introduce group.version feature flag (#16149) This patch updates the system tests to correctly enable the new consumer protocol/coordinator in the tests requiring them. I went with the simplest approach for now. Long term, I think that we should refactor the tests to better handle features and non-production features. I got a successful run of the consumer system tests with this patch combined with https://github.com/apache/kafka/pull/16120: https://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/system-test-kafka-branch-builder--1717155071--dajac--KAFKA-16860-2--29028ae0dd/2024-05-31--001./2024-05-31--001./report.html. Reviewers: Justine Olshan --- .../kafkatest/services/kafka/config_property.py | 3 +++ tests/kafkatest/services/kafka/kafka.py | 16 +++++++++++++++- 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index 335cb02bcc..fe1e3491b6 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -70,6 +70,9 @@ SASL_ENABLED_MECHANISMS="sasl.enabled.mechanisms" NEW_GROUP_COORDINATOR_ENABLE="group.coordinator.new.enable" +GROUP_COORDINATOR_REBALANCE_PROTOCOLS="group.coordinator.rebalance.protocols" + +UNSTABLE_FEATURE_VERSIONS_ENABLE="unstable.feature.versions.enable" """ From KafkaConfig.scala diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 5f245b1005..7425ac551c 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -409,6 +409,12 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.interbroker_sasl_mechanism = interbroker_sasl_mechanism self._security_config = None + # When the new group coordinator is enabled, the new consumer rebalance + # protocol is enabled too. + rebalance_protocols = "classic" + if self.use_new_coordinator: + rebalance_protocols = "classic,consumer" + for node in self.nodes: node_quorum_info = quorum.NodeQuorumInfo(self.quorum_info, node) @@ -422,7 +428,9 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI kraft_broker_configs = { config_property.PORT: config_property.FIRST_BROKER_PORT, config_property.NODE_ID: self.idx(node), - config_property.NEW_GROUP_COORDINATOR_ENABLE: use_new_coordinator + config_property.UNSTABLE_FEATURE_VERSIONS_ENABLE: use_new_coordinator, + config_property.NEW_GROUP_COORDINATOR_ENABLE: use_new_coordinator, + config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS: rebalance_protocols } kraft_broker_plus_zk_configs = kraft_broker_configs.copy() kraft_broker_plus_zk_configs.update(zk_broker_configs) @@ -781,7 +789,9 @@ def prop_file(self, node): override_configs[config_property.ZOOKEEPER_SSL_CLIENT_ENABLE] = 'false' if self.use_new_coordinator: + override_configs[config_property.UNSTABLE_FEATURE_VERSIONS_ENABLE] = 'true' override_configs[config_property.NEW_GROUP_COORDINATOR_ENABLE] = 'true' + override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = 'classic,consumer' for prop in self.server_prop_overrides: override_configs[prop[0]] = prop[1] @@ -884,6 +894,10 @@ def start_node(self, node, timeout_sec=60, **kwargs): # format log directories if necessary kafka_storage_script = self.path.script("kafka-storage.sh", node) cmd = "%s format --ignore-formatted --config %s --cluster-id %s" % (kafka_storage_script, KafkaService.CONFIG_FILE, config_property.CLUSTER_ID) + + if self.use_new_coordinator: + cmd += " -f group.version=1" + self.logger.info("Running log directory format command...\n%s" % cmd) node.account.ssh(cmd) From 92ed1ed58694d1df9df386a286bb856dbefb683a Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 31 May 2024 22:17:59 +0200 Subject: [PATCH 003/128] KAFKA-16864; Optimize uniform (homogenous) assignor (#16088) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch optimizes uniform (homogenous) assignor by avoiding creating a copy of all the assignments. Instead, the assignor creates a copy only if the assignment is updated. It is a sort of copy-on-write. This change reduces the overhead of the TargetAssignmentBuilder when ran with the uniform (homogenous) assignor. Trunk: ``` Benchmark (memberCount) (partitionsToMemberRatio) (topicCount) Mode Cnt Score Error Units TargetAssignmentBuilderBenchmark.build 10000 10 100 avgt 5 24.535 ± 1.583 ms/op TargetAssignmentBuilderBenchmark.build 10000 10 1000 avgt 5 24.094 ± 0.223 ms/op JMH benchmarks done ``` ``` Benchmark (assignmentType) (assignorType) (isRackAware) (memberCount) (partitionsToMemberRatio) (subscriptionType) (topicCount) Mode Cnt Score Error Units ServerSideAssignorBenchmark.doAssignment INCREMENTAL UNIFORM false 10000 10 HOMOGENEOUS 100 avgt 5 14.697 ± 0.133 ms/op ServerSideAssignorBenchmark.doAssignment INCREMENTAL UNIFORM false 10000 10 HOMOGENEOUS 1000 avgt 5 15.073 ± 0.135 ms/op JMH benchmarks done ``` Patch: ``` Benchmark (memberCount) (partitionsToMemberRatio) (topicCount) Mode Cnt Score Error Units TargetAssignmentBuilderBenchmark.build 10000 10 100 avgt 5 3.376 ± 0.577 ms/op TargetAssignmentBuilderBenchmark.build 10000 10 1000 avgt 5 3.731 ± 0.359 ms/op JMH benchmarks done ``` ``` Benchmark (assignmentType) (assignorType) (isRackAware) (memberCount) (partitionsToMemberRatio) (subscriptionType) (topicCount) Mode Cnt Score Error Units ServerSideAssignorBenchmark.doAssignment INCREMENTAL UNIFORM false 10000 10 HOMOGENEOUS 100 avgt 5 1.975 ± 0.086 ms/op ServerSideAssignorBenchmark.doAssignment INCREMENTAL UNIFORM false 10000 10 HOMOGENEOUS 1000 avgt 5 2.026 ± 0.190 ms/op JMH benchmarks done ``` Reviewers: Ritika Reddy , Jeff Kim , Justine Olshan --- .../OptimizedUniformAssignmentBuilder.java | 370 +++++++----------- .../group/assignor/UniformAssignor.java | 10 +- .../coordinator/group/AssignmentTestUtil.java | 17 +- .../group/CoordinatorRecordHelpersTest.java | 18 +- ...OptimizedUniformAssignmentBuilderTest.java | 123 +++--- .../assignor/ServerSideAssignorBenchmark.java | 2 +- .../TargetAssignmentBuilderBenchmark.java | 5 +- 7 files changed, 223 insertions(+), 322 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java index 3ea1361d69..34e8256652 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java @@ -18,29 +18,18 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.server.common.TopicIdPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Queue; import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static java.lang.Math.min; /** - * The optimized uniform assignment builder is used to generate the target assignment for a consumer group with + * The homogenous uniform assignment builder is used to generate the target assignment for a consumer group with * all its members subscribed to the same set of topics. - * It is optimized since the assignment can be done in fewer, less complicated steps compared to when - * the subscriptions are different across the members. * * Assignments are done according to the following principles: * @@ -53,8 +42,17 @@ * The assignment builder prioritizes the properties in the following order: * Balance > Stickiness. */ -public class OptimizedUniformAssignmentBuilder extends AbstractUniformAssignmentBuilder { - private static final Logger LOG = LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class); +public class OptimizedUniformAssignmentBuilder { + private static final Class UNMODIFIABLE_MAP_CLASS = Collections.unmodifiableMap(new HashMap<>()).getClass(); + private static final Class EMPTY_MAP_CLASS = Collections.emptyMap().getClass(); + + /** + * @return True if the provided map is an UnmodifiableMap or EmptyMap. Those classes are not + * public hence we cannot use the `instanceof` operator. + */ + private static boolean isImmutableMap(Map map) { + return UNMODIFIABLE_MAP_CLASS.isInstance(map) || EMPTY_MAP_CLASS.isInstance(map); + } /** * The assignment specification which includes member metadata. @@ -72,62 +70,53 @@ public class OptimizedUniformAssignmentBuilder extends AbstractUniformAssignment private final Set subscribedTopicIds; /** - * The number of members to receive an extra partition beyond the minimum quota. - * Minimum Quota = Total Partitions / Total Members - * Example: If there are 11 partitions to be distributed among 3 members, - * each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 3) members get an extra partition. + * The members that are below their quota. */ - private int remainingMembersToGetAnExtraPartition; - - /** - * Members mapped to the remaining number of partitions needed to meet the minimum quota. - * Minimum quota = total partitions / total members. - */ - private Map potentiallyUnfilledMembers; + private final List unfilledMembers; /** * The partitions that still need to be assigned. * Initially this contains all the subscribed topics' partitions. */ - private final Set unassignedPartitions; + private final List unassignedPartitions; /** * The target assignment. */ private final Map targetAssignment; + /** + * The minimum number of partitions that a member must have. + * Minimum quota = total partitions / total members. + */ + private int minimumMemberQuota; + + /** + * The number of members to receive an extra partition beyond the minimum quota. + * Example: If there are 11 partitions to be distributed among 3 members, + * each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 3) members get an extra partition. + */ + private int remainingMembersToGetAnExtraPartition; + OptimizedUniformAssignmentBuilder(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) { this.groupSpec = groupSpec; this.subscribedTopicDescriber = subscribedTopicDescriber; this.subscribedTopicIds = new HashSet<>(groupSpec.members().values().iterator().next().subscribedTopicIds()); - this.potentiallyUnfilledMembers = new HashMap<>(); - this.unassignedPartitions = new HashSet<>(); + this.unfilledMembers = new ArrayList<>(); + this.unassignedPartitions = new ArrayList<>(); this.targetAssignment = new HashMap<>(); } /** - * Here's the step-by-step breakdown of the assignment process: - * - *
  • Compute the quotas of partitions for each member based on the total partitions and member count.
  • - *
  • Initialize unassigned partitions with all the topic partitions that aren't present in the - * current target assignment.
  • - *
  • For existing assignments, retain partitions based on the determined quota. Add extras to unassigned partitions.
  • - *
  • Identify members that haven't fulfilled their partition quota or are eligible to receive extra partitions.
  • - *
  • Proceed with a round-robin assignment according to quotas. - * For each unassigned partition, locate the first compatible member from the potentially unfilled list.
  • + * Compute the new assignment for the group. */ - @Override - protected GroupAssignment buildAssignment() throws PartitionAssignorException { - int totalPartitionsCount = 0; - + public GroupAssignment build() throws PartitionAssignorException { if (subscribedTopicIds.isEmpty()) { - LOG.debug("The subscription list is empty, returning an empty assignment"); return new GroupAssignment(Collections.emptyMap()); } - // Check if the subscribed topicId is still valid. - // Update unassigned partitions based on the current target assignment - // and topic metadata. + // Compute the list of unassigned partitions. + int totalPartitionsCount = 0; for (Uuid topicId : subscribedTopicIds) { int partitionCount = subscribedTopicDescriber.numPartitions(topicId); if (partitionCount == -1) { @@ -144,216 +133,149 @@ protected GroupAssignment buildAssignment() throws PartitionAssignorException { } } - // The minimum required quota that each member needs to meet for a balanced assignment. - // This is the same for all members. - final int numberOfMembers = groupSpec.members().size(); - final int minQuota = totalPartitionsCount / numberOfMembers; + // Compute the minimum required quota per member and the number of members + // that should receive an extra partition. + int numberOfMembers = groupSpec.members().size(); + minimumMemberQuota = totalPartitionsCount / numberOfMembers; remainingMembersToGetAnExtraPartition = totalPartitionsCount % numberOfMembers; - groupSpec.members().keySet().forEach(memberId -> - targetAssignment.put(memberId, new MemberAssignment(new HashMap<>()) - )); - - potentiallyUnfilledMembers = assignStickyPartitions(minQuota); + // Revoke the partitions that either are not part of the member's subscriptions or + // exceed the maximum quota assigned to each member. + maybeRevokePartitions(); - unassignedPartitionsRoundRobinAssignment(); - - if (!unassignedPartitions.isEmpty()) { - throw new PartitionAssignorException("Partitions were left unassigned"); - } + // Assign the unassigned partitions to the members with space. + assignRemainingPartitions(); return new GroupAssignment(targetAssignment); } /** - * Retains a set of partitions from the existing assignment and includes them in the target assignment. - * Only relevant partitions that exist in the current topic metadata and subscriptions are considered. + * Revoke the partitions that either are not part of the member's subscriptions or + * exceed the maximum quota assigned to each member. * - *

    For each member: - *

      - *
    1. Find the valid current assignment considering topic subscriptions and metadata
    2. - *
    3. If the current assignment exists, retain partitions up to the minimum quota.
    4. - *
    5. If the current assignment size is greater than the minimum quota and - * there are members that could get an extra partition, assign the next partition as well.
    6. - *
    7. Finally, if the member's current assignment size is less than the minimum quota, - * add them to the potentially unfilled members map and track the number of remaining - * partitions required to meet the quota.
    8. - *
    - *

    - * - * @return Members mapped to the remaining number of partitions needed to meet the minimum quota, - * including members that are eligible to receive an extra partition. + * This method ensures that the original assignment is not copied if it is not + * altered. */ - private Map assignStickyPartitions(int minQuota) { - Map potentiallyUnfilledMembers = new HashMap<>(); - - groupSpec.members().forEach((memberId, assignmentMemberSpec) -> { - List validCurrentMemberAssignment = validCurrentMemberAssignment( - assignmentMemberSpec.assignedPartitions() - ); - - int currentAssignmentSize = validCurrentMemberAssignment.size(); - // Number of partitions required to meet the minimum quota. - int remaining = minQuota - currentAssignmentSize; - - if (currentAssignmentSize > 0) { - int retainedPartitionsCount = min(currentAssignmentSize, minQuota); - IntStream.range(0, retainedPartitionsCount).forEach(i -> { - TopicIdPartition topicIdPartition = validCurrentMemberAssignment.get(i); - addPartitionToAssignment( - targetAssignment, - memberId, - topicIdPartition.topicId(), - topicIdPartition.partitionId() - ); - }); - - if (remaining < 0) { - // The extra partition is located at the last index from the previous step. - if (remainingMembersToGetAnExtraPartition > 0) { - TopicIdPartition topicIdPartition = validCurrentMemberAssignment.get(retainedPartitionsCount++); - addPartitionToAssignment( - targetAssignment, - memberId, - topicIdPartition.topicId(), - topicIdPartition.partitionId() - ); - remainingMembersToGetAnExtraPartition--; + private void maybeRevokePartitions() { + for (Map.Entry entry : groupSpec.members().entrySet()) { + String memberId = entry.getKey(); + AssignmentMemberSpec assignmentMemberSpec = entry.getValue(); + Map> oldAssignment = assignmentMemberSpec.assignedPartitions(); + Map> newAssignment = null; + + // The assignor expects to receive the assignment as an immutable map. It leverages + // this knowledge in order to avoid having to copy all assignments. + if (!isImmutableMap(oldAssignment)) { + throw new IllegalStateException("The assignor expect an immutable map."); + } + + int quota = minimumMemberQuota; + if (remainingMembersToGetAnExtraPartition > 0) { + quota++; + remainingMembersToGetAnExtraPartition--; + } + + for (Map.Entry> topicPartitions : oldAssignment.entrySet()) { + Uuid topicId = topicPartitions.getKey(); + Set partitions = topicPartitions.getValue(); + + if (subscribedTopicIds.contains(topicId)) { + if (partitions.size() <= quota) { + quota -= partitions.size(); + } else { + for (Integer partition : partitions) { + if (quota > 0) { + quota--; + } else { + if (newAssignment == null) { + // If the new assignment is null, we create a deep copy of the + // original assignment so that we can alter it. + newAssignment = deepCopy(oldAssignment); + } + // Remove the partition from the new assignment. + Set parts = newAssignment.get(topicId); + parts.remove(partition); + if (parts.isEmpty()) { + newAssignment.remove(topicId); + } + // Add the partition to the unassigned set to be re-assigned later on. + unassignedPartitions.add(new TopicIdPartition(topicId, partition)); + } + } } - // Any previously owned partitions that weren't retained due to the quotas - // are added to the unassigned partitions set. - if (retainedPartitionsCount < currentAssignmentSize) { - unassignedPartitions.addAll(validCurrentMemberAssignment.subList( - retainedPartitionsCount, - currentAssignmentSize - )); + } else { + if (newAssignment == null) { + // If the new assignment is null, we create a deep copy of the + // original assignment so that we can alter it. + newAssignment = deepCopy(oldAssignment); } + // Remove the entire topic. + newAssignment.remove(topicId); } } - if (remaining >= 0) { - potentiallyUnfilledMembers.put(memberId, remaining); + if (quota > 0) { + unfilledMembers.add(new MemberWithRemainingQuota(memberId, quota)); } - }); - return potentiallyUnfilledMembers; - } - - /** - * Filters the current assignment of partitions for a given member based on certain criteria. - * - * Any partition that still belongs to the member's subscribed topics list is considered valid. - * - * @param currentMemberAssignment The map of topics to partitions currently assigned to the member. - * - * @return List of valid partitions after applying the filters. - */ - private List validCurrentMemberAssignment( - Map> currentMemberAssignment - ) { - List validCurrentAssignmentList = new ArrayList<>(); - currentMemberAssignment.forEach((topicId, partitions) -> { - if (subscribedTopicIds.contains(topicId)) { - partitions.forEach(partition -> { - TopicIdPartition topicIdPartition = new TopicIdPartition(topicId, partition); - validCurrentAssignmentList.add(topicIdPartition); - }); + if (newAssignment == null) { + targetAssignment.put(memberId, new MemberAssignment(oldAssignment)); } else { - LOG.debug("The topic " + topicId + " is no longer present in the subscribed topics list"); + targetAssignment.put(memberId, new MemberAssignment(newAssignment)); } - }); - - return validCurrentAssignmentList; + } } /** - * Allocates the unassigned partitions to unfilled members in a round-robin fashion. + * Assign the unassigned partitions to the unfilled members. */ - private void unassignedPartitionsRoundRobinAssignment() { - Queue roundRobinMembers = new LinkedList<>(potentiallyUnfilledMembers.keySet()); - - // Partitions are sorted to ensure an even topic wise distribution across members. - // This not only balances the load but also makes partition-to-member mapping more predictable. - List sortedPartitionsList = unassignedPartitions.stream() - .sorted(Comparator.comparing(TopicIdPartition::topicId).thenComparing(TopicIdPartition::partitionId)) - .collect(Collectors.toList()); - - for (TopicIdPartition topicIdPartition : sortedPartitionsList) { - boolean assigned = false; - - for (int i = 0; i < roundRobinMembers.size() && !assigned; i++) { - String memberId = roundRobinMembers.poll(); - if (potentiallyUnfilledMembers.containsKey(memberId)) { - assigned = maybeAssignPartitionToMember(memberId, topicIdPartition); - } - // Only re-add the member to the end of the queue if it's still available for assignment. - if (potentiallyUnfilledMembers.containsKey(memberId)) { - roundRobinMembers.add(memberId); - } + private void assignRemainingPartitions() { + int unassignedPartitionIndex = 0; + + for (MemberWithRemainingQuota unfilledMember : unfilledMembers) { + String memberId = unfilledMember.memberId; + int remainingQuota = unfilledMember.remainingQuota; + + Map> newAssignment = targetAssignment.get(memberId).targetPartitions(); + if (isImmutableMap(newAssignment)) { + // If the new assignment is immutable, we must create a deep copy of it + // before altering it. + newAssignment = deepCopy(newAssignment); + targetAssignment.put(memberId, new MemberAssignment(newAssignment)); } - if (assigned) { - unassignedPartitions.remove(topicIdPartition); + for (int i = 0; i < remainingQuota && unassignedPartitionIndex < unassignedPartitions.size(); i++) { + TopicIdPartition unassignedTopicIdPartition = unassignedPartitions.get(unassignedPartitionIndex); + unassignedPartitionIndex++; + newAssignment + .computeIfAbsent(unassignedTopicIdPartition.topicId(), __ -> new HashSet<>()) + .add(unassignedTopicIdPartition.partitionId()); } } - } - /** - * Assigns the specified partition to the given member and updates the potentially unfilled members map. - * Only assign extra partitions once the member has met its minimum quota = total partitions / total members. - * - *
      - *
    1. If the minimum quota hasn't been met aka remaining > 0 directly assign the partition. - * After assigning the partition, if the min quota has been met aka remaining = 0, remove the member - * if there's no members left to receive an extra partition. Otherwise, keep it in the - * potentially unfilled map.
    2. - *
    3. If the minimum quota has been met and if there is potential to receive an extra partition, assign it. - * Remove the member from the potentially unfilled map since it has already received the extra partition - * and met the min quota.
    4. - *
    5. Else, don't assign the partition.
    6. - *
    - * - * @param memberId The Id of the member to which the partition will be assigned. - * @param topicIdPartition The topicIdPartition to be assigned. - * @return true if the assignment was successful, false otherwise. - */ - private boolean maybeAssignPartitionToMember(String memberId, TopicIdPartition topicIdPartition) { - int remaining = potentiallyUnfilledMembers.get(memberId); - boolean shouldAssign = false; - - // If the member hasn't met the minimum quota, set the flag for assignment. - // If member has met minimum quota and there's an extra partition available, set the flag for assignment. - if (remaining > 0) { - potentiallyUnfilledMembers.put(memberId, --remaining); - shouldAssign = true; - - // If the member meets the minimum quota due to this assignment, - // check if any extra partitions are available. - // Removing the member from the list reduces an iteration for when remaining = 0 but there's no extras left. - if (remaining == 0 && remainingMembersToGetAnExtraPartition == 0) { - potentiallyUnfilledMembers.remove(memberId); - } - } else if (remaining == 0 && remainingMembersToGetAnExtraPartition > 0) { - remainingMembersToGetAnExtraPartition--; - // Each member can only receive one extra partition, once they meet the minimum quota and receive an extra - // partition they can be removed from the potentially unfilled members map. - potentiallyUnfilledMembers.remove(memberId); - shouldAssign = true; + if (unassignedPartitionIndex < unassignedPartitions.size()) { + throw new PartitionAssignorException("Partitions were left unassigned"); } + } - // Assign the partition if flag is set. - if (shouldAssign) { - addPartitionToAssignment( - targetAssignment, - memberId, - topicIdPartition.topicId(), - topicIdPartition.partitionId() - ); - return true; + private static Map> deepCopy(Map> map) { + Map> copy = new HashMap<>(map.size()); + for (Map.Entry> entry : map.entrySet()) { + copy.put(entry.getKey(), new HashSet<>(entry.getValue())); } + return copy; + } + + private static class MemberWithRemainingQuota { + final String memberId; + final int remainingQuota; - // No assignment possible because the member met the minimum quota but - // number of members to receive an extra partition is zero. - return false; + MemberWithRemainingQuota( + String memberId, + int remainingQuota + ) { + this.memberId = memberId; + this.remainingQuota = remainingQuota; + } } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java index 7da7c2d8c8..648b0161d6 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java @@ -66,21 +66,19 @@ public GroupAssignment assign( GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber ) throws PartitionAssignorException { - AbstractUniformAssignmentBuilder assignmentBuilder; - if (groupSpec.members().isEmpty()) return new GroupAssignment(Collections.emptyMap()); if (groupSpec.subscriptionType().equals(HOMOGENEOUS)) { LOG.debug("Detected that all members are subscribed to the same set of topics, invoking the " + "optimized assignment algorithm"); - assignmentBuilder = new OptimizedUniformAssignmentBuilder(groupSpec, subscribedTopicDescriber); + return new OptimizedUniformAssignmentBuilder(groupSpec, subscribedTopicDescriber) + .build(); } else { LOG.debug("Detected that the members are subscribed to different sets of topics, invoking the " + "general assignment algorithm"); - assignmentBuilder = new GeneralUniformAssignmentBuilder(groupSpec, subscribedTopicDescriber); + return new GeneralUniformAssignmentBuilder(groupSpec, subscribedTopicDescriber) + .buildAssignment(); } - - return assignmentBuilder.buildAssignment(); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java index 74bb303abd..ffc5455cea 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java @@ -22,12 +22,13 @@ import java.util.AbstractMap; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; -import java.util.TreeSet; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -42,13 +43,13 @@ public static Map.Entry> mkTopicAssignment( ); } - public static Map.Entry> mkSortedTopicAssignment( + public static Map.Entry> mkOrderedTopicAssignment( Uuid topicId, Integer... partitions ) { return new AbstractMap.SimpleEntry<>( topicId, - new TreeSet<>(Arrays.asList(partitions)) + new LinkedHashSet<>(Arrays.asList(partitions)) ); } @@ -56,18 +57,18 @@ public static Map.Entry> mkSortedTopicAssignment( public static Map> mkAssignment(Map.Entry>... entries) { Map> assignment = new HashMap<>(); for (Map.Entry> entry : entries) { - assignment.put(entry.getKey(), entry.getValue()); + assignment.put(entry.getKey(), Collections.unmodifiableSet(entry.getValue())); } - return assignment; + return Collections.unmodifiableMap(assignment); } @SafeVarargs - public static Map> mkSortedAssignment(Map.Entry>... entries) { + public static Map> mkOrderedAssignment(Map.Entry>... entries) { Map> assignment = new LinkedHashMap<>(); for (Map.Entry> entry : entries) { - assignment.put(entry.getKey(), entry.getValue()); + assignment.put(entry.getKey(), Collections.unmodifiableSet(entry.getValue())); } - return assignment; + return Collections.unmodifiableMap(assignment); } /** diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpersTest.java index 77bad7a483..900ba31839 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpersTest.java @@ -69,8 +69,8 @@ import java.util.stream.Stream; import static org.apache.kafka.coordinator.group.Assertions.assertRecordEquals; -import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkSortedAssignment; -import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkSortedTopicAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkOrderedAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkOrderedTopicAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpers.newCurrentAssignmentRecord; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord; @@ -297,7 +297,7 @@ public void testNewTargetAssignmentRecord() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); - Map> partitions = mkSortedAssignment( + Map> partitions = mkOrderedAssignment( mkTopicAssignment(topicId1, 11, 12, 13), mkTopicAssignment(topicId2, 21, 22, 23) ); @@ -379,14 +379,14 @@ public void testNewCurrentAssignmentRecord() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); - Map> assigned = mkSortedAssignment( - mkSortedTopicAssignment(topicId1, 11, 12, 13), - mkSortedTopicAssignment(topicId2, 21, 22, 23) + Map> assigned = mkOrderedAssignment( + mkOrderedTopicAssignment(topicId1, 11, 12, 13), + mkOrderedTopicAssignment(topicId2, 21, 22, 23) ); - Map> revoking = mkSortedAssignment( - mkSortedTopicAssignment(topicId1, 14, 15, 16), - mkSortedTopicAssignment(topicId2, 24, 25, 26) + Map> revoking = mkOrderedAssignment( + mkOrderedTopicAssignment(topicId1, 14, 15, 16), + mkOrderedTopicAssignment(topicId2, 24, 25, 26) ); CoordinatorRecord expectedRecord = new CoordinatorRecord( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java index f21bd63735..fdc4f5941f 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java @@ -34,6 +34,7 @@ import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.assertAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkOrderedAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.invertedTargetAssignment; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpersTest.mkMapOfPartitionRacks; @@ -158,12 +159,11 @@ public void testFirstAssignmentTwoMembersTwoTopicsNoMemberRacks() { Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic3Uuid, 1) + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic3Uuid, 0, 1) )); expectedAssignment.put(memberB, mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic3Uuid, 0) + mkTopicAssignment(topic1Uuid, 1, 2) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -295,30 +295,25 @@ public void testReassignmentForTwoMembersTwoTopicsGivenUnbalancedPrevAssignment( )); Map members = new TreeMap<>(); - Map> currentAssignmentForA = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1), - mkTopicAssignment(topic2Uuid, 0, 1) - ) - ); + members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForA + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 0, 1), + mkTopicAssignment(topic2Uuid, 0, 1) + ) )); - Map> currentAssignmentForB = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 2), - mkTopicAssignment(topic2Uuid, 2) - ) - ); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 2), + mkTopicAssignment(topic2Uuid, 2) + ) )); Map>> expectedAssignment = new HashMap<>(); @@ -366,40 +361,34 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembersTwoTopics() { Map members = new TreeMap<>(); - Map> currentAssignmentForA = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic2Uuid, 0) - ) - ); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForA + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 0, 2), + mkTopicAssignment(topic2Uuid, 0) + ) )); - Map> currentAssignmentForB = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1, 2) - ) - ); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1, 2) + ) )); Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2, 3, 5), - mkTopicAssignment(topic2Uuid, 0, 4) + mkTopicAssignment(topic1Uuid, 0, 2, 3), + mkTopicAssignment(topic2Uuid, 0, 3, 4) )); expectedAssignment.put(memberB, mkAssignment( - mkTopicAssignment(topic1Uuid, 1, 4), - mkTopicAssignment(topic2Uuid, 1, 2, 3) + mkTopicAssignment(topic1Uuid, 1, 4, 5), + mkTopicAssignment(topic2Uuid, 1, 2) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -436,26 +425,24 @@ public void testReassignmentWhenOneMemberAddedAfterInitialAssignmentWithTwoMembe Map members = new HashMap<>(); - Map> currentAssignmentForA = new TreeMap<>(mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic2Uuid, 0) - )); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForA + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 0, 2), + mkTopicAssignment(topic2Uuid, 0) + ) )); - Map> currentAssignmentForB = new TreeMap<>(mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1, 2) - )); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1, 2) + ) )); // Add a new member to trigger a re-assignment. @@ -512,38 +499,36 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithThreeM Map members = new HashMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0), - mkTopicAssignment(topic2Uuid, 0) - ); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForA + mkAssignment( + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic2Uuid, 0) + ) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1) - ); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + mkAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1) + ) )); // Member C was removed Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic2Uuid, 0) + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic2Uuid, 0, 2) )); expectedAssignment.put(memberB, mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1, 2) + mkTopicAssignment(topic1Uuid, 1, 2), + mkTopicAssignment(topic2Uuid, 1) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -581,26 +566,24 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith // Initial subscriptions were [T1, T2] Map members = new HashMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0), - mkTopicAssignment(topic2Uuid, 0) - ); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), Collections.singleton(topic2Uuid), - currentAssignmentForA + mkAssignment( + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic2Uuid, 0) + ) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1) - ); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), Collections.singleton(topic2Uuid), - currentAssignmentForB + mkAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1) + ) )); Map>> expectedAssignment = new HashMap<>(); diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java index 2349350e4b..77a38ab7f4 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java @@ -268,7 +268,7 @@ private void simulateIncrementalRebalance() { assignmentMemberSpec.instanceId(), assignmentMemberSpec.rackId(), assignmentMemberSpec.subscribedTopicIds(), - memberAssignment.targetPartitions() + Collections.unmodifiableMap(memberAssignment.targetPartitions()) )); }); diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java index 511db01c86..7d67f07d3e 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java @@ -184,10 +184,7 @@ private Map generateMockInitialTargetAssignmentAndUpdateInve for (Map.Entry entry : groupAssignment.members().entrySet()) { String memberId = entry.getKey(); Map> topicPartitions = entry.getValue().targetPartitions(); - - Assignment assignment = new Assignment(topicPartitions); - - initialTargetAssignment.put(memberId, assignment); + initialTargetAssignment.put(memberId, new Assignment(topicPartitions)); } return initialTargetAssignment; From a39f3ec815868e0d89b6c9d0dfd1b1b4bba08905 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?TingI=C4=81u=20=22Ting=22=20K=C3=AC?= <51072200+frankvicky@users.noreply.github.com> Date: Sat, 1 Jun 2024 04:14:15 +0800 Subject: [PATCH 004/128] KAFKA-16639 Ensure HeartbeatRequestManager generates leave request regardless of in-flight heartbeats. (#16017) Fix the bug where the heartbeat is not sent when a newly created consumer is immediately closed. When there is a heartbeat request in flight and the consumer is then closed. In the current code, the HeartbeatRequestManager does not correctly send the closing heartbeat because a previous heartbeat request is still in flight. However, the closing heartbeat is only sent once, so in this situation, the broker will not know that the consumer has left the consumer group until the consumer's heartbeat times out. This situation causes the broker to wait until the consumer's heartbeat times out before triggering a consumer group rebalance, which in turn affects message consumption. Reviewers: Lianet Magrans , Chia-Ping Tsai --- .../internals/HeartbeatRequestManager.java | 7 ++++++- .../HeartbeatRequestManagerTest.java | 21 ++++++++++++++++++- 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index a956ef3a93..d31d412c65 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -45,6 +45,7 @@ import java.util.TreeSet; import java.util.stream.Collectors; + /** *

    Manages the request creation and response handling for the heartbeat. The module creates a * {@link ConsumerGroupHeartbeatRequest} using the state stored in the {@link MembershipManager} and enqueue it to @@ -208,7 +209,11 @@ public NetworkClientDelegate.PollResult poll(long currentTimeMs) { return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(leaveHeartbeat)); } - boolean heartbeatNow = membershipManager.shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight(); + // Case 1: The member is leaving + boolean heartbeatNow = membershipManager.state() == MemberState.LEAVING || + // Case 2: The member state indicates it should send a heartbeat without waiting for the interval, and there is no heartbeat request currently in-flight + (membershipManager.shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight()); + if (!heartbeatRequestState.canSendRequest(currentTimeMs) && !heartbeatNow) { return new NetworkClientDelegate.PollResult(heartbeatRequestState.timeToNextHeartbeatMs(currentTimeMs)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 8334fb2360..f63dd55754 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -277,7 +277,7 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + "previous one is in-flight"); - + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent when the " + @@ -752,6 +752,25 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) + public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short version) { + mockStableMember(); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a previous one is in-flight"); + + membershipManager.leaveGroup(); + + ConsumerGroupHeartbeatRequest heartbeatToLeave = getHeartbeatRequest(heartbeatRequestManager, version); + assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, heartbeatToLeave.data().memberEpoch()); + + NetworkClientDelegate.PollResult pollAgain = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, pollAgain.unsentRequests.size()); + } + private void assertHeartbeat(HeartbeatRequestManager hrm, int nextPollMs) { NetworkClientDelegate.PollResult pollResult = hrm.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); From 34f5d5bab26c59fef325a72725c05d7563b38f44 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Sat, 1 Jun 2024 15:51:39 -0700 Subject: [PATCH 005/128] KAFKA-16757: Fix broker re-registration issues around MV 3.7-IV2 (#15945) When upgrading from a MetadataVersion older than 3.7-IV2, we need to resend the broker registration, so that the controller can record the storage directories. The current code for doing this has several problems, however. One is that it tends to trigger even in cases where we don't actually need it. Another is that when re-registering the broker, the broker is marked as fenced. This PR moves the handling of the re-registration case out of BrokerMetadataPublisher and into BrokerRegistrationTracker. The re-registration code there will only trigger in the case where the broker sees an existing registration for itself with no directories set. This is much more targetted than the original code. Additionally, in ClusterControlManager, when re-registering the same broker, we now preserve its fencing and shutdown state, rather than clearing those. (There isn't any good reason re-registering the same broker should clear these things... this was purely an oversight.) Note that we can tell the broker is "the same" because it has the same IncarnationId. Reviewers: Gaurav Narula , Igor Soarez --- .../kafka/server/BrokerLifecycleManager.scala | 6 +- .../scala/kafka/server/BrokerServer.scala | 8 +- .../metadata/BrokerMetadataPublisher.scala | 16 -- .../server/BrokerLifecycleManagerTest.scala | 2 +- .../BrokerMetadataPublisherTest.scala | 101 +----------- .../controller/ClusterControlManager.java | 7 + .../publisher/BrokerRegistrationTracker.java | 136 ++++++++++++++++ .../BrokerRegistrationTrackerTest.java | 151 ++++++++++++++++++ 8 files changed, 306 insertions(+), 121 deletions(-) create mode 100644 metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java create mode 100644 metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 5f3fdc8188..51bc16fb09 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -264,11 +264,11 @@ class BrokerLifecycleManager( new OfflineDirBrokerFailureEvent(directory)) } - def handleKraftJBODMetadataVersionUpdate(): Unit = { - eventQueue.append(new KraftJBODMetadataVersionUpdateEvent()) + def resendBrokerRegistrationUnlessZkMode(): Unit = { + eventQueue.append(new ResendBrokerRegistrationUnlessZkModeEvent()) } - private class KraftJBODMetadataVersionUpdateEvent extends EventQueue.Event { + private class ResendBrokerRegistrationUnlessZkModeEvent extends EventQueue.Event { override def run(): Unit = { if (!isZkBroker) { registered = false diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 112a03c50a..64a4fd7474 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -37,7 +37,7 @@ import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{ClusterResource, TopicPartition, Uuid} import org.apache.kafka.coordinator.group.metrics.{GroupCoordinatorMetrics, GroupCoordinatorRuntimeMetrics} import org.apache.kafka.coordinator.group.{CoordinatorRecord, GroupCoordinator, GroupCoordinatorConfig, GroupCoordinatorService, CoordinatorRecordSerde} -import org.apache.kafka.image.publisher.MetadataPublisher +import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher} import org.apache.kafka.metadata.{BrokerState, ListenerInfo, VersionRange} import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.{AssignmentsManager, ClientMetricsManager, NodeToControllerChannelManager} @@ -139,6 +139,8 @@ class BrokerServer( var brokerMetadataPublisher: BrokerMetadataPublisher = _ + var brokerRegistrationTracker: BrokerRegistrationTracker = _ + val brokerFeatures: BrokerFeatures = BrokerFeatures.createDefault(config.unstableFeatureVersionsEnabled) def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE @@ -482,6 +484,10 @@ class BrokerServer( lifecycleManager ) metadataPublishers.add(brokerMetadataPublisher) + brokerRegistrationTracker = new BrokerRegistrationTracker(config.brokerId, + logManager.directoryIdsSet.toList.asJava, + () => lifecycleManager.resendBrokerRegistrationUnlessZkMode()) + metadataPublishers.add(brokerRegistrationTracker) // Register parts of the broker that can be reconfigured via dynamic configs. This needs to // be done before we publish the dynamic configs, so that we don't miss anything. diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 048a665757..ee7bfa2157 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -29,7 +29,6 @@ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} -import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.fault.FaultHandler import java.util.concurrent.CompletableFuture @@ -129,21 +128,6 @@ class BrokerMetadataPublisher( debug(s"Publishing metadata at offset $highestOffsetAndEpoch with $metadataVersionLogMsg.") } - Option(delta.featuresDelta()).foreach { featuresDelta => - featuresDelta.metadataVersionChange().ifPresent{ metadataVersion => - info(s"Updating metadata.version to ${metadataVersion.featureLevel()} at offset $highestOffsetAndEpoch.") - val currentMetadataVersion = delta.image().features().metadataVersion() - if (currentMetadataVersion.isLessThan(MetadataVersion.IBP_3_7_IV2) && metadataVersion.isAtLeast(MetadataVersion.IBP_3_7_IV2)) { - info( - s"""Resending BrokerRegistration with existing incarnation-id to inform the - |controller about log directories in the broker following metadata update: - |previousMetadataVersion: ${delta.image().features().metadataVersion()} - |newMetadataVersion: $metadataVersion""".stripMargin.linesIterator.mkString(" ").trim) - brokerLifecycleManager.handleKraftJBODMetadataVersionUpdate() - } - } - } - // Apply topic deltas. Option(delta.topicsDelta()).foreach { topicsDelta => try { diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index 34f9d139a0..b0162dc635 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -285,7 +285,7 @@ class BrokerLifecycleManagerTest { assertEquals(1000L, manager.brokerEpoch) // Trigger JBOD MV update - manager.handleKraftJBODMetadataVersionUpdate() + manager.resendBrokerRegistrationUnlessZkMode() // Accept new registration, response sets epoch to 1200 nextRegistrationRequest(1200L) diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index c2926c3b67..26f4fb3dae 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -30,7 +30,6 @@ import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, NewTopic} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.BROKER -import org.apache.kafka.common.metadata.FeatureLevelRecord import org.apache.kafka.common.utils.Exit import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataImageTest, MetadataProvenance} @@ -43,7 +42,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any import org.mockito.Mockito -import org.mockito.Mockito.{clearInvocations, doThrow, mock, times, verify, verifyNoInteractions} +import org.mockito.Mockito.{doThrow, mock, verify} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -221,102 +220,4 @@ class BrokerMetadataPublisherTest { verify(groupCoordinator).onNewMetadataImage(image, delta) } - - @Test - def testMetadataVersionUpdateToIBP_3_7_IV2OrAboveTriggersBrokerReRegistration(): Unit = { - val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, "")) - val metadataCache = new KRaftMetadataCache(0) - val logManager = mock(classOf[LogManager]) - val replicaManager = mock(classOf[ReplicaManager]) - val groupCoordinator = mock(classOf[GroupCoordinator]) - val faultHandler = mock(classOf[FaultHandler]) - val brokerLifecycleManager = mock(classOf[BrokerLifecycleManager]) - - val metadataPublisher = new BrokerMetadataPublisher( - config, - metadataCache, - logManager, - replicaManager, - groupCoordinator, - mock(classOf[TransactionCoordinator]), - mock(classOf[DynamicConfigPublisher]), - mock(classOf[DynamicClientQuotaPublisher]), - mock(classOf[ScramPublisher]), - mock(classOf[DelegationTokenPublisher]), - mock(classOf[AclPublisher]), - faultHandler, - faultHandler, - brokerLifecycleManager, - ) - - var image = MetadataImage.EMPTY - var delta = new MetadataDelta.Builder() - .setImage(image) - .build() - - // We first upgrade metadata version to 3_6_IV2 - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_6_IV2.featureLevel())) - var newImage = delta.apply(new MetadataProvenance(100, 4, 2000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This should NOT trigger broker reregistration - verifyNoInteractions(brokerLifecycleManager) - - // We then upgrade to IBP_3_7_IV2 - image = newImage - delta = new MetadataDelta.Builder() - .setImage(image) - .build() - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_7_IV2.featureLevel())) - newImage = delta.apply(new MetadataProvenance(100, 4, 2000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This SHOULD trigger a broker registration - verify(brokerLifecycleManager, times(1)).handleKraftJBODMetadataVersionUpdate() - clearInvocations(brokerLifecycleManager) - - // Finally upgrade to IBP_3_8_IV0 - image = newImage - delta = new MetadataDelta.Builder() - .setImage(image) - .build() - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_8_IV0.featureLevel())) - newImage = delta.apply(new MetadataProvenance(200, 4, 3000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This should NOT trigger broker reregistration - verify(brokerLifecycleManager, times(0)).handleKraftJBODMetadataVersionUpdate() - - metadataPublisher.close() - } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 0974c31d1b..8b9c5b19ea 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -408,6 +408,13 @@ public ControllerResult registerBroker( setBrokerEpoch(brokerEpoch). setRack(request.rack()). setEndPoints(listenerInfo.toBrokerRegistrationRecord()); + + if (existing != null && request.incarnationId().equals(existing.incarnationId())) { + log.info("Amending registration of broker {}", request.brokerId()); + record.setFenced(existing.fenced()); + record.setInControlledShutdown(existing.inControlledShutdown()); + } + for (BrokerRegistrationRequestData.Feature feature : request.features()) { record.features().add(processRegistrationFeature(brokerId, finalizedFeatures, feature)); } diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java b/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java new file mode 100644 index 0000000000..51ac2bdfa4 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java @@ -0,0 +1,136 @@ +/* + * 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.image.publisher; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.loader.LoaderManifest; +import org.apache.kafka.metadata.BrokerRegistration; +import org.apache.kafka.server.common.MetadataVersion; +import org.slf4j.Logger; + +import java.util.List; + +/** + * Tracks the registration of a specific broker, and executes a callback if it should be refreshed. + * + * This tracker handles cases where we might want to re-register the broker. The only such case + * right now is during the transition from non-JBOD mode, to JBOD mode. In other words, the + * transition from a MetadataVersion less than 3.7-IV2, to one greater than or equal to 3.7-IV2. + * In this case, the broker registration will start out containing no directories, and we need to + * resend the BrokerRegistrationRequest to fix that. + * + * As much as possible, the goal here is to keep things simple. We just compare the desired state + * with the actual state, and try to make changes only if necessary. + */ +public class BrokerRegistrationTracker implements MetadataPublisher { + private final Logger log; + private final int id; + private final Runnable refreshRegistrationCallback; + + /** + * Create the tracker. + * + * @param id The ID of this broker. + * @param targetDirectories The directories managed by this broker. + * @param refreshRegistrationCallback Callback to run if we need to refresh the registration. + */ + public BrokerRegistrationTracker( + int id, + List targetDirectories, + Runnable refreshRegistrationCallback + ) { + this.log = new LogContext("[BrokerRegistrationTracker id=" + id + "] "). + logger(BrokerRegistrationTracker.class); + this.id = id; + this.refreshRegistrationCallback = refreshRegistrationCallback; + } + + @Override + public String name() { + return "BrokerRegistrationTracker(id=" + id + ")"; + } + + @Override + public void onMetadataUpdate( + MetadataDelta delta, + MetadataImage newImage, + LoaderManifest manifest + ) { + boolean checkBrokerRegistration = false; + if (delta.featuresDelta() != null) { + if (delta.metadataVersionChanged().isPresent()) { + if (log.isTraceEnabled()) { + log.trace("Metadata version change is present: {}", + delta.metadataVersionChanged()); + } + checkBrokerRegistration = true; + } + } + if (delta.clusterDelta() != null) { + if (delta.clusterDelta().changedBrokers().get(id) != null) { + if (log.isTraceEnabled()) { + log.trace("Broker change is present: {}", + delta.clusterDelta().changedBrokers().get(id)); + } + checkBrokerRegistration = true; + } + } + if (checkBrokerRegistration) { + if (brokerRegistrationNeedsRefresh(newImage.features().metadataVersion(), + delta.clusterDelta().broker(id))) { + refreshRegistrationCallback.run(); + } + } + } + + /** + * Check if the current broker registration needs to be refreshed. + * + * @param metadataVersion The current metadata version. + * @param registration The current broker registration, or null if there is none. + * @return True only if we should refresh. + */ + boolean brokerRegistrationNeedsRefresh( + MetadataVersion metadataVersion, + BrokerRegistration registration + ) { + // If there is no existing registration, the BrokerLifecycleManager must still be sending it. + // So we don't need to do anything yet. + if (registration == null) { + log.debug("No current broker registration to check."); + return false; + } + // Check to see if the directory list has changed. Note that this check could certainly be + // triggered spuriously. For example, if the broker's directory list has been changed in the + // past, and we are in the process of replaying that change log, we will end up here. + // That's fine because resending the broker registration does not cause any problems. And, + // of course, as soon as a snapshot is made, we will no longer need to worry about those + // old metadata log entries being replayed on startup. + if (metadataVersion.isAtLeast(MetadataVersion.IBP_3_7_IV2) && + registration.directories().isEmpty()) { + log.info("Current directory set is empty, but MV supports JBOD. Resending " + + "broker registration."); + return true; + } + log.debug("Broker registration does not need to be resent."); + return false; + } +} diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java new file mode 100644 index 0000000000..855a96cd8a --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java @@ -0,0 +1,151 @@ +/* + * 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.image.publisher; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.loader.LogDeltaManifest; +import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.server.common.MetadataVersion; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +@Timeout(value = 40) +public class BrokerRegistrationTrackerTest { + static final Uuid INCARNATION_ID = Uuid.fromString("jyjLbk31Tpa53pFrU9Y-Ng"); + + static final Uuid A = Uuid.fromString("Ahw3vXfnThqeZbb7HD1w6Q"); + + static final Uuid B = Uuid.fromString("BjOacT0OTNqIvUWIlKhahg"); + + static final Uuid C = Uuid.fromString("CVHi_iv2Rvy5_1rtPdasfg"); + + static class BrokerRegistrationTrackerTestContext { + AtomicInteger numCalls = new AtomicInteger(0); + BrokerRegistrationTracker tracker = new BrokerRegistrationTracker(1, + Arrays.asList(B, A), () -> numCalls.incrementAndGet()); + + MetadataImage image = MetadataImage.EMPTY; + + void onMetadataUpdate(MetadataDelta delta) { + MetadataProvenance provenance = new MetadataProvenance(0, 0, 0); + image = delta.apply(provenance); + LogDeltaManifest manifest = new LogDeltaManifest.Builder(). + provenance(provenance). + leaderAndEpoch(LeaderAndEpoch.UNKNOWN). + numBatches(1). + elapsedNs(1). + numBytes(1). + build(); + tracker.onMetadataUpdate(delta, image, manifest); + } + + MetadataDelta newDelta() { + return new MetadataDelta.Builder(). + setImage(image). + build(); + } + } + + @Test + public void testTrackerName() { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + assertEquals("BrokerRegistrationTracker(id=1)", ctx.tracker.name()); + } + + @Test + public void testMetadataVersionUpdateWithoutRegistrationDoesNothing() { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + MetadataDelta delta = ctx.newDelta(); + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(MetadataVersion.IBP_3_7_IV2.featureLevel())); + ctx.onMetadataUpdate(delta); + assertEquals(0, ctx.numCalls.get()); + } + + @Test + public void testBrokerUpdateWithoutNewMvDoesNothing() { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + MetadataDelta delta = ctx.newDelta(); + delta.replay(new RegisterBrokerRecord(). + setBrokerId(1). + setIncarnationId(INCARNATION_ID). + setLogDirs(Arrays.asList(A, B, C))); + ctx.onMetadataUpdate(delta); + assertEquals(0, ctx.numCalls.get()); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testBrokerUpdateWithNewMv(boolean jbodMv) { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + MetadataDelta delta = ctx.newDelta(); + delta.replay(new RegisterBrokerRecord(). + setBrokerId(1). + setIncarnationId(INCARNATION_ID). + setLogDirs(Arrays.asList())); + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(jbodMv ? MetadataVersion.IBP_3_7_IV2.featureLevel() : + MetadataVersion.IBP_3_7_IV1.featureLevel())); + ctx.onMetadataUpdate(delta); + if (jbodMv) { + assertEquals(1, ctx.numCalls.get()); + } else { + assertEquals(0, ctx.numCalls.get()); + } + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testBrokerUpdateWithNewMvWithTwoDeltas(boolean jbodMv) { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + MetadataDelta delta = ctx.newDelta(); + delta.replay(new RegisterBrokerRecord(). + setBrokerId(1). + setIncarnationId(INCARNATION_ID). + setLogDirs(Arrays.asList())); + ctx.onMetadataUpdate(delta); + // No calls are made because MetadataVersion is 3.0-IV1 initially + assertEquals(0, ctx.numCalls.get()); + + delta = ctx.newDelta(); + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(jbodMv ? MetadataVersion.IBP_3_7_IV2.featureLevel() : + MetadataVersion.IBP_3_7_IV1.featureLevel())); + ctx.onMetadataUpdate(delta); + if (jbodMv) { + assertEquals(1, ctx.numCalls.get()); + } else { + assertEquals(0, ctx.numCalls.get()); + } + } +} From dc5a22bf830b57ad0688fd8d544d631523e1fa26 Mon Sep 17 00:00:00 2001 From: Ken Huang <100591800+m1a2st@users.noreply.github.com> Date: Sun, 2 Jun 2024 18:33:02 +0900 Subject: [PATCH 006/128] KAFKA-16807 DescribeLogDirsResponseData#results#topics have unexpected topics having empty partitions (#16042) Reviewers: Chia-Ping Tsai --- .../scala/kafka/server/ReplicaManager.scala | 25 +++++++------- .../kafka/server/ReplicaManagerTest.scala | 33 +++++++++++++++++++ 2 files changed, 47 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index aa56269a2f..a2a070bcd0 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -33,6 +33,7 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult +import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaderAndIsrResponseData.{LeaderAndIsrPartitionError, LeaderAndIsrTopicError} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic @@ -67,7 +68,7 @@ import java.util import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.Lock import java.util.concurrent.{CompletableFuture, Future, RejectedExecutionException, TimeUnit} -import java.util.{Optional, OptionalInt, OptionalLong} +import java.util.{Collections, Optional, OptionalInt, OptionalLong} import scala.collection.{Map, Seq, Set, mutable} import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ @@ -1249,9 +1250,9 @@ class ReplicaManager(val config: KafkaConfig, val fileStore = Files.getFileStore(file) val totalBytes = adjustForLargeFileSystems(fileStore.getTotalSpace) val usableBytes = adjustForLargeFileSystems(fileStore.getUsableSpace) - logsByDir.get(absolutePath) match { + val topicInfos = logsByDir.get(absolutePath) match { case Some(logs) => - val topicInfos = logs.groupBy(_.topicPartition.topic).map{case (topic, logs) => + logs.groupBy(_.topicPartition.topic).map { case (topic, logs) => new DescribeLogDirsResponseData.DescribeLogDirsTopic().setName(topic).setPartitions( logs.filter { log => partitions.contains(log.topicPartition) @@ -1262,17 +1263,19 @@ class ReplicaManager(val config: KafkaConfig, .setOffsetLag(getLogEndOffsetLag(log.topicPartition, log.logEndOffset, log.isFuture)) .setIsFutureKey(log.isFuture) }.toList.asJava) - }.toList.asJava - - new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath) - .setErrorCode(Errors.NONE.code).setTopics(topicInfos) - .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + }.filterNot(_.partitions().isEmpty).toList.asJava case None => - new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath) - .setErrorCode(Errors.NONE.code) - .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + Collections.emptyList[DescribeLogDirsTopic]() } + val describeLogDirsResult = new DescribeLogDirsResponseData.DescribeLogDirsResult() + .setLogDir(absolutePath).setTopics(topicInfos) + .setErrorCode(Errors.NONE.code) + .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + if (!topicInfos.isEmpty) + describeLogDirsResult.setTopics(topicInfos) + describeLogDirsResult + } catch { case e: KafkaStorageException => warn("Unable to describe replica dirs for %s".format(absolutePath), e) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 6b655ea783..97ba10d8be 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -6450,6 +6450,39 @@ class ReplicaManagerTest { assertEquals(Errors.NONE.code, response.errorCode) assertTrue(response.totalBytes > 0) assertTrue(response.usableBytes >= 0) + assertFalse(response.topics().isEmpty) + response.topics().forEach(t => assertFalse(t.partitions().isEmpty)) + } + } finally { + replicaManager.shutdown(checkpointHW = false) + } + } + + @Test + def testDescribeLogDirsWithoutAnyPartitionTopic(): Unit = { + val noneTopic = "none-topic" + val topicPartition = 0 + val topicId = Uuid.randomUuid() + val followerBrokerId = 0 + val leaderBrokerId = 1 + val leaderEpoch = 1 + val leaderEpochIncrement = 2 + val countDownLatch = new CountDownLatch(1) + val offsetFromLeader = 5 + + // Prepare the mocked components for the test + val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager(new MockTimer(time), + topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, + expectTruncation = false, localLogOffset = Some(10), offsetFromLeader = offsetFromLeader, topicId = Some(topicId)) + + try { + val responses = replicaManager.describeLogDirs(Set(new TopicPartition(noneTopic, topicPartition))) + assertEquals(mockLogMgr.liveLogDirs.size, responses.size) + responses.foreach { response => + assertEquals(Errors.NONE.code, response.errorCode) + assertTrue(response.totalBytes > 0) + assertTrue(response.usableBytes >= 0) + assertTrue(response.topics().isEmpty) } } finally { replicaManager.shutdown(checkpointHW = false) From 495ec16fb23310ccf863beecb32cbf0687114af2 Mon Sep 17 00:00:00 2001 From: Ken Huang <100591800+m1a2st@users.noreply.github.com> Date: Tue, 4 Jun 2024 02:34:58 +0900 Subject: [PATCH 007/128] KAFKA-16881: InitialState type leaks into the Connect REST API OpenAPI spec (#16175) Reviewers: Chris Egerton --- .../connect/runtime/rest/entities/CreateConnectorRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java index da8e235e42..9d338936db 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java @@ -47,7 +47,7 @@ public Map config() { return config; } - @JsonProperty + @JsonProperty("initial_state") public InitialState initialState() { return initialState; } From 1b11cf0fe3514fb4c3642516012898740f4000d4 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 3 Jun 2024 20:32:39 +0200 Subject: [PATCH 008/128] MINOR: Small refactor in TargetAssignmentBuilder (#16174) This patch is a small refactoring which mainly aims at avoid to construct a copy of the new target assignment in the TargetAssignmentBuilder because the copy is not used by the caller. The change relies on the exiting tests and it does not really have an impact on performance (e.g. validated with TargetAssignmentBuilderBenchmark). Reviewers: Chia-Ping Tsai --- .../group/GroupMetadataManager.java | 27 ++++++---- .../consumer/TargetAssignmentBuilder.java | 30 ++++------- .../consumer/TargetAssignmentBuilderTest.java | 50 +++++++++---------- 3 files changed, 50 insertions(+), 57 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index b912cb6ac3..ef1abb4e83 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -59,6 +59,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.MemberAssignment; import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; import org.apache.kafka.coordinator.group.assignor.SubscriptionType; import org.apache.kafka.coordinator.group.consumer.Assignment; @@ -1904,24 +1905,28 @@ private Assignment updateTargetAssignment( .withInvertedTargetAssignment(group.invertedTargetAssignment()) .withTopicsImage(metadataImage.topics()) .addOrUpdateMember(updatedMember.memberId(), updatedMember); - TargetAssignmentBuilder.TargetAssignmentResult assignmentResult; - // A new static member is replacing an older one with the same subscriptions. - // We just need to remove the older member and add the newer one. The new member should - // reuse the target assignment of the older member. + if (staticMemberReplaced) { - assignmentResult = assignmentResultBuilder - .removeMember(member.memberId()) - .build(); - } else { - assignmentResult = assignmentResultBuilder - .build(); + // A new static member is replacing an older one with the same subscriptions. + // We just need to remove the older member and add the newer one. The new member should + // reuse the target assignment of the older member. + assignmentResultBuilder.removeMember(member.memberId()); } + TargetAssignmentBuilder.TargetAssignmentResult assignmentResult = + assignmentResultBuilder.build(); + log.info("[GroupId {}] Computed a new target assignment for epoch {} with '{}' assignor: {}.", group.groupId(), groupEpoch, preferredServerAssignor, assignmentResult.targetAssignment()); records.addAll(assignmentResult.records()); - return assignmentResult.targetAssignment().get(updatedMember.memberId()); + + MemberAssignment newMemberAssignment = assignmentResult.targetAssignment().get(updatedMember.memberId()); + if (newMemberAssignment != null) { + return new Assignment(newMemberAssignment.targetPartitions()); + } else { + return Assignment.EMPTY; + } } catch (PartitionAssignorException ex) { String msg = String.format("Failed to compute a new target assignment for epoch %d: %s", groupEpoch, ex.getMessage()); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java index 57d6039fa0..daea9938bf 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java @@ -64,11 +64,11 @@ public static class TargetAssignmentResult { /** * The new target assignment for the group. */ - private final Map targetAssignment; + private final Map targetAssignment; TargetAssignmentResult( List records, - Map targetAssignment + Map targetAssignment ) { Objects.requireNonNull(records); Objects.requireNonNull(targetAssignment); @@ -86,7 +86,7 @@ public List records() { /** * @return The target assignment. */ - public Map targetAssignment() { + public Map targetAssignment() { return targetAssignment; } } @@ -347,38 +347,26 @@ public TargetAssignmentResult build() throws PartitionAssignorException { // Compute delta from previous to new target assignment and create the // relevant records. List records = new ArrayList<>(); - Map newTargetAssignment = new HashMap<>(); - memberSpecs.keySet().forEach(memberId -> { + for (String memberId : memberSpecs.keySet()) { Assignment oldMemberAssignment = targetAssignment.get(memberId); Assignment newMemberAssignment = newMemberAssignment(newGroupAssignment, memberId); - newTargetAssignment.put(memberId, newMemberAssignment); - - if (oldMemberAssignment == null) { - // If the member had no assignment, we always create a record for it. + if (!newMemberAssignment.equals(oldMemberAssignment)) { + // If the member had no assignment or had a different assignment, we + // create a record for the new assignment. records.add(newTargetAssignmentRecord( groupId, memberId, newMemberAssignment.partitions() )); - } else { - // If the member had an assignment, we only create a record if the - // new assignment is different. - if (!newMemberAssignment.equals(oldMemberAssignment)) { - records.add(newTargetAssignmentRecord( - groupId, - memberId, - newMemberAssignment.partitions() - )); - } } - }); + } // Bump the target assignment epoch. records.add(newTargetAssignmentEpochRecord(groupId, groupEpoch)); - return new TargetAssignmentResult(records, newTargetAssignment); + return new TargetAssignmentResult(records, newGroupAssignment.members()); } private Assignment newMemberAssignment( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java index d5ba038f31..e2e572b6bf 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java @@ -337,12 +337,12 @@ public void testAssignmentHasNotChanged() { 20 )), result.records()); - Map expectedAssignment = new HashMap<>(); - expectedAssignment.put("member-1", new Assignment(mkAssignment( + Map expectedAssignment = new HashMap<>(); + expectedAssignment.put("member-1", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1, 2, 3), mkTopicAssignment(barTopicId, 1, 2, 3) ))); - expectedAssignment.put("member-2", new Assignment(mkAssignment( + expectedAssignment.put("member-2", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5, 6), mkTopicAssignment(barTopicId, 4, 5, 6) ))); @@ -400,12 +400,12 @@ public void testAssignmentSwapped() { 20 ), result.records().get(2)); - Map expectedAssignment = new HashMap<>(); - expectedAssignment.put("member-2", new Assignment(mkAssignment( + Map expectedAssignment = new HashMap<>(); + expectedAssignment.put("member-2", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1, 2, 3), mkTopicAssignment(barTopicId, 1, 2, 3) ))); - expectedAssignment.put("member-1", new Assignment(mkAssignment( + expectedAssignment.put("member-1", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5, 6), mkTopicAssignment(barTopicId, 4, 5, 6) ))); @@ -474,16 +474,16 @@ public void testNewMember() { 20 ), result.records().get(3)); - Map expectedAssignment = new HashMap<>(); - expectedAssignment.put("member-1", new Assignment(mkAssignment( + Map expectedAssignment = new HashMap<>(); + expectedAssignment.put("member-1", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1, 2), mkTopicAssignment(barTopicId, 1, 2) ))); - expectedAssignment.put("member-2", new Assignment(mkAssignment( + expectedAssignment.put("member-2", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 3, 4), mkTopicAssignment(barTopicId, 3, 4) ))); - expectedAssignment.put("member-3", new Assignment(mkAssignment( + expectedAssignment.put("member-3", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 5, 6), mkTopicAssignment(barTopicId, 5, 6) ))); @@ -561,16 +561,16 @@ public void testUpdateMember() { 20 ), result.records().get(3)); - Map expectedAssignment = new HashMap<>(); - expectedAssignment.put("member-1", new Assignment(mkAssignment( + Map expectedAssignment = new HashMap<>(); + expectedAssignment.put("member-1", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1, 2), mkTopicAssignment(barTopicId, 1, 2) ))); - expectedAssignment.put("member-2", new Assignment(mkAssignment( + expectedAssignment.put("member-2", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 3, 4), mkTopicAssignment(barTopicId, 3, 4) ))); - expectedAssignment.put("member-3", new Assignment(mkAssignment( + expectedAssignment.put("member-3", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 5, 6), mkTopicAssignment(barTopicId, 5, 6) ))); @@ -639,16 +639,16 @@ public void testPartialAssignmentUpdate() { 20 ), result.records().get(2)); - Map expectedAssignment = new HashMap<>(); - expectedAssignment.put("member-1", new Assignment(mkAssignment( + Map expectedAssignment = new HashMap<>(); + expectedAssignment.put("member-1", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1, 2), mkTopicAssignment(barTopicId, 1, 2) ))); - expectedAssignment.put("member-2", new Assignment(mkAssignment( + expectedAssignment.put("member-2", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 3, 4, 5), mkTopicAssignment(barTopicId, 3, 4, 5) ))); - expectedAssignment.put("member-3", new Assignment(mkAssignment( + expectedAssignment.put("member-3", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 6), mkTopicAssignment(barTopicId, 6) ))); @@ -713,12 +713,12 @@ public void testDeleteMember() { 20 ), result.records().get(2)); - Map expectedAssignment = new HashMap<>(); - expectedAssignment.put("member-1", new Assignment(mkAssignment( + Map expectedAssignment = new HashMap<>(); + expectedAssignment.put("member-1", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1, 2, 3), mkTopicAssignment(barTopicId, 1, 2, 3) ))); - expectedAssignment.put("member-2", new Assignment(mkAssignment( + expectedAssignment.put("member-2", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5, 6), mkTopicAssignment(barTopicId, 4, 5, 6) ))); @@ -788,17 +788,17 @@ public void testReplaceStaticMember() { 20 ), result.records().get(1)); - Map expectedAssignment = new HashMap<>(); - expectedAssignment.put("member-1", new Assignment(mkAssignment( + Map expectedAssignment = new HashMap<>(); + expectedAssignment.put("member-1", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1, 2), mkTopicAssignment(barTopicId, 1, 2) ))); - expectedAssignment.put("member-2", new Assignment(mkAssignment( + expectedAssignment.put("member-2", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 3, 4), mkTopicAssignment(barTopicId, 3, 4) ))); - expectedAssignment.put("member-3-a", new Assignment(mkAssignment( + expectedAssignment.put("member-3-a", new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 5, 6), mkTopicAssignment(barTopicId, 5, 6) ))); From 9c72048a883c0bc21911998741fff29ccd2a4260 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?TingI=C4=81u=20=22Ting=22=20K=C3=AC?= <51072200+frankvicky@users.noreply.github.com> Date: Tue, 4 Jun 2024 02:36:07 +0800 Subject: [PATCH 009/128] KAFKA-16861: Don't convert to group to classic if the size is larger than group max size. (#16163) Fix the bug where the group downgrade to a classic one when a member leaves, even though the consumer group size is still larger than `classicGroupMaxSize`. Reviewers: Chia-Ping Tsai , David Jacot --- .../group/GroupMetadataManager.java | 1 + .../group/GroupMetadataManagerTest.java | 43 +++++++++++++++++++ 2 files changed, 44 insertions(+) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index ef1abb4e83..a1295397a8 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -809,6 +809,7 @@ private boolean validateOnlineDowngrade(ConsumerGroup consumerGroup, String memb } else if (consumerGroup.numMembers() - 1 > classicGroupMaxSize) { log.info("Cannot downgrade consumer group {} to classic group because its group size is greater than classic group max size.", consumerGroup.groupId()); + return false; } return true; } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 3664a7a61d..abf48fd641 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -13166,6 +13166,49 @@ public void testClassicGroupLeaveToConsumerGroupWithoutValidLeaveGroupMember() { assertEquals(Collections.emptyList(), leaveResult.records()); } + @Test + public void testNoConversionWhenSizeExceedsClassicMaxGroupSize() throws Exception { + String groupId = "group-id"; + String nonClassicMemberId = "1"; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(new byte[0]) + ); + + ConsumerGroupMember member = new ConsumerGroupMember.Builder(nonClassicMemberId).build(); + ConsumerGroupMember classicMember1 = new ConsumerGroupMember.Builder("2") + .setClassicMemberMetadata(new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata().setSupportedProtocols(protocols)) + .build(); + ConsumerGroupMember classicMember2 = new ConsumerGroupMember.Builder("3") + .setClassicMemberMetadata(new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata().setSupportedProtocols(protocols)) + .build(); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withClassicGroupMaxSize(1) + .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) + .withConsumerGroup( + new ConsumerGroupBuilder(groupId, 10) + .withMember(member) + .withMember(classicMember1) + .withMember(classicMember2) + ) + .build(); + + assertEquals(Group.GroupType.CONSUMER, context.groupMetadataManager.group(groupId).type()); + + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(nonClassicMemberId) + .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) + .setRebalanceTimeoutMs(5000) + ); + + assertEquals(Group.GroupType.CONSUMER, context.groupMetadataManager.group(groupId).type()); + } + private static void checkJoinGroupResponse( JoinGroupResponseData expectedResponse, JoinGroupResponseData actualResponse, From cd52f33746de68c497843e4cd94416d5ec7270ee Mon Sep 17 00:00:00 2001 From: Anatoly Popov Date: Mon, 3 Jun 2024 23:46:40 +0300 Subject: [PATCH 010/128] KAFKA-16105: Reset read offsets when seeking to beginning in TBRLMM (#15165) Reviewers: Greg Harris , Luke Chen , Kamal Chandraprakash --- .../remote/metadata/storage/ConsumerTask.java | 15 +++-- .../metadata/storage/ConsumerTaskTest.java | 62 ++++++++++++++++++- 2 files changed, 71 insertions(+), 6 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java index f36909b66b..a328b256ee 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java @@ -238,10 +238,15 @@ void maybeWaitForPartitionAssignments() throws InterruptedException { this.assignedMetadataPartitions = Collections.unmodifiableSet(metadataPartitionSnapshot); // for newly assigned user-partitions, read from the beginning of the corresponding metadata partition final Set seekToBeginOffsetPartitions = assignedUserTopicIdPartitionsSnapshot - .stream() - .filter(utp -> !utp.isAssigned) - .map(utp -> toRemoteLogPartition(utp.metadataPartition)) - .collect(Collectors.toSet()); + .stream() + .filter(utp -> !utp.isAssigned) + .map(utp -> utp.metadataPartition) + // When reset to beginning is happening, we also need to reset the last read offset + // Otherwise if the next reassignment request for the same metadata partition comes in + // before the record of already assigned topic has been read, then the reset will happen again to the last read offset + .peek(readOffsetsByMetadataPartition::remove) + .map(ConsumerTask::toRemoteLogPartition) + .collect(Collectors.toSet()); consumer.seekToBeginning(seekToBeginOffsetPartitions); // for other metadata partitions, read from the offset where the processing left last time. remoteLogPartitions.stream() @@ -463,4 +468,4 @@ public String toString() { '}'; } } -} \ No newline at end of file +} diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java index 424c86b6df..cef1d335b9 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.server.log.remote.metadata.storage; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicIdPartition; @@ -64,8 +65,12 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; public class ConsumerTaskTest { @@ -85,7 +90,7 @@ public class ConsumerTaskTest { public void beforeEach() { final Map offsets = remoteLogPartitions.stream() .collect(Collectors.toMap(Function.identity(), e -> 0L)); - consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + consumer = spy(new MockConsumer<>(OffsetResetStrategy.EARLIEST)); consumer.updateBeginningOffsets(offsets); consumerTask = new ConsumerTask(handler, partitioner, consumer, 10L, 300_000L, new SystemTime()); thread = new Thread(consumerTask); @@ -254,6 +259,61 @@ public void testCanProcessRecord() throws InterruptedException { assertEquals(3, handler.metadataCounter); } + @Test + public void testCanReprocessSkippedRecords() throws InterruptedException { + final Uuid topicId = Uuid.fromString("Bp9TDduJRGa9Q5rlvCJOxg"); + final TopicIdPartition tpId0 = new TopicIdPartition(topicId, new TopicPartition("sample", 0)); + final TopicIdPartition tpId1 = new TopicIdPartition(topicId, new TopicPartition("sample", 1)); + final TopicIdPartition tpId3 = new TopicIdPartition(topicId, new TopicPartition("sample", 3)); + assertEquals(partitioner.metadataPartition(tpId0), partitioner.metadataPartition(tpId1)); + assertNotEquals(partitioner.metadataPartition(tpId3), partitioner.metadataPartition(tpId0)); + + final int metadataPartition = partitioner.metadataPartition(tpId0); + final int anotherMetadataPartition = partitioner.metadataPartition(tpId3); + + // Mocking the consumer to be able to wait for the second reassignment + doAnswer(invocation -> { + if (consumerTask.isUserPartitionAssigned(tpId1) && !consumerTask.isUserPartitionAssigned(tpId3)) { + return ConsumerRecords.empty(); + } else { + return invocation.callRealMethod(); + } + }).when(consumer).poll(any()); + + consumer.updateEndOffsets(Collections.singletonMap(toRemoteLogPartition(metadataPartition), 0L)); + consumer.updateEndOffsets(Collections.singletonMap(toRemoteLogPartition(anotherMetadataPartition), 0L)); + final Set assignments = Collections.singleton(tpId0); + consumerTask.addAssignmentsForPartitions(assignments); + thread.start(); + TestUtils.waitForCondition(() -> consumerTask.isUserPartitionAssigned(tpId0), "Timed out waiting for " + tpId0 + " to be assigned"); + + // Adding metadata records in the order opposite to the order of assignments + addRecord(consumer, metadataPartition, tpId1, 0); + addRecord(consumer, metadataPartition, tpId0, 1); + TestUtils.waitForCondition(() -> consumerTask.readOffsetForMetadataPartition(metadataPartition).equals(Optional.of(1L)), "Couldn't read record"); + // Only one record is processed, tpId1 record is skipped as unassigned + // but read offset is 1 e.g., record for tpId1 has been read by consumer + assertEquals(1, handler.metadataCounter); + + // Adding assignment for tpId1 after related metadata records have already been read + consumerTask.addAssignmentsForPartitions(Collections.singleton(tpId1)); + TestUtils.waitForCondition(() -> consumerTask.isUserPartitionAssigned(tpId1), "Timed out waiting for " + tpId1 + " to be assigned"); + + // Adding assignment for tpId0 to trigger the reset to last read offset + // and assignment for tpId3 that has different metadata partition to trigger the update of metadata snapshot + HashSet partitions = new HashSet<>(); + partitions.add(tpId0); + partitions.add(tpId3); + consumerTask.addAssignmentsForPartitions(partitions); + // explicitly re-adding the records since MockConsumer drops them on poll. + addRecord(consumer, metadataPartition, tpId1, 0); + addRecord(consumer, metadataPartition, tpId0, 1); + // Waiting for all metadata records to be re-read from the first metadata partition number + TestUtils.waitForCondition(() -> consumerTask.readOffsetForMetadataPartition(metadataPartition).equals(Optional.of(1L)), "Couldn't read record"); + // Verifying that all the metadata records from the first metadata partition were processed properly. + TestUtils.waitForCondition(() -> handler.metadataCounter == 2, "Couldn't read record"); + } + @Test public void testMaybeMarkUserPartitionsAsReady() throws InterruptedException { final TopicIdPartition tpId = getIdPartitions("hello", 1).get(0); From 961c28ae711d91682d708189056e79a249dd2fdb Mon Sep 17 00:00:00 2001 From: David Jacot Date: Tue, 4 Jun 2024 05:48:04 +0200 Subject: [PATCH 011/128] MINOR: Fix type in MetadataVersion.IBP_4_0_IV0 (#16181) This patch fixes a typo in MetadataVersion.IBP_4_0_IV0. It should be 0 not O. Reviewers: Justine Olshan , Jun Rao , Chia-Ping Tsai --- .../test/java/kafka/test/ClusterTestExtensionsTest.java | 2 +- core/src/test/java/kafka/test/annotation/ClusterTest.java | 2 +- .../integration/kafka/zk/ZkMigrationIntegrationTest.scala | 2 +- .../scala/unit/kafka/server/ApiVersionsRequestTest.scala | 4 ++-- .../scala/unit/kafka/server/ReplicationQuotasTest.scala | 2 +- .../org/apache/kafka/controller/QuorumControllerTest.java | 8 ++++---- .../java/org/apache/kafka/server/common/GroupVersion.java | 2 +- .../org/apache/kafka/server/common/MetadataVersion.java | 2 +- .../apache/kafka/server/common/MetadataVersionTest.java | 6 +++--- 9 files changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java index c8b53f8b8a..aba96eccdd 100644 --- a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java +++ b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java @@ -185,7 +185,7 @@ public void testNoAutoStart() { @ClusterTest public void testDefaults(ClusterInstance clusterInstance) { - Assertions.assertEquals(MetadataVersion.IBP_4_0_IVO, clusterInstance.config().metadataVersion()); + Assertions.assertEquals(MetadataVersion.IBP_4_0_IV0, clusterInstance.config().metadataVersion()); } @ClusterTests({ diff --git a/core/src/test/java/kafka/test/annotation/ClusterTest.java b/core/src/test/java/kafka/test/annotation/ClusterTest.java index bd95249b4f..5557abeb33 100644 --- a/core/src/test/java/kafka/test/annotation/ClusterTest.java +++ b/core/src/test/java/kafka/test/annotation/ClusterTest.java @@ -40,7 +40,7 @@ AutoStart autoStart() default AutoStart.DEFAULT; SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT; String listener() default ""; - MetadataVersion metadataVersion() default MetadataVersion.IBP_4_0_IVO; + MetadataVersion metadataVersion() default MetadataVersion.IBP_4_0_IV0; ClusterConfigProperty[] serverProperties() default {}; // users can add tags that they want to display in test String[] tags() default {}; diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index c0b6d916ec..e98a8fdecc 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -73,7 +73,7 @@ object ZkMigrationIntegrationTest { MetadataVersion.IBP_3_7_IV2, MetadataVersion.IBP_3_7_IV4, MetadataVersion.IBP_3_8_IV0, - MetadataVersion.IBP_4_0_IVO + MetadataVersion.IBP_4_0_IV0 ).map { mv => val serverProperties = new util.HashMap[String, String]() serverProperties.put("inter.broker.listener.name", "EXTERNAL") diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index a7415b5d50..d17872099b 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -47,7 +47,7 @@ object ApiVersionsRequestTest { List(ClusterConfig.defaultBuilder() .setTypes(java.util.Collections.singleton(Type.ZK)) .setServerProperties(serverProperties) - .setMetadataVersion(MetadataVersion.IBP_4_0_IVO) + .setMetadataVersion(MetadataVersion.IBP_4_0_IV0) .build()).asJava } @@ -83,7 +83,7 @@ object ApiVersionsRequestTest { class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { @ClusterTemplate("testApiVersionsRequestTemplate") - @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_4_0_IVO, serverProperties = Array( + @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_4_0_IV0, serverProperties = Array( new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "true") )) diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 03623bab41..1f947dd8fa 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -299,7 +299,7 @@ class ReplicationQuotasTest extends QuorumTestHarness { features.add(new BrokerRegistrationRequestData.Feature() .setName(MetadataVersion.FEATURE_NAME) .setMinSupportedVersion(MetadataVersion.IBP_3_0_IV1.featureLevel()) - .setMaxSupportedVersion(MetadataVersion.IBP_4_0_IVO.featureLevel())) + .setMaxSupportedVersion(MetadataVersion.IBP_4_0_IV0.featureLevel())) controllerServer.controller.registerBroker( ControllerRequestContextUtil.ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData() diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 4498778592..e494ca3e9a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -196,7 +196,7 @@ public void testConfigurationOperations() throws Throwable { ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData(). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IV0)). setBrokerId(0). setLogDirs(Collections.singletonList(Uuid.fromString("iiaQjkRPQcuMULNII0MUeA"))). setClusterId(logEnv.clusterId())).get(); @@ -240,7 +240,7 @@ public void testDelayedConfigurationOperations() throws Throwable { ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData(). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IV0)). setBrokerId(0). setLogDirs(Collections.singletonList(Uuid.fromString("sTbzRAMnTpahIyIPNjiLhw"))). setClusterId(logEnv.clusterId())).get(); @@ -298,7 +298,7 @@ public void testFenceMultipleBrokers() throws Throwable { new BrokerRegistrationRequestData(). setBrokerId(brokerId). setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IV0)). setIncarnationId(Uuid.randomUuid()). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); @@ -717,7 +717,7 @@ public void testUnregisterBroker() throws Throwable { setBrokerId(0). setClusterId(active.clusterId()). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IV0)). setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). setListeners(listeners)); assertEquals(5L, reply.get().epoch()); diff --git a/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java index 002d7ef33f..b59df64666 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java @@ -22,7 +22,7 @@ public enum GroupVersion implements FeatureVersion { // Version 1 enables the consumer rebalance protocol (KIP-848). - GV_1(1, MetadataVersion.IBP_4_0_IVO, Collections.emptyMap()); + GV_1(1, MetadataVersion.IBP_4_0_IV0, Collections.emptyMap()); public static final String FEATURE_NAME = "group.version"; diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index 26b67321e4..f867edb54e 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -205,7 +205,7 @@ public enum MetadataVersion { IBP_3_8_IV0(20, "3.8", "IV0", true), // Introduce version 1 of the GroupVersion feature (KIP-848). - IBP_4_0_IVO(21, "4.0", "IV0", false); + IBP_4_0_IV0(21, "4.0", "IV0", false); // NOTES when adding a new version: // Update the default version in @ClusterTest annotation to point to the latest version diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index cfcdcf3afe..d8397ce615 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -186,7 +186,7 @@ public void testFromVersionString() { assertEquals(IBP_3_8_IV0, MetadataVersion.fromVersionString("3.8-IV0")); - assertEquals(IBP_4_0_IVO, MetadataVersion.fromVersionString("4.0-IV0")); + assertEquals(IBP_4_0_IV0, MetadataVersion.fromVersionString("4.0-IV0")); } @Test @@ -247,7 +247,7 @@ public void testShortVersion() { assertEquals("3.7", IBP_3_7_IV3.shortVersion()); assertEquals("3.7", IBP_3_7_IV4.shortVersion()); assertEquals("3.8", IBP_3_8_IV0.shortVersion()); - assertEquals("4.0", IBP_4_0_IVO.shortVersion()); + assertEquals("4.0", IBP_4_0_IV0.shortVersion()); } @Test @@ -297,7 +297,7 @@ public void testVersion() { assertEquals("3.7-IV3", IBP_3_7_IV3.version()); assertEquals("3.7-IV4", IBP_3_7_IV4.version()); assertEquals("3.8-IV0", IBP_3_8_IV0.version()); - assertEquals("4.0-IV0", IBP_4_0_IVO.version()); + assertEquals("4.0-IV0", IBP_4_0_IV0.version()); } @Test From 0aa0a01d9cb44e89e4e86cece11022e866cad2b3 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Tue, 4 Jun 2024 02:27:35 -0400 Subject: [PATCH 012/128] KAFKA-16664; Re-add EventAccumulator.poll(long, TimeUnit) (#16144) We have revamped the thread idle ratio metric in https://github.com/apache/kafka/pull/15835. https://github.com/apache/kafka/pull/15835#discussion_r1588068337 describes a case where the metric loses accuracy and in order to set a lower bound to the accuracy, this patch re-adds a poll with a timeout that was removed as part of https://github.com/apache/kafka/pull/15430. Reviewers: David Jacot --- .../group/runtime/EventAccumulator.java | 35 +++++++------------ .../runtime/MultiThreadedEventProcessor.java | 8 ++++- .../group/runtime/EventAccumulatorTest.java | 18 ++++++---- .../MultiThreadedEventProcessorTest.java | 4 +-- 4 files changed, 33 insertions(+), 32 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java index 2c22232c47..cc1ab69cd7 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java @@ -27,6 +27,7 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -162,43 +163,31 @@ public void addFirst(T event) throws RejectedExecutionException { } /** - * Returns the next {{@link Event}} available or null if no event is - * available. + * Immediately returns the next {{@link Event}} available or null + * if the accumulator is empty. * * @return The next event available or null. */ public T poll() { - lock.lock(); - try { - K key = randomKey(); - if (key == null) return null; - - Deque queue = queues.get(key); - T event = queue.poll(); - - if (queue.isEmpty()) queues.remove(key); - inflightKeys.add(key); - size--; - - return event; - } finally { - lock.unlock(); - } + return poll(0, TimeUnit.MILLISECONDS); } /** - * Returns the next {{@link Event}} available. This method blocks until an - * event is available or accumulator is closed. + * Returns the next {{@link Event}} available. This method blocks for the provided + * time and returns null of no event is available. * + * @param timeout The timeout. + * @param unit The timeout unit. * @return The next event available or null. */ - public T take() { + public T poll(long timeout, TimeUnit unit) { lock.lock(); try { K key = randomKey(); - while (key == null && !closed) { + long nanos = unit.toNanos(timeout); + while (key == null && !closed && nanos > 0) { try { - condition.await(); + nanos = condition.awaitNanos(nanos); } catch (InterruptedException e) { // Ignore. } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java index 31fa52ea7d..6265334872 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -34,6 +35,11 @@ */ public class MultiThreadedEventProcessor implements CoordinatorEventProcessor { + /** + * The poll timeout to wait for an event by the EventProcessorThread. + */ + private static final long POLL_TIMEOUT_MS = 300L; + /** * The logger. */ @@ -129,7 +135,7 @@ private void handleEvents() { // time should be discounted by # threads. long idleStartTimeMs = time.milliseconds(); - CoordinatorEvent event = accumulator.take(); + CoordinatorEvent event = accumulator.poll(POLL_TIMEOUT_MS, TimeUnit.MILLISECONDS); long idleEndTimeMs = time.milliseconds(); long idleTimeMs = idleEndTimeMs - idleStartTimeMs; metrics.recordThreadIdleTime(idleTimeMs / threads.size()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java index faac0f46f6..602614414a 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java @@ -190,9 +190,12 @@ public void testDoneUnblockWaitingThreads() throws ExecutionException, Interrupt MockEvent event1 = new MockEvent(1, 1); MockEvent event2 = new MockEvent(1, 2); - CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::take); - CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::take); - CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future0 = CompletableFuture.supplyAsync(() -> + accumulator.poll(Long.MAX_VALUE, TimeUnit.MILLISECONDS)); + CompletableFuture future1 = CompletableFuture.supplyAsync(() -> + accumulator.poll(Long.MAX_VALUE, TimeUnit.MILLISECONDS)); + CompletableFuture future2 = CompletableFuture.supplyAsync(() -> + accumulator.poll(Long.MAX_VALUE, TimeUnit.MILLISECONDS)); List> futures = Arrays.asList(future0, future1, future2); assertFalse(future0.isDone()); @@ -245,9 +248,12 @@ public void testDoneUnblockWaitingThreads() throws ExecutionException, Interrupt public void testCloseUnblockWaitingThreads() throws ExecutionException, InterruptedException, TimeoutException { EventAccumulator accumulator = new EventAccumulator<>(); - CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::take); - CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::take); - CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future0 = CompletableFuture.supplyAsync(() -> + accumulator.poll(Long.MAX_VALUE, TimeUnit.MILLISECONDS)); + CompletableFuture future1 = CompletableFuture.supplyAsync(() -> + accumulator.poll(Long.MAX_VALUE, TimeUnit.MILLISECONDS)); + CompletableFuture future2 = CompletableFuture.supplyAsync(() -> + accumulator.poll(Long.MAX_VALUE, TimeUnit.MILLISECONDS)); assertFalse(future0.isDone()); assertFalse(future1.isDone()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java index 0f2801daec..0b8f04fe34 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java @@ -60,8 +60,8 @@ public DelayEventAccumulator(Time time, long takeDelayMs) { } @Override - public CoordinatorEvent take() { - CoordinatorEvent event = super.take(); + public CoordinatorEvent poll(long timeout, TimeUnit unit) { + CoordinatorEvent event = super.poll(timeout, unit); time.sleep(takeDelayMs); return event; } From c295feff3c9e6a7764fee6ff52d4130f8fef10b9 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Tue, 4 Jun 2024 11:45:11 +0100 Subject: [PATCH 013/128] KAFKA-16047: Use REQUEST_TIMEOUT_MS_CONFIG in AdminClient.fenceProducers (#16151) Use REQUEST_TIMEOUT_MS_CONFIG in AdminClient.fenceProducers, or options.timeoutMs if specified, as transaction timeout. No transaction will be started with this timeout, but ReplicaManager.appendRecords uses this value as its timeout. Use REQUEST_TIMEOUT_MS_CONFIG like a regular producer append to allow for replication to take place. Co-Authored-By: Adrian Preston --- .../kafka/clients/admin/KafkaAdminClient.java | 2 +- .../internals/FenceProducersHandler.java | 12 ++++++---- .../internals/FenceProducersHandlerTest.java | 23 ++++++++++++++----- 3 files changed, 26 insertions(+), 11 deletions(-) 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 71d39900cd..92ba6ad3d6 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 @@ -4569,7 +4569,7 @@ public ListTransactionsResult listTransactions(ListTransactionsOptions options) public FenceProducersResult fenceProducers(Collection transactionalIds, FenceProducersOptions options) { AdminApiFuture.SimpleAdminApiFuture future = FenceProducersHandler.newFuture(transactionalIds); - FenceProducersHandler handler = new FenceProducersHandler(logContext); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); invokeDriver(handler, future, options.timeoutMs); return new FenceProducersResult(future.all()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java index 23572dd441..9a12bc1959 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.clients.admin.FenceProducersOptions; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; @@ -38,12 +39,16 @@ public class FenceProducersHandler extends AdminApiHandler.Unbatched { private final Logger log; private final AdminApiLookupStrategy lookupStrategy; + private final int txnTimeoutMs; public FenceProducersHandler( - LogContext logContext + FenceProducersOptions options, + LogContext logContext, + int requestTimeoutMs ) { this.log = logContext.logger(FenceProducersHandler.class); this.lookupStrategy = new CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.TRANSACTION, logContext); + this.txnTimeoutMs = options.timeoutMs() != null ? options.timeoutMs() : requestTimeoutMs; } public static AdminApiFuture.SimpleAdminApiFuture newFuture( @@ -82,9 +87,8 @@ InitProducerIdRequest.Builder buildSingleRequest(int brokerId, CoordinatorKey ke .setProducerEpoch(ProducerIdAndEpoch.NONE.epoch) .setProducerId(ProducerIdAndEpoch.NONE.producerId) .setTransactionalId(key.idValue) - // Set transaction timeout to 1 since it's only being initialized to fence out older producers with the same transactional ID, - // and shouldn't be used for any actual record writes - .setTransactionTimeoutMs(1); + // This timeout is used by the coordinator to append the record with the new producer epoch to the transaction log. + .setTransactionTimeoutMs(txnTimeoutMs); return new InitProducerIdRequest.Builder(data); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java index 34ed2e6772..9665bd0bdf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.clients.admin.FenceProducersOptions; import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult; import org.apache.kafka.common.Node; import org.apache.kafka.common.message.InitProducerIdResponseData; @@ -39,11 +40,21 @@ public class FenceProducersHandlerTest { private final LogContext logContext = new LogContext(); private final Node node = new Node(1, "host", 1234); + private final int requestTimeoutMs = 30000; + private final FenceProducersOptions options = new FenceProducersOptions(); @Test public void testBuildRequest() { - FenceProducersHandler handler = new FenceProducersHandler(logContext); - mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId)); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); + mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, requestTimeoutMs)); + } + + @Test + public void testBuildRequestOptionsTimeout() { + final int optionsTimeoutMs = 50000; + options.timeoutMs(optionsTimeoutMs); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); + mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, optionsTimeoutMs)); } @Test @@ -51,7 +62,7 @@ public void testHandleSuccessfulResponse() { String transactionalId = "foo"; CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId); - FenceProducersHandler handler = new FenceProducersHandler(logContext); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); short epoch = 57; long producerId = 7; @@ -73,7 +84,7 @@ public void testHandleSuccessfulResponse() { @Test public void testHandleErrorResponse() { String transactionalId = "foo"; - FenceProducersHandler handler = new FenceProducersHandler(logContext); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); assertFatalError(handler, transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED); assertFatalError(handler, transactionalId, Errors.CLUSTER_AUTHORIZATION_FAILED); assertFatalError(handler, transactionalId, Errors.UNKNOWN_SERVER_ERROR); @@ -136,10 +147,10 @@ private ApiResult handleResponseError( return result; } - private void assertLookup(FenceProducersHandler handler, String transactionalId) { + private void assertLookup(FenceProducersHandler handler, String transactionalId, int txnTimeoutMs) { CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId); InitProducerIdRequest.Builder request = handler.buildSingleRequest(1, key); assertEquals(transactionalId, request.data.transactionalId()); - assertEquals(1, request.data.transactionTimeoutMs()); + assertEquals(txnTimeoutMs, request.data.transactionTimeoutMs()); } } From 7404fdffa671ed55188444fc96319b4dc301da74 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 4 Jun 2024 15:36:24 +0200 Subject: [PATCH 014/128] KAFKA-16837, KAFKA-16838: Ignore task configs for deleted connectors, and compare raw task configs before publishing them (#16122) Reviewers: Mickael Maison --- .../kafka/connect/runtime/AbstractHerder.java | 10 +- .../distributed/DistributedHerder.java | 4 +- .../runtime/standalone/StandaloneHerder.java | 4 +- .../storage/KafkaConfigBackingStore.java | 29 ++- .../ConnectWorkerIntegrationTest.java | 199 ++++++++++++++++++ .../connect/runtime/AbstractHerderTest.java | 27 +++ .../KafkaConfigBackingStoreMockitoTest.java | 53 ++++- .../util/clusters/EmbeddedKafkaCluster.java | 16 +- 8 files changed, 325 insertions(+), 17 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index 2a27103079..c6aeea80a2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -124,7 +124,7 @@ */ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, ConnectorStatus.Listener { - private final Logger log = LoggerFactory.getLogger(AbstractHerder.class); + private static final Logger log = LoggerFactory.getLogger(AbstractHerder.class); private final String workerId; protected final Worker worker; @@ -1039,16 +1039,16 @@ public static List> reverseTransform(String connName, return result; } - public boolean taskConfigsChanged(ClusterConfigState configState, String connName, List> taskProps) { + public static boolean taskConfigsChanged(ClusterConfigState configState, String connName, List> rawTaskProps) { int currentNumTasks = configState.taskCount(connName); boolean result = false; - if (taskProps.size() != currentNumTasks) { - log.debug("Connector {} task count changed from {} to {}", connName, currentNumTasks, taskProps.size()); + if (rawTaskProps.size() != currentNumTasks) { + log.debug("Connector {} task count changed from {} to {}", connName, currentNumTasks, rawTaskProps.size()); result = true; } else { for (int index = 0; index < currentNumTasks; index++) { ConnectorTaskId taskId = new ConnectorTaskId(connName, index); - if (!taskProps.get(index).equals(configState.taskConfig(taskId))) { + if (!rawTaskProps.get(index).equals(configState.rawTaskConfig(taskId))) { log.debug("Connector {} has change in configuration for task {}-{}", connName, connName, index); result = true; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index cdffbb8787..ab46ee536a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -2229,11 +2229,11 @@ private void reconfigureConnector(final String connName, final Callback cb } private void publishConnectorTaskConfigs(String connName, List> taskProps, Callback cb) { - if (!taskConfigsChanged(configState, connName, taskProps)) { + List> rawTaskProps = reverseTransform(connName, configState, taskProps); + if (!taskConfigsChanged(configState, connName, rawTaskProps)) { return; } - List> rawTaskProps = reverseTransform(connName, configState, taskProps); if (isLeader()) { writeTaskConfigs(connName, rawTaskProps); cb.onCompletion(null, null); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index e773eeefd5..2768d910d4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -519,10 +519,10 @@ private synchronized void updateConnectorTasks(String connName) { } List> newTaskConfigs = recomputeTaskConfigs(connName); + List> rawTaskConfigs = reverseTransform(connName, configState, newTaskConfigs); - if (taskConfigsChanged(configState, connName, newTaskConfigs)) { + if (taskConfigsChanged(configState, connName, rawTaskConfigs)) { removeConnectorTasks(connName); - List> rawTaskConfigs = reverseTransform(connName, configState, newTaskConfigs); configBackingStore.putTaskConfigs(connName, rawTaskConfigs); createConnectorTasks(connName); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index c24dd6c790..7981f4425d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -997,11 +997,8 @@ private void processConnectorConfigRecord(String connectorName, SchemaAndValue v synchronized (lock) { if (value.value() == null) { // Connector deletion will be written as a null value + processConnectorRemoval(connectorName); log.info("Successfully processed removal of connector '{}'", connectorName); - connectorConfigs.remove(connectorName); - connectorTaskCounts.remove(connectorName); - taskConfigs.keySet().removeIf(taskId -> taskId.connector().equals(connectorName)); - deferredTaskUpdates.remove(connectorName); removed = true; } else { // Connector configs can be applied and callbacks invoked immediately @@ -1064,6 +1061,21 @@ private void processTaskConfigRecord(ConnectorTaskId taskId, SchemaAndValue valu private void processTasksCommitRecord(String connectorName, SchemaAndValue value) { List updatedTasks = new ArrayList<>(); synchronized (lock) { + // Edge case: connector was deleted before these task configs were published, + // but compaction took place and both the original connector config and the + // tombstone message for it have been removed from the config topic + // We should ignore these task configs + if (!connectorConfigs.containsKey(connectorName)) { + processConnectorRemoval(connectorName); + log.debug( + "Ignoring task configs for connector {}; it appears that the connector was deleted previously " + + "and that log compaction has since removed any trace of its previous configurations " + + "from the config topic", + connectorName + ); + return; + } + // Apply any outstanding deferred task updates for the given connector. Note that just because we // encounter a commit message does not mean it will result in consistent output. In particular due to // compaction, there may be cases where . For example if we have the following sequence of writes: @@ -1168,7 +1180,7 @@ private void processTaskCountRecord(String connectorName, SchemaAndValue value) log.debug("Setting task count record for connector '{}' to {}", connectorName, taskCount); connectorTaskCountRecords.put(connectorName, taskCount); - // If a task count record appears after the latest task configs, the connectors doesn't need a round of zombie + // If a task count record appears after the latest task configs, the connector doesn't need a round of zombie // fencing before it can start tasks with the latest configs connectorsPendingFencing.remove(connectorName); } @@ -1244,6 +1256,13 @@ private void processLoggerLevelRecord(String namespace, SchemaAndValue value) { } } + private void processConnectorRemoval(String connectorName) { + connectorConfigs.remove(connectorName); + connectorTaskCounts.remove(connectorName); + taskConfigs.keySet().removeIf(taskId -> taskId.connector().equals(connectorName)); + deferredTaskUpdates.remove(connectorName); + } + private ConnectorTaskId parseTaskId(String key) { String[] parts = key.split("-"); if (parts.length < 3) return null; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java index 8a730c198d..24cbd1c280 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.integration; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.provider.FileConfigProvider; import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; @@ -39,11 +40,14 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; +import java.io.File; +import java.io.FileOutputStream; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -56,6 +60,9 @@ import static javax.ws.rs.core.Response.Status.INTERNAL_SERVER_ERROR; import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.common.config.AbstractConfig.CONFIG_PROVIDERS_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.DELETE_RETENTION_MS_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.SEGMENT_MS_CONFIG; import static org.apache.kafka.connect.integration.BlockingConnectorTest.TASK_STOP; import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; @@ -71,6 +78,7 @@ import static org.apache.kafka.connect.runtime.WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_STORAGE_PREFIX; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.SCHEDULED_REBALANCE_MAX_DELAY_MS_CONFIG; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.REBALANCE_TIMEOUT_MS_CONFIG; @@ -108,6 +116,9 @@ public class ConnectWorkerIntegrationTest { @Rule public TestRule watcher = ConnectIntegrationTestUtils.newTestWatcher(log); + @Rule + public TemporaryFolder tmp = new TemporaryFolder(); + @Before public void setup() { // setup Connect worker properties @@ -1123,6 +1134,194 @@ public void testTasksMaxEnforcement() throws Exception { ); } + /** + * Task configs are not removed from the config topic after a connector is deleted. + * When topic compaction takes place, this can cause the tombstone message for the + * connector config to be deleted, leaving the task configs in the config topic with no + * explicit record of the connector's deletion. + *

    + * This test guarantees that those older task configs are never used, even when the + * connector is recreated later. + */ + @Test + public void testCompactedDeletedOlderConnectorConfig() throws Exception { + brokerProps.put("log.cleaner.backoff.ms", "100"); + brokerProps.put("log.cleaner.delete.retention.ms", "1"); + brokerProps.put("log.cleaner.max.compaction.lag.ms", "1"); + brokerProps.put("log.cleaner.min.cleanable.ratio", "0"); + brokerProps.put("log.cleaner.min.compaction.lag.ms", "1"); + brokerProps.put("log.cleaner.threads", "1"); + + final String configTopic = "kafka-16838-configs"; + final int offsetCommitIntervalMs = 100; + workerProps.put(CONFIG_TOPIC_CONFIG, configTopic); + workerProps.put(CONFIG_STORAGE_PREFIX + SEGMENT_MS_CONFIG, "100"); + workerProps.put(CONFIG_STORAGE_PREFIX + DELETE_RETENTION_MS_CONFIG, "1"); + workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Integer.toString(offsetCommitIntervalMs)); + + final int numWorkers = 1; + connect = connectBuilder + .numWorkers(numWorkers) + .build(); + // start the clusters + connect.start(); + + connect.assertions().assertAtLeastNumWorkersAreUp( + numWorkers, + "Initial group of workers did not start in time." + ); + + final String connectorTopic = "connector-topic"; + connect.kafka().createTopic(connectorTopic, 1); + + ConnectorHandle connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); + connectorHandle.expectedCommits(NUM_TASKS * 2); + + Map connectorConfig = defaultSourceConnectorProps(connectorTopic); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + NUM_TASKS, + "Connector or its tasks did not start in time" + ); + connectorHandle.awaitCommits(offsetCommitIntervalMs * 3); + + connect.deleteConnector(CONNECTOR_NAME); + + // Roll the entire cluster + connect.activeWorkers().forEach(connect::removeWorker); + + // Miserable hack: produce directly to the config topic and then wait a little bit + // in order to trigger segment rollover and allow compaction to take place + connect.kafka().produce(configTopic, "garbage-key-1", null); + Thread.sleep(1_000); + connect.kafka().produce(configTopic, "garbage-key-2", null); + Thread.sleep(1_000); + + for (int i = 0; i < numWorkers; i++) + connect.addWorker(); + + connect.assertions().assertAtLeastNumWorkersAreUp( + numWorkers, + "Initial group of workers did not start in time." + ); + + final TopicPartition connectorTopicPartition = new TopicPartition(connectorTopic, 0); + final long initialEndOffset = connect.kafka().endOffset(connectorTopicPartition); + assertTrue( + "Source connector should have published at least one record to Kafka", + initialEndOffset > 0 + ); + + connectorHandle.expectedCommits(NUM_TASKS * 2); + + // Re-create the connector with a different config (targets a different topic) + final String otherConnectorTopic = "other-topic"; + connect.kafka().createTopic(otherConnectorTopic, 1); + connectorConfig.put(TOPIC_CONFIG, otherConnectorTopic); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + NUM_TASKS, + "Connector or its tasks did not start in time" + ); + connectorHandle.awaitCommits(offsetCommitIntervalMs * 3); + + // See if any new records got written to the old topic + final long nextEndOffset = connect.kafka().endOffset(connectorTopicPartition); + assertEquals( + "No new records should have been written to the older topic", + initialEndOffset, + nextEndOffset + ); + } + + /** + * If a connector has existing tasks, and then generates new task configs, workers compare the + * new and existing configs before publishing them to the config topic. If there is no difference, + * workers do not publish task configs (this is a workaround to prevent infinite loops with eager + * rebalancing). + *

    + * This test tries to guarantee that, if the old task configs become invalid because of + * an invalid config provider reference, it will still be possible to reconfigure the connector. + */ + @Test + public void testReconfigureConnectorWithFailingTaskConfigs() throws Exception { + final int offsetCommitIntervalMs = 100; + workerProps.put(CONFIG_PROVIDERS_CONFIG, "file"); + workerProps.put(CONFIG_PROVIDERS_CONFIG + ".file.class", FileConfigProvider.class.getName()); + workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Integer.toString(offsetCommitIntervalMs)); + + final int numWorkers = 1; + connect = connectBuilder + .numWorkers(numWorkers) + .build(); + // start the clusters + connect.start(); + + connect.assertions().assertAtLeastNumWorkersAreUp( + numWorkers, + "Initial group of workers did not start in time." + ); + + final String firstConnectorTopic = "connector-topic-1"; + connect.kafka().createTopic(firstConnectorTopic); + + final File secretsFile = tmp.newFile("test-secrets"); + final Properties secrets = new Properties(); + final String throughputSecretKey = "secret-throughput"; + secrets.put(throughputSecretKey, "10"); + try (FileOutputStream secretsOutputStream = new FileOutputStream(secretsFile)) { + secrets.store(secretsOutputStream, null); + } + + ConnectorHandle connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); + connectorHandle.expectedCommits(NUM_TASKS * 2); + + Map connectorConfig = defaultSourceConnectorProps(firstConnectorTopic); + connectorConfig.put( + "throughput", + "${file:" + secretsFile.getAbsolutePath() + ":" + throughputSecretKey + "}" + ); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + NUM_TASKS, + "Connector or its tasks did not start in time" + ); + connectorHandle.awaitCommits(offsetCommitIntervalMs * 3); + + // Delete the secrets file, which should render the old task configs invalid + assertTrue("Failed to delete secrets file", secretsFile.delete()); + + // Use a start latch here instead of assertConnectorAndExactlyNumTasksAreRunning + // since failure to reconfigure the tasks (which may occur if the bug this test was written + // to help catch resurfaces) will not cause existing tasks to fail or stop running + StartAndStopLatch restarts = connectorHandle.expectedStarts(1); + connectorHandle.expectedCommits(NUM_TASKS * 2); + + final String secondConnectorTopic = "connector-topic-2"; + connect.kafka().createTopic(secondConnectorTopic, 1); + + // Stop using the config provider for this connector, and instruct it to start writing to the + // old topic again + connectorConfig.put("throughput", "10"); + connectorConfig.put(TOPIC_CONFIG, secondConnectorTopic); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + assertTrue( + "Connector tasks were not restarted in time", + restarts.await(10, TimeUnit.SECONDS) + ); + connectorHandle.awaitCommits(offsetCommitIntervalMs * 3); + + final long endOffset = connect.kafka().endOffset(new TopicPartition(secondConnectorTopic, 0)); + assertTrue( + "Source connector should have published at least one record to new Kafka topic " + + "after being reconfigured", + endOffset > 0 + ); + } + private Map defaultSourceConnectorProps(String topic) { // setup props for the source connector Map props = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index 62283a0277..5bfbe2498c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -84,12 +84,14 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.CALLS_REAL_METHODS; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -1116,6 +1118,31 @@ public void testConnectorOffsets() throws Exception { assertEquals(offsets, cb.get(1000, TimeUnit.MILLISECONDS)); } + @Test + public void testTaskConfigComparison() { + ClusterConfigState snapshot = mock(ClusterConfigState.class); + + when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIGS.size()); + TASK_CONFIGS_MAP.forEach((task, config) -> when(snapshot.rawTaskConfig(task)).thenReturn(config)); + // Same task configs, same number of tasks--no change + assertFalse(AbstractHerder.taskConfigsChanged(snapshot, CONN1, TASK_CONFIGS)); + + when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIGS.size() + 1); + // Different number of tasks; should report a change + assertTrue(AbstractHerder.taskConfigsChanged(snapshot, CONN1, TASK_CONFIGS)); + + when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIG.size()); + List> alteredTaskConfigs = new ArrayList<>(TASK_CONFIGS); + alteredTaskConfigs.set(alteredTaskConfigs.size() - 1, Collections.emptyMap()); + // Last task config is different; should report a change + assertTrue(AbstractHerder.taskConfigsChanged(snapshot, CONN1, alteredTaskConfigs)); + + // Make sure we used exclusively raw task configs and never attempted transformation, + // since otherwise failures in transformation could either cause an infinite loop of task + // config generation, or prevent any task configs from being published + verify(snapshot, never()).taskConfig(any()); + } + protected void addConfigKey(Map keys, String name, String group) { ConfigDef configDef = new ConfigDef().define(name, ConfigDef.Type.STRING, null, null, ConfigDef.Importance.HIGH, "doc", group, 10, diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java index 6ebac34103..b41bd99b91 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java @@ -170,7 +170,8 @@ public class KafkaConfigBackingStoreMockitoTest { .put("state.v2", "STOPPED"); private static final List CONNECTOR_TASK_COUNT_RECORD_STRUCTS = Arrays.asList( new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 6), - new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 9) + new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 9), + new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 2) ); // The exact format doesn't matter here since both conversions are mocked @@ -813,6 +814,56 @@ public void testRestoreZeroTasks() { verify(configLog).stop(); } + @Test + public void testRestoreCompactedDeletedConnector() { + // When a connector is deleted, we emit a tombstone record for its config (with key + // "connector-") and its target state (with key "target-state-"), but not + // for its task configs + // As a result, we need to carefully handle the case where task configs are present in + // the config topic for a connector, but there is no accompanying config for the + // connector itself + + int offset = 0; + List> existingRecords = Arrays.asList( + new ConsumerRecord<>(TOPIC, 0, offset++, 0L, TimestampType.CREATE_TIME, 0, 0, + TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, offset++, 0L, TimestampType.CREATE_TIME, 0, 0, + TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, offset++, 0L, TimestampType.CREATE_TIME, 0, 0, + COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, offset++, 0L, TimestampType.CREATE_TIME, 0, 0, + CONNECTOR_TASK_COUNT_RECORD_KEYS.get(0), CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty())); + LinkedHashMap deserialized = new LinkedHashMap<>(); + deserialized.put(CONFIGS_SERIALIZED.get(0), TASK_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(2), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR); + deserialized.put(CONFIGS_SERIALIZED.get(3), CONNECTOR_TASK_COUNT_RECORD_STRUCTS.get(2)); + logOffset = offset; + expectStart(existingRecords, deserialized); + when(configLog.partitionCount()).thenReturn(1); + + configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); + verifyConfigure(); + configStorage.start(); + + // Should see no connectors and no task configs + ClusterConfigState configState = configStorage.snapshot(); + assertEquals(Collections.emptySet(), configState.connectors()); + assertEquals(0, configState.taskCount(CONNECTOR_1_NAME)); + assertNull(configState.rawTaskConfig(TASK_IDS.get(0))); + assertNull(configState.rawTaskConfig(TASK_IDS.get(1))); + + // Probe internal collections just to be sure + assertEquals(Collections.emptyMap(), configState.connectorConfigs); + assertEquals(Collections.emptyMap(), configState.taskConfigs); + assertEquals(Collections.emptyMap(), configState.connectorTaskCounts); + + // Exception: we still include task count records, for the unlikely-but-possible case + // where there are still zombie instances of the tasks for this long-deleted connector + // running somewhere on the cluster + assertEquals(2, (int) configState.taskCountRecord(CONNECTOR_1_NAME)); + } + @Test public void testRecordToRestartRequest() { ConsumerRecord record = new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, RESTART_CONNECTOR_KEYS.get(0), diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index f959d22537..fe97476a71 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -50,7 +50,6 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.errors.ConnectException; @@ -112,7 +111,7 @@ public class EmbeddedKafkaCluster { // Kafka Config private final KafkaServer[] brokers; private final Properties brokerConfig; - private final Time time = new MockTime(); + private final Time time = Time.SYSTEM; private final int[] currentBrokerPorts; private final String[] currentBrokerLogDirs; private final boolean hasListenerConfig; @@ -611,6 +610,19 @@ public ConsumerRecords consumeAll( return new ConsumerRecords<>(records); } + public long endOffset(TopicPartition topicPartition) throws TimeoutException, InterruptedException, ExecutionException { + try (Admin admin = createAdminClient()) { + Map offsets = Collections.singletonMap( + topicPartition, OffsetSpec.latest() + ); + return admin.listOffsets(offsets) + .partitionResult(topicPartition) + // Hardcode duration for now; if necessary, we can add a parameter for it later + .get(10, TimeUnit.SECONDS) + .offset(); + } + } + /** * List all the known partitions for the given {@link Collection} of topics * @param maxDurationMs the max duration to wait for while fetching metadata from Kafka (in milliseconds). From 15ab07a822cfdc2221c6b1406a2ed011be2a0223 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Tue, 4 Jun 2024 15:38:56 +0200 Subject: [PATCH 015/128] MINOR: Log time taken to compute the target assignment (#16185) The time taken to compute a new assignment is critical. This patches extending the existing logging to log it too. This is very useful information to have. Reviewers: Luke Chen --- .../kafka/coordinator/group/GroupMetadataManager.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index a1295397a8..34dcaf146b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -1914,11 +1914,13 @@ private Assignment updateTargetAssignment( assignmentResultBuilder.removeMember(member.memberId()); } + long startTimeMs = time.milliseconds(); TargetAssignmentBuilder.TargetAssignmentResult assignmentResult = assignmentResultBuilder.build(); + long assignorTimeMs = time.milliseconds() - startTimeMs; - log.info("[GroupId {}] Computed a new target assignment for epoch {} with '{}' assignor: {}.", - group.groupId(), groupEpoch, preferredServerAssignor, assignmentResult.targetAssignment()); + log.info("[GroupId {}] Computed a new target assignment for epoch {} with '{}' assignor in {}ms: {}.", + group.groupId(), groupEpoch, preferredServerAssignor, assignorTimeMs, assignmentResult.targetAssignment()); records.addAll(assignmentResult.records()); From ebc68f00e4cca348361bcf783ea11fb79f6f18df Mon Sep 17 00:00:00 2001 From: Ritika Reddy <98577846+rreddy-22@users.noreply.github.com> Date: Tue, 4 Jun 2024 06:44:37 -0700 Subject: [PATCH 016/128] KAFKA-16821; Member Subscription Spec Interface (#16068) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch reworks the `PartitionAssignor` interface to use interfaces instead of POJOs. It mainly introduces the `MemberSubscriptionSpec` interface that represents a member subscription and changes the `GroupSpec` interfaces to expose the subscriptions and the assignments via different methods. The patch does not change the performance. before: ``` Benchmark (memberCount) (partitionsToMemberRatio) (topicCount) Mode Cnt Score Error Units TargetAssignmentBuilderBenchmark.build 10000 10 100 avgt 5 3.462 ± 0.687 ms/op TargetAssignmentBuilderBenchmark.build 10000 10 1000 avgt 5 3.626 ± 0.412 ms/op JMH benchmarks done ``` after: ``` Benchmark (memberCount) (partitionsToMemberRatio) (topicCount) Mode Cnt Score Error Units TargetAssignmentBuilderBenchmark.build 10000 10 100 avgt 5 3.677 ± 0.683 ms/op TargetAssignmentBuilderBenchmark.build 10000 10 1000 avgt 5 3.991 ± 0.065 ms/op JMH benchmarks done ``` Reviewers: David Jacot --- .../group/assignor/AssignmentMemberSpec.java | 122 ------ .../GeneralUniformAssignmentBuilder.java | 37 +- .../coordinator/group/assignor/GroupSpec.java | 24 +- .../group/assignor/GroupSpecImpl.java | 66 +++- .../assignor/MemberSubscriptionSpec.java | 41 ++ .../assignor/MemberSubscriptionSpecImpl.java | 91 +++++ .../OptimizedUniformAssignmentBuilder.java | 12 +- .../group/assignor/RangeAssignor.java | 16 +- .../group/assignor/UniformAssignor.java | 2 +- .../consumer/TargetAssignmentBuilder.java | 28 +- .../coordinator/group/consumer/TopicIds.java | 7 + .../coordinator/group/AssignmentTestUtil.java | 15 +- .../group/NoOpPartitionAssignor.java | 7 +- .../GeneralUniformAssignmentBuilderTest.java | 212 +++++------ .../group/assignor/GroupSpecImplTest.java | 46 ++- ...OptimizedUniformAssignmentBuilderTest.java | 155 ++++---- .../group/assignor/RangeAssignorTest.java | 358 ++++++++---------- .../consumer/TargetAssignmentBuilderTest.java | 37 +- .../assignor/ServerSideAssignorBenchmark.java | 50 +-- .../TargetAssignmentBuilderBenchmark.java | 15 +- 20 files changed, 673 insertions(+), 668 deletions(-) delete mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java create mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberSubscriptionSpec.java create mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberSubscriptionSpecImpl.java diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java deleted file mode 100644 index 2a91a111aa..0000000000 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java +++ /dev/null @@ -1,122 +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.coordinator.group.assignor; - -import org.apache.kafka.common.Uuid; - -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; - -/** - * The assignment specification for a consumer group member. - */ -public class AssignmentMemberSpec { - /** - * The instance ID if provided. - */ - private final Optional instanceId; - - /** - * The rack ID if provided. - */ - private final Optional rackId; - - /** - * Topics Ids that the member is subscribed to. - */ - private final Set subscribedTopicIds; - - /** - * Partitions assigned keyed by topicId. - */ - private final Map> assignedPartitions; - - /** - * @return The instance ID as an Optional. - */ - public Optional instanceId() { - return instanceId; - } - - /** - * @return The rack ID as an Optional. - */ - public Optional rackId() { - return rackId; - } - - /** - * @return Set of subscribed topic Ids. - */ - public Set subscribedTopicIds() { - return subscribedTopicIds; - } - - /** - * @return Assigned partitions keyed by topic Ids. - */ - public Map> assignedPartitions() { - return assignedPartitions; - } - - public AssignmentMemberSpec( - Optional instanceId, - Optional rackId, - Set subscribedTopicIds, - Map> assignedPartitions - ) { - Objects.requireNonNull(instanceId); - Objects.requireNonNull(rackId); - Objects.requireNonNull(subscribedTopicIds); - Objects.requireNonNull(assignedPartitions); - this.instanceId = instanceId; - this.rackId = rackId; - this.subscribedTopicIds = subscribedTopicIds; - this.assignedPartitions = assignedPartitions; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - AssignmentMemberSpec that = (AssignmentMemberSpec) o; - if (!instanceId.equals(that.instanceId)) return false; - if (!rackId.equals(that.rackId)) return false; - if (!subscribedTopicIds.equals(that.subscribedTopicIds)) return false; - return assignedPartitions.equals(that.assignedPartitions); - } - - @Override - public int hashCode() { - int result = instanceId.hashCode(); - result = 31 * result + rackId.hashCode(); - result = 31 * result + subscribedTopicIds.hashCode(); - result = 31 * result + assignedPartitions.hashCode(); - return result; - } - - @Override - public String toString() { - return "AssignmentMemberSpec(instanceId=" + instanceId + - ", rackId=" + rackId + - ", subscribedTopicIds=" + subscribedTopicIds + - ", assignedPartitions=" + assignedPartitions + - ')'; - } -} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java index d6f83df005..884594a057 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java @@ -53,9 +53,9 @@ public class GeneralUniformAssignmentBuilder extends AbstractUniformAssignmentBu private static final Logger LOG = LoggerFactory.getLogger(GeneralUniformAssignmentBuilder.class); /** - * The member metadata obtained from the assignment specification. + * The group metadata specification. */ - private final Map members; + private final GroupSpec groupSpec; /** * The topic and partition metadata describer. @@ -108,13 +108,13 @@ public class GeneralUniformAssignmentBuilder extends AbstractUniformAssignmentBu private final PartitionMovements partitionMovements; public GeneralUniformAssignmentBuilder(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) { - this.members = groupSpec.members(); + this.groupSpec = groupSpec; this.subscribedTopicDescriber = subscribedTopicDescriber; this.subscribedTopicIds = new HashSet<>(); this.membersPerTopic = new HashMap<>(); this.targetAssignment = new HashMap<>(); - members.forEach((memberId, memberMetadata) -> - memberMetadata.subscribedTopicIds().forEach(topicId -> { + groupSpec.memberIds().forEach(memberId -> + groupSpec.memberSubscription(memberId).subscribedTopicIds().forEach(topicId -> { // Check if the subscribed topic exists. int partitionCount = subscribedTopicDescriber.numPartitions(topicId); if (partitionCount == -1) { @@ -129,8 +129,8 @@ public GeneralUniformAssignmentBuilder(GroupSpec groupSpec, SubscribedTopicDescr ); this.unassignedPartitions = new HashSet<>(topicIdPartitions(subscribedTopicIds, subscribedTopicDescriber)); this.assignedStickyPartitions = new HashSet<>(); - this.assignmentManager = new AssignmentManager(this.members, this.subscribedTopicDescriber); - this.sortedMembersByAssignmentSize = assignmentManager.sortMembersByAssignmentSize(members.keySet()); + this.assignmentManager = new AssignmentManager(this.subscribedTopicDescriber); + this.sortedMembersByAssignmentSize = assignmentManager.sortMembersByAssignmentSize(groupSpec.memberIds()); this.partitionOwnerInTargetAssignment = new HashMap<>(); this.partitionMovements = new PartitionMovements(); } @@ -149,7 +149,6 @@ protected GroupAssignment buildAssignment() { return new GroupAssignment(Collections.emptyMap()); } - // All existing partitions are retained until max assignment size. assignStickyPartitions(); unassignedPartitionsAssignment(); @@ -191,9 +190,9 @@ private List sortTopicIdPartitions(Collection Partitions from topics that are still present in both the new subscriptions and the topic metadata. */ private void assignStickyPartitions() { - members.forEach((memberId, assignmentMemberSpec) -> - assignmentMemberSpec.assignedPartitions().forEach((topicId, currentAssignment) -> { - if (assignmentMemberSpec.subscribedTopicIds().contains(topicId)) { + groupSpec.memberIds().forEach(memberId -> + groupSpec.memberAssignment(memberId).forEach((topicId, currentAssignment) -> { + if (groupSpec.memberSubscription(memberId).subscribedTopicIds().contains(topicId)) { currentAssignment.forEach(partition -> { TopicIdPartition topicIdPartition = new TopicIdPartition(topicId, partition); assignmentManager.addPartitionToTargetAssignment(topicIdPartition, memberId); @@ -292,7 +291,7 @@ private boolean isBalanced() { continue; // Otherwise make sure it cannot get any more partitions. - for (Uuid topicId : members.get(member).subscribedTopicIds()) { + for (Uuid topicId : groupSpec.memberSubscription(member).subscribedTopicIds()) { Set assignedPartitions = targetAssignment.get(member).targetPartitions().get(topicId); for (int i = 0; i < subscribedTopicDescriber.numPartitions(topicId); i++) { TopicIdPartition topicIdPartition = new TopicIdPartition(topicId, i); @@ -332,7 +331,7 @@ private void balance() { unassignedPartitions.removeAll(fixedPartitions); // Narrow down the reassignment scope to only those members that are subject to reassignment. - for (String member : members.keySet()) { + for (String member : groupSpec.memberIds()) { if (!canMemberParticipateInReassignment(member)) { sortedMembersByAssignmentSize.remove(member); } @@ -411,7 +410,7 @@ private void reassignPartition(TopicIdPartition partition) { // Find the new member with the least assignment size. String newOwner = null; for (String anotherMember : sortedMembersByAssignmentSize) { - if (members.get(anotherMember).subscribedTopicIds().contains(partition.topicId())) { + if (groupSpec.memberSubscription(anotherMember).subscribedTopicIds().contains(partition.topicId())) { newOwner = anotherMember; break; } @@ -646,9 +645,11 @@ public String toString() { /** * Initializes an AssignmentManager, setting up the necessary data structures. */ - public AssignmentManager(Map members, SubscribedTopicDescriber subscribedTopicDescriber) { - members.forEach((memberId, member) -> { - int maxSize = member.subscribedTopicIds().stream() + public AssignmentManager( + SubscribedTopicDescriber subscribedTopicDescriber + ) { + groupSpec.memberIds().forEach(memberId -> { + int maxSize = groupSpec.memberSubscription(memberId).subscribedTopicIds().stream() .mapToInt(subscribedTopicDescriber::numPartitions) .sum(); @@ -734,7 +735,7 @@ private boolean maybeAssignPartitionToMember( String memberId ) { // If member is not subscribed to the partition's topic, return false without assigning. - if (!members.get(memberId).subscribedTopicIds().contains(topicIdPartition.topicId())) { + if (!groupSpec.memberSubscription(memberId).subscribedTopicIds().contains(topicIdPartition.topicId())) { return false; } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpec.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpec.java index 296dedb529..6a0dc509c5 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpec.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpec.java @@ -18,16 +18,18 @@ import org.apache.kafka.common.Uuid; +import java.util.Collection; import java.util.Map; +import java.util.Set; /** * The group metadata specifications required to compute the target assignment. */ public interface GroupSpec { /** - * @return Member metadata keyed by member Id. + * @return All the member Ids of the consumer group. */ - Map members(); + Collection memberIds(); /** * @return The group's subscription type. @@ -39,4 +41,22 @@ public interface GroupSpec { * False, otherwise. */ boolean isPartitionAssigned(Uuid topicId, int partitionId); + + /** + * Gets the member subscription specification for a member. + * + * @param memberId The member Id. + * @return The member's subscription metadata. + * @throws IllegalArgumentException If the member Id isn't found. + */ + MemberSubscriptionSpec memberSubscription(String memberId); + + /** + * Gets the current assignment of the member. + * + * @param memberId The member Id. + * @return A map of topic Ids to sets of partition numbers. + * An empty map is returned if the member Id isn't found. + */ + Map> memberAssignment(String memberId); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImpl.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImpl.java index 0194727c7d..3f5b1d8955 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImpl.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImpl.java @@ -18,20 +18,23 @@ import org.apache.kafka.common.Uuid; +import java.util.Collection; +import java.util.Collections; import java.util.Map; import java.util.Objects; +import java.util.Set; /** * The assignment specification for a consumer group. */ public class GroupSpecImpl implements GroupSpec { /** - * The member metadata keyed by member Id. + * Member subscription metadata keyed by member Id. */ - private final Map members; + private final Map memberSubscriptions; /** - * The subscription type followed by the group. + * The subscription type of the group. */ private final SubscriptionType subscriptionType; @@ -39,27 +42,24 @@ public class GroupSpecImpl implements GroupSpec { * Reverse lookup map representing topic partitions with * their current member assignments. */ - private final Map> invertedTargetAssignment; + private final Map> invertedMemberAssignment; public GroupSpecImpl( - Map members, + Map memberSubscriptions, SubscriptionType subscriptionType, - Map> invertedTargetAssignment + Map> invertedMemberAssignment ) { - Objects.requireNonNull(members); - Objects.requireNonNull(subscriptionType); - Objects.requireNonNull(invertedTargetAssignment); - this.members = members; - this.subscriptionType = subscriptionType; - this.invertedTargetAssignment = invertedTargetAssignment; + this.memberSubscriptions = Objects.requireNonNull(memberSubscriptions); + this.subscriptionType = Objects.requireNonNull(subscriptionType); + this.invertedMemberAssignment = Objects.requireNonNull(invertedMemberAssignment); } /** * {@inheritDoc} */ @Override - public Map members() { - return members; + public Collection memberIds() { + return memberSubscriptions.keySet(); } /** @@ -75,36 +75,60 @@ public SubscriptionType subscriptionType() { */ @Override public boolean isPartitionAssigned(Uuid topicId, int partitionId) { - Map partitionMap = invertedTargetAssignment.get(topicId); + Map partitionMap = invertedMemberAssignment.get(topicId); if (partitionMap == null) { return false; } return partitionMap.containsKey(partitionId); } + /** + * {@inheritDoc} + */ + @Override + public MemberSubscriptionSpec memberSubscription(String memberId) { + MemberSubscriptionSpec memberSubscription = memberSubscriptions.get(memberId); + if (memberSubscription == null) { + throw new IllegalArgumentException("Member Id " + memberId + " not found."); + } + return memberSubscription; + } + + /** + * {@inheritDoc} + */ + @Override + public Map> memberAssignment(String memberId) { + MemberSubscriptionSpecImpl memberSubscription = memberSubscriptions.get(memberId); + if (memberSubscription == null) { + return Collections.emptyMap(); + } + return memberSubscription.memberAssignment(); + } + @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; GroupSpecImpl that = (GroupSpecImpl) o; return subscriptionType == that.subscriptionType && - members.equals(that.members) && - invertedTargetAssignment.equals(that.invertedTargetAssignment); + memberSubscriptions.equals(that.memberSubscriptions) && + invertedMemberAssignment.equals(that.invertedMemberAssignment); } @Override public int hashCode() { - int result = members.hashCode(); + int result = memberSubscriptions.hashCode(); result = 31 * result + subscriptionType.hashCode(); - result = 31 * result + invertedTargetAssignment.hashCode(); + result = 31 * result + invertedMemberAssignment.hashCode(); return result; } @Override public String toString() { - return "GroupSpecImpl(members=" + members + + return "GroupSpecImpl(memberSubscriptions=" + memberSubscriptions + ", subscriptionType=" + subscriptionType + - ", invertedTargetAssignment=" + invertedTargetAssignment + + ", invertedMemberAssignment=" + invertedMemberAssignment + ')'; } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberSubscriptionSpec.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberSubscriptionSpec.java new file mode 100644 index 0000000000..382183a13c --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberSubscriptionSpec.java @@ -0,0 +1,41 @@ +/* + * 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.coordinator.group.assignor; + +import org.apache.kafka.common.Uuid; + +import java.util.Optional; +import java.util.Set; + +/** + * Interface representing the subscription metadata for a group member. + */ +public interface MemberSubscriptionSpec { + /** + * Gets the rack Id if present. + * + * @return An Optional containing the rack Id, or an empty Optional if not present. + */ + Optional rackId(); + + /** + * Gets the set of subscribed topic Ids. + * + * @return The set of subscribed topic Ids. + */ + Set subscribedTopicIds(); +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberSubscriptionSpecImpl.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberSubscriptionSpecImpl.java new file mode 100644 index 0000000000..00d86c367e --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberSubscriptionSpecImpl.java @@ -0,0 +1,91 @@ +/* + * 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.coordinator.group.assignor; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.coordinator.group.consumer.Assignment; + +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** + * Implementation of the {@link MemberSubscriptionSpec} interface. + */ +public class MemberSubscriptionSpecImpl implements MemberSubscriptionSpec { + private final Optional rackId; + private final Set subscribedTopicIds; + private final Assignment memberAssignment; + + /** + * Constructs a new {@code MemberSubscriptionSpecImpl}. + * + * @param rackId The rack Id. + * @param subscribedTopicIds The set of subscribed topic Ids. + * @param memberAssignment The current member assignment. + */ + public MemberSubscriptionSpecImpl( + Optional rackId, + Set subscribedTopicIds, + Assignment memberAssignment + ) { + this.rackId = Objects.requireNonNull(rackId); + this.subscribedTopicIds = Objects.requireNonNull(subscribedTopicIds); + this.memberAssignment = Objects.requireNonNull(memberAssignment); + } + + @Override + public Optional rackId() { + return rackId; + } + + @Override + public Set subscribedTopicIds() { + return subscribedTopicIds; + } + + public Map> memberAssignment() { + return memberAssignment.partitions(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + MemberSubscriptionSpecImpl that = (MemberSubscriptionSpecImpl) o; + return rackId.equals(that.rackId) && + subscribedTopicIds.equals(that.subscribedTopicIds) && + memberAssignment.equals(that.memberAssignment); + } + + @Override + public int hashCode() { + int result = rackId.hashCode(); + result = 31 * result + subscribedTopicIds.hashCode(); + result = 31 * result + memberAssignment.hashCode(); + return result; + } + + @Override + public String toString() { + return "MemberSubscriptionSpecImpl(rackId=" + rackId.orElse("N/A") + + ", subscribedTopicIds=" + subscribedTopicIds + + ", memberAssignment=" + memberAssignment + + ')'; + } +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java index 34e8256652..a97e0b98ba 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java @@ -101,9 +101,11 @@ private static boolean isImmutableMap(Map map) { OptimizedUniformAssignmentBuilder(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) { this.groupSpec = groupSpec; this.subscribedTopicDescriber = subscribedTopicDescriber; - this.subscribedTopicIds = new HashSet<>(groupSpec.members().values().iterator().next().subscribedTopicIds()); + this.subscribedTopicIds = new HashSet<>(groupSpec.memberSubscription(groupSpec.memberIds().iterator().next()) + .subscribedTopicIds()); this.unfilledMembers = new ArrayList<>(); this.unassignedPartitions = new ArrayList<>(); + this.targetAssignment = new HashMap<>(); } @@ -135,7 +137,7 @@ public GroupAssignment build() throws PartitionAssignorException { // Compute the minimum required quota per member and the number of members // that should receive an extra partition. - int numberOfMembers = groupSpec.members().size(); + int numberOfMembers = groupSpec.memberIds().size(); minimumMemberQuota = totalPartitionsCount / numberOfMembers; remainingMembersToGetAnExtraPartition = totalPartitionsCount % numberOfMembers; @@ -157,10 +159,8 @@ public GroupAssignment build() throws PartitionAssignorException { * altered. */ private void maybeRevokePartitions() { - for (Map.Entry entry : groupSpec.members().entrySet()) { - String memberId = entry.getKey(); - AssignmentMemberSpec assignmentMemberSpec = entry.getValue(); - Map> oldAssignment = assignmentMemberSpec.assignedPartitions(); + for (String memberId : groupSpec.memberIds()) { + Map> oldAssignment = groupSpec.memberAssignment(memberId); Map> newAssignment = null; // The assignor expects to receive the assignment as an immutable map. It leverages diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java index fe067901cb..d59bef034b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java @@ -81,7 +81,7 @@ public MemberWithRemainingAssignments(String memberId, int remaining) { * Returns a map of topic Ids to a list of members subscribed to them, * based on the given assignment specification and metadata. * - * @param groupSpec The specification for member assignments. + * @param groupSpec The specification required for group assignments. * @param subscribedTopicDescriber The metadata describer for subscribed topics and clusters. * @return A map of topic Ids to a list of member Ids subscribed to them. * @@ -92,11 +92,11 @@ private Map> membersPerTopic( final SubscribedTopicDescriber subscribedTopicDescriber ) { Map> membersPerTopic = new HashMap<>(); - Map membersData = groupSpec.members(); if (groupSpec.subscriptionType().equals(HOMOGENEOUS)) { - Set allMembers = membersData.keySet(); - Collection topics = membersData.values().iterator().next().subscribedTopicIds(); + Collection allMembers = groupSpec.memberIds(); + Collection topics = groupSpec.memberSubscription(groupSpec.memberIds().iterator().next()) + .subscribedTopicIds(); for (Uuid topicId : topics) { if (subscribedTopicDescriber.numPartitions(topicId) == -1) { @@ -105,8 +105,8 @@ private Map> membersPerTopic( membersPerTopic.put(topicId, allMembers); } } else { - membersData.forEach((memberId, memberMetadata) -> { - Collection topics = memberMetadata.subscribedTopicIds(); + groupSpec.memberIds().forEach(memberId -> { + Collection topics = groupSpec.memberSubscription(memberId).subscribedTopicIds(); for (Uuid topicId : topics) { if (subscribedTopicDescriber.numPartitions(topicId) == -1) { throw new PartitionAssignorException("Member is subscribed to a non-existent topic"); @@ -162,8 +162,8 @@ public GroupAssignment assign( List potentiallyUnfilledMembers = new ArrayList<>(); for (String memberId : membersForTopic) { - Set assignedPartitionsForTopic = groupSpec.members().get(memberId) - .assignedPartitions().getOrDefault(topicId, Collections.emptySet()); + Set assignedPartitionsForTopic = groupSpec.memberAssignment(memberId) + .getOrDefault(topicId, Collections.emptySet()); int currentAssignmentSize = assignedPartitionsForTopic.size(); List currentAssignmentListForTopic = new ArrayList<>(assignedPartitionsForTopic); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java index 648b0161d6..177c511b54 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java @@ -66,7 +66,7 @@ public GroupAssignment assign( GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber ) throws PartitionAssignorException { - if (groupSpec.members().isEmpty()) + if (groupSpec.memberIds().isEmpty()) return new GroupAssignment(Collections.emptyMap()); if (groupSpec.subscriptionType().equals(HOMOGENEOUS)) { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java index daea9938bf..38e6999383 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java @@ -18,8 +18,8 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.coordinator.group.CoordinatorRecord; -import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; import org.apache.kafka.coordinator.group.assignor.GroupSpecImpl; +import org.apache.kafka.coordinator.group.assignor.MemberSubscriptionSpecImpl; import org.apache.kafka.coordinator.group.assignor.SubscriptionType; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; @@ -293,14 +293,16 @@ public TargetAssignmentBuilder removeMember( * @throws PartitionAssignorException if the target assignment cannot be computed. */ public TargetAssignmentResult build() throws PartitionAssignorException { - Map memberSpecs = new HashMap<>(); + Map memberSpecs = new HashMap<>(); // Prepare the member spec for all members. - members.forEach((memberId, member) -> memberSpecs.put(memberId, createAssignmentMemberSpec( - member, - targetAssignment.getOrDefault(memberId, Assignment.EMPTY), - topicsImage - ))); + members.forEach((memberId, member) -> + memberSpecs.put(memberId, createMemberSubscriptionSpecImpl( + member, + targetAssignment.getOrDefault(memberId, Assignment.EMPTY), + topicsImage + )) + ); // Update the member spec if updated or deleted members. updatedMembers.forEach((memberId, updatedMemberOrNull) -> { @@ -317,7 +319,7 @@ public TargetAssignmentResult build() throws PartitionAssignorException { } } - memberSpecs.put(memberId, createAssignmentMemberSpec( + memberSpecs.put(memberId, createMemberSubscriptionSpecImpl( updatedMemberOrNull, assignment, topicsImage @@ -381,16 +383,16 @@ private Assignment newMemberAssignment( } } - static AssignmentMemberSpec createAssignmentMemberSpec( + // private for testing + static MemberSubscriptionSpecImpl createMemberSubscriptionSpecImpl( ConsumerGroupMember member, - Assignment targetAssignment, + Assignment memberAssignment, TopicsImage topicsImage ) { - return new AssignmentMemberSpec( - Optional.ofNullable(member.instanceId()), + return new MemberSubscriptionSpecImpl( Optional.ofNullable(member.rackId()), new TopicIds(member.subscribedTopicNames(), topicsImage), - targetAssignment.partitions() + memberAssignment ); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TopicIds.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TopicIds.java index 8485c1f560..744f3a86b7 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TopicIds.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TopicIds.java @@ -173,4 +173,11 @@ public int hashCode() { result = 31 * result + image.hashCode(); return result; } + + @Override + public String toString() { + return "TopicIds(topicNames=" + topicNames + + ", image=" + image + + ')'; + } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java index ffc5455cea..5042c8a759 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java @@ -17,8 +17,8 @@ package org.apache.kafka.coordinator.group; import org.apache.kafka.common.Uuid; -import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; +import org.apache.kafka.coordinator.group.assignor.MemberSubscriptionSpecImpl; import java.util.AbstractMap; import java.util.Arrays; @@ -86,21 +86,20 @@ public static void assertAssignment( } /** - * Generate a reverse look up map of partition to member target assignments from the given member spec. + * Generate a reverse look up map of partition to member target assignments from the given metadata. * - * @param memberSpec A map where the key is the member Id and the value is an - * AssignmentMemberSpec object containing the member's partition assignments. + * @param members The member subscription specs. * @return Map of topic partition to member assignments. */ public static Map> invertedTargetAssignment( - Map memberSpec + Map members ) { Map> invertedTargetAssignment = new HashMap<>(); - for (Map.Entry memberEntry : memberSpec.entrySet()) { + for (Map.Entry memberEntry : members.entrySet()) { String memberId = memberEntry.getKey(); - Map> topicsAndPartitions = memberEntry.getValue().assignedPartitions(); + Map> memberAssignment = memberEntry.getValue().memberAssignment(); - for (Map.Entry> topicEntry : topicsAndPartitions.entrySet()) { + for (Map.Entry> topicEntry : memberAssignment.entrySet()) { Uuid topicId = topicEntry.getKey(); Set partitions = topicEntry.getValue(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/NoOpPartitionAssignor.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/NoOpPartitionAssignor.java index 2cf8309a72..460929506d 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/NoOpPartitionAssignor.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/NoOpPartitionAssignor.java @@ -22,7 +22,6 @@ import org.apache.kafka.coordinator.group.assignor.MemberAssignment; import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber; -import java.util.Map; import java.util.stream.Collectors; public class NoOpPartitionAssignor implements ConsumerGroupPartitionAssignor { @@ -35,11 +34,11 @@ public String name() { @Override public GroupAssignment assign(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) { - return new GroupAssignment(groupSpec.members().entrySet() + return new GroupAssignment(groupSpec.memberIds() .stream() .collect(Collectors.toMap( - Map.Entry::getKey, - entry -> new MemberAssignment(entry.getValue().assignedPartitions()) + memberId -> memberId, + memberId -> new MemberAssignment(groupSpec.memberAssignment(memberId)) ))); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilderTest.java index f5a7be24bf..c78a3f78a9 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilderTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.coordinator.group.assignor; import org.apache.kafka.common.Uuid; +import org.apache.kafka.coordinator.group.consumer.Assignment; import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; import org.junit.jupiter.api.Test; @@ -66,18 +67,16 @@ public void testTwoMembersNoTopicSubscription() { ) ); - Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + Map members = new TreeMap<>(); + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.emptySet(), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.emptySet(), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( @@ -108,18 +107,16 @@ public void testTwoMembersSubscribedToNonexistentTopics() { ) ); - Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + Map members = new TreeMap<>(); + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic2Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( @@ -148,18 +145,18 @@ public void testFirstAssignmentTwoMembersTwoTopics() { mkMapOfPartitionRacks(6) )); - Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + Map members = new TreeMap<>(); + + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( @@ -202,24 +199,24 @@ public void testFirstAssignmentNumMembersGreaterThanTotalNumPartitions() { mkMapOfPartitionRacks(2) )); - Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + Map members = new TreeMap<>(); + + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(memberC, new AssignmentMemberSpec( - Optional.empty(), + + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic1Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( @@ -271,45 +268,33 @@ public void testReassignmentForTwoMembersThreeTopicsGivenUnbalancedPrevAssignmen mkMapOfPartitionRacks(4) )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1, 2) - ) - ); - members.put(memberA, new AssignmentMemberSpec( + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), - Optional.of("rack0"), Collections.singleton(topic1Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 1, 2) + )) )); - Map> currentAssignmentForB = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 3), - mkTopicAssignment(topic2Uuid, 0) - ) - ); - members.put(memberB, new AssignmentMemberSpec( + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), - Optional.of("rack1"), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 3), + mkTopicAssignment(topic2Uuid, 0) + )) )); - Map> currentAssignmentForC = new TreeMap<>( - mkAssignment( + members.put(memberC, new MemberSubscriptionSpecImpl( + Optional.empty(), + mkSet(topic1Uuid, topic2Uuid, topic3Uuid), + new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 4, 5), mkTopicAssignment(topic2Uuid, 1, 2, 3), mkTopicAssignment(topic3Uuid, 0, 1, 2, 3) - ) - ); - members.put(memberC, new AssignmentMemberSpec( - Optional.empty(), - Optional.of("rack2"), - mkSet(topic1Uuid, topic2Uuid, topic3Uuid), - currentAssignmentForC + )) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -368,32 +353,24 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembers() { mkMapOfPartitionRacks(3) )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1, 2, 3), - mkTopicAssignment(topic3Uuid, 0, 1) - ) - ); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 1, 2, 3), + mkTopicAssignment(topic3Uuid, 0, 1) + )) )); - Map> currentAssignmentForB = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic2Uuid, 0, 1, 2), - mkTopicAssignment(topic4Uuid, 0, 1, 2) - ) - ); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid, topic3Uuid, topic4Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic2Uuid, 0, 1, 2), + mkTopicAssignment(topic4Uuid, 0, 1, 2) + )) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -438,36 +415,31 @@ public void testReassignmentWhenOneMemberAddedAndPartitionsAddedTwoMembersTwoTop mkMapOfPartitionRacks(7) )); - Map members = new HashMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = new TreeMap<>(mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic2Uuid, 0) - )); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic1Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 2), + mkTopicAssignment(topic2Uuid, 0) + )) )); - Map> currentAssignmentForB = new TreeMap<>(mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1, 2) - )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1, 2) + )) )); // Add a new member to trigger a re-assignment. - members.put(memberC, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( @@ -519,27 +491,23 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithThreeM mkMapOfPartitionRacks(3) )); - Map members = new HashMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1, 2), - mkTopicAssignment(topic3Uuid, 0, 1) - ); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 1, 2), + mkTopicAssignment(topic3Uuid, 0, 1) + )) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic2Uuid, 3, 4, 5, 6) - ); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic2Uuid, 3, 4, 5, 6) + )) )); // Member C was removed @@ -585,28 +553,24 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith )); // Initial subscriptions were [T1, T2] - Map members = new HashMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic2Uuid, 1, 3) - ); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic1Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 2), + mkTopicAssignment(topic2Uuid, 1, 3) + )) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 0, 2, 4) - ); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 0, 2, 4) + )) )); GroupSpec groupSpec = new GroupSpecImpl( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java index 4060b1a4f1..db1860c504 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java @@ -17,23 +17,26 @@ package org.apache.kafka.coordinator.group.assignor; import org.apache.kafka.common.Uuid; +import org.apache.kafka.coordinator.group.consumer.Assignment; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Optional; +import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class GroupSpecImplTest { - - private Map members; + private static final String TEST_MEMBER = "test-member"; + private Map members; private SubscriptionType subscriptionType; private Map> invertedTargetAssignment; private GroupSpecImpl groupSpec; @@ -42,17 +45,15 @@ public class GroupSpecImplTest { @BeforeEach void setUp() { members = new HashMap<>(); - subscriptionType = SubscriptionType.HOMOGENEOUS; invertedTargetAssignment = new HashMap<>(); topicId = Uuid.randomUuid(); - members.put("test-member", new AssignmentMemberSpec( + members.put(TEST_MEMBER, new MemberSubscriptionSpecImpl( Optional.empty(), - Optional.empty(), - new HashSet<>(Collections.singletonList(topicId)), - Collections.emptyMap()) - ); + mkSet(topicId), + Assignment.EMPTY + )); groupSpec = new GroupSpecImpl( members, @@ -62,8 +63,8 @@ void setUp() { } @Test - void testMembers() { - assertEquals(members, groupSpec.members()); + void testMemberIds() { + assertEquals(members.keySet(), groupSpec.memberIds()); } @Test @@ -81,4 +82,27 @@ void testIsPartitionAssigned() { assertFalse(groupSpec.isPartitionAssigned(topicId, 2)); assertFalse(groupSpec.isPartitionAssigned(Uuid.randomUuid(), 2)); } + + @Test + void testMemberSubscription() { + assertEquals(members.get(TEST_MEMBER), groupSpec.memberSubscription(TEST_MEMBER)); + assertThrows(IllegalArgumentException.class, () -> groupSpec.memberSubscription("unknown-member")); + } + + @Test + void testMemberAssignment() { + Map> topicPartitions = new HashMap<>(); + topicPartitions.put( + topicId, + mkSet(0, 1) + ); + members.put(TEST_MEMBER, new MemberSubscriptionSpecImpl( + Optional.empty(), + mkSet(topicId), + new Assignment(topicPartitions) + )); + + assertEquals(topicPartitions, groupSpec.memberAssignment(TEST_MEMBER)); + assertEquals(Collections.emptyMap(), groupSpec.memberAssignment("unknown-member")); + } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java index fdc4f5941f..878e142cb8 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.coordinator.group.assignor; import org.apache.kafka.common.Uuid; +import org.apache.kafka.coordinator.group.consumer.Assignment; import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; import org.junit.jupiter.api.Test; @@ -69,13 +70,12 @@ public void testOneMemberNoTopicSubscription() { ) ); - Map members = Collections.singletonMap( + Map members = Collections.singletonMap( memberA, - new AssignmentMemberSpec( - Optional.empty(), + new MemberSubscriptionSpecImpl( Optional.empty(), Collections.emptySet(), - Collections.emptyMap() + Assignment.EMPTY ) ); @@ -107,13 +107,12 @@ public void testOneMemberSubscribedToNonexistentTopic() { ) ); - Map members = Collections.singletonMap( + Map members = Collections.singletonMap( memberA, - new AssignmentMemberSpec( - Optional.empty(), + new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic2Uuid), - Collections.emptyMap() + Assignment.EMPTY ) ); @@ -143,18 +142,18 @@ public void testFirstAssignmentTwoMembersTwoTopicsNoMemberRacks() { mkMapOfPartitionRacks(2) )); - Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + Map members = new TreeMap<>(); + + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); Map>> expectedAssignment = new HashMap<>(); @@ -192,24 +191,24 @@ public void testFirstAssignmentNumMembersGreaterThanTotalNumPartitions() { mkMapOfPartitionRacks(2) )); - Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + Map members = new TreeMap<>(); + + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(memberC, new AssignmentMemberSpec( - Optional.empty(), + + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); // Topic 3 has 2 partitions but three members subscribed to it - one of them should not get an assignment. @@ -253,13 +252,12 @@ public void testValidityAndBalanceForLargeSampleSet() { )); } - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); for (int i = 1; i < 50; i++) { - members.put("member" + i, new AssignmentMemberSpec( - Optional.empty(), + members.put("member" + i, new MemberSubscriptionSpecImpl( Optional.empty(), topicMetadata.keySet(), - Collections.emptyMap() + Assignment.EMPTY )); } @@ -294,26 +292,24 @@ public void testReassignmentForTwoMembersTwoTopicsGivenUnbalancedPrevAssignment( mkMapOfPartitionRacks(3) )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - mkOrderedAssignment( + new Assignment(mkOrderedAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic2Uuid, 0, 1) - ) + )) )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - mkOrderedAssignment( + new Assignment(mkOrderedAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) - ) + )) )); Map>> expectedAssignment = new HashMap<>(); @@ -359,26 +355,24 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembersTwoTopics() { mkMapOfPartitionRacks(5) )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - mkOrderedAssignment( + new Assignment(mkOrderedAssignment( mkTopicAssignment(topic1Uuid, 0, 2), mkTopicAssignment(topic2Uuid, 0) - ) + )) )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - mkOrderedAssignment( + new Assignment(mkOrderedAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1, 2) - ) + )) )); Map>> expectedAssignment = new HashMap<>(); @@ -423,34 +417,31 @@ public void testReassignmentWhenOneMemberAddedAfterInitialAssignmentWithTwoMembe mkMapOfPartitionRacks(3) )); - Map members = new HashMap<>(); + Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - mkOrderedAssignment( + new Assignment(mkOrderedAssignment( mkTopicAssignment(topic1Uuid, 0, 2), mkTopicAssignment(topic2Uuid, 0) - ) + )) )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - mkOrderedAssignment( + new Assignment(mkOrderedAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1, 2) - ) + )) )); // Add a new member to trigger a re-assignment. - members.put(memberC, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - Collections.emptyMap() + Assignment.EMPTY )); Map>> expectedAssignment = new HashMap<>(); @@ -497,26 +488,24 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithThreeM mkMapOfPartitionRacks(3) )); - Map members = new HashMap<>(); + Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - mkAssignment( + new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic2Uuid, 0) - ) + )) )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - mkAssignment( + new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1) - ) + )) )); // Member C was removed @@ -564,26 +553,24 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith )); // Initial subscriptions were [T1, T2] - Map members = new HashMap<>(); + Map members = new TreeMap<>(); - members.put(memberA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic2Uuid), - mkAssignment( + new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic2Uuid, 0) - ) + )) )); - members.put(memberB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), Collections.singleton(topic2Uuid), - mkAssignment( + new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1) - ) + )) )); Map>> expectedAssignment = new HashMap<>(); @@ -616,14 +603,14 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith * - each member is subscribed to topics of all partitions assigned to it, and * - each partition is assigned to no more than one member. * Balance requirements: - * - the assignment is fully balanced (the numbers of topic partitions assigned to members differ by at most one), or + * - the assignment is fully balanced (the numbers of topic partitions assigned to memberSubscriptionSpec differ by at most one), or * - there is no topic partition that can be moved from one member to another with 2+ fewer topic partitions. * - * @param members Members data structure from the assignment Spec. - * @param computedGroupAssignment Assignment computed by the uniform assignor. + * @param memberSubscriptionSpec Members subscription metadata structure from the group Spec. + * @param computedGroupAssignment Assignment computed by the uniform assignor. */ private void checkValidityAndBalance( - Map members, + Map memberSubscriptionSpec, GroupAssignment computedGroupAssignment ) { List membersList = new ArrayList<>(computedGroupAssignment.members().keySet()); @@ -643,9 +630,9 @@ private void checkValidityAndBalance( // Each member is subscribed to topics of all the partitions assigned to it. computedAssignmentForMember.keySet().forEach(topicId -> { // Check if the topic exists in the subscription. - assertTrue(members.get(memberId).subscribedTopicIds().contains(topicId), - "Error: Partitions for topic " + topicId + " are assigned to member " + memberId + - " but it is not part of the members subscription "); + assertTrue(memberSubscriptionSpec.get(memberId).subscribedTopicIds().contains(topicId), + "Error: Partitions for topic " + topicId + " are assigned to member " + memberId + + " but it is not part of the members subscription "); }); for (int j = i + 1; j < numMembers; j++) { diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java index 7acb9bd5f7..20b97490c2 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.coordinator.group.assignor; import org.apache.kafka.common.Uuid; - +import org.apache.kafka.coordinator.group.consumer.Assignment; import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; import org.junit.jupiter.api.Test; @@ -47,9 +47,9 @@ public class RangeAssignorTest { private final String topic2Name = "topic2"; private final Uuid topic3Uuid = Uuid.randomUuid(); private final String topic3Name = "topic3"; - private final String consumerA = "A"; - private final String consumerB = "B"; - private final String consumerC = "C"; + private final String memberA = "A"; + private final String memberB = "B"; + private final String memberC = "C"; @Test public void testOneConsumerNoTopic() { @@ -60,18 +60,17 @@ public void testOneConsumerNoTopic() { topic1Uuid, topic1Name, 3, - createPartitionRacks(3) + Collections.emptyMap() ) ) ); - Map members = Collections.singletonMap( - consumerA, - new AssignmentMemberSpec( - Optional.empty(), + Map members = Collections.singletonMap( + memberA, + new MemberSubscriptionSpecImpl( Optional.empty(), Collections.emptySet(), - Collections.emptyMap() + Assignment.EMPTY ) ); @@ -98,18 +97,17 @@ public void testOneConsumerSubscribedToNonExistentTopic() { topic1Uuid, topic1Name, 3, - createPartitionRacks(3) + Collections.emptyMap() ) ) ); - Map members = Collections.singletonMap( - consumerA, - new AssignmentMemberSpec( + Map members = Collections.singletonMap( + memberA, + new MemberSubscriptionSpecImpl( Optional.empty(), - Optional.empty(), - Collections.singleton(topic2Uuid), - Collections.emptyMap() + mkSet(topic2Uuid), + Assignment.EMPTY ) ); @@ -130,35 +128,33 @@ public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() { topic1Uuid, topic1Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); topicMetadata.put(topic3Uuid, new TopicMetadata( topic3Uuid, topic3Name, 2, - createPartitionRacks(2) + Collections.emptyMap() )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - members.put(consumerA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + invertedTargetAssignment(members) ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); @@ -168,11 +164,11 @@ public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() { ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( + expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic3Uuid, 0) )); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic3Uuid, 1) )); @@ -187,48 +183,45 @@ public void testFirstAssignmentThreeConsumersThreeTopicsDifferentSubscriptions() topic1Uuid, topic1Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); topicMetadata.put(topic2Uuid, new TopicMetadata( topic2Uuid, topic2Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); topicMetadata.put(topic3Uuid, new TopicMetadata( topic3Uuid, topic3Name, 2, - createPartitionRacks(2) + Collections.emptyMap() )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - members.put(consumerA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), - Collections.singleton(topic3Uuid), - Collections.emptyMap() + mkSet(topic3Uuid), + Assignment.EMPTY )); - members.put(consumerC, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic2Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( members, HETEROGENEOUS, - Collections.emptyMap() + invertedTargetAssignment(members) ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); @@ -238,14 +231,14 @@ public void testFirstAssignmentThreeConsumersThreeTopicsDifferentSubscriptions() ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( + expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1, 2), mkTopicAssignment(topic2Uuid, 0, 1) )); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic3Uuid, 0) )); - expectedAssignment.put(consumerC, mkAssignment( + expectedAssignment.put(memberC, mkAssignment( mkTopicAssignment(topic2Uuid, 2), mkTopicAssignment(topic3Uuid, 1) )); @@ -260,42 +253,39 @@ public void testFirstAssignmentNumConsumersGreaterThanNumPartitions() { topic1Uuid, topic1Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); topicMetadata.put(topic3Uuid, new TopicMetadata( topic3Uuid, topic3Name, 2, - createPartitionRacks(2) + Collections.emptyMap() )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - members.put(consumerA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); - members.put(consumerC, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic3Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + invertedTargetAssignment(members) ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); @@ -306,15 +296,15 @@ public void testFirstAssignmentNumConsumersGreaterThanNumPartitions() { Map>> expectedAssignment = new HashMap<>(); // Topic 3 has 2 partitions but three consumers subscribed to it - one of them will not get a partition. - expectedAssignment.put(consumerA, mkAssignment( + expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic3Uuid, 0) )); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic3Uuid, 1) )); - expectedAssignment.put(consumerC, mkAssignment( + expectedAssignment.put(memberC, mkAssignment( mkTopicAssignment(topic1Uuid, 2) )); @@ -328,45 +318,40 @@ public void testReassignmentNumConsumersGreaterThanNumPartitionsWhenOneConsumerA topic1Uuid, topic1Name, 2, - createPartitionRacks(2) + Collections.emptyMap() )); topicMetadata.put(topic2Uuid, new TopicMetadata( topic2Uuid, topic2Name, 2, - createPartitionRacks(2) + Collections.emptyMap() )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0), - mkTopicAssignment(topic2Uuid, 0) - ); - members.put(consumerA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic2Uuid, 0) + )) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1) - ); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1) + )) )); // Add a new consumer to trigger a re-assignment - members.put(consumerC, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( @@ -382,17 +367,17 @@ public void testReassignmentNumConsumersGreaterThanNumPartitionsWhenOneConsumerA ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( + expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic2Uuid, 0) )); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1) )); // Consumer C shouldn't get any assignment, due to stickiness A, B retain their assignments - assertNull(computedAssignment.members().get(consumerC)); + assertNull(computedAssignment.members().get(memberC)); assertAssignment(expectedAssignment, computedAssignment); } @@ -404,37 +389,33 @@ public void testReassignmentWhenOnePartitionAddedForTwoConsumersTwoTopics() { topic1Uuid, topic1Name, 4, - createPartitionRacks(4) + Collections.emptyMap() )); topicMetadata.put(topic2Uuid, new TopicMetadata( topic2Uuid, topic2Name, 4, - createPartitionRacks(4) + Collections.emptyMap() )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1), - mkTopicAssignment(topic2Uuid, 0, 1) - ); - members.put(consumerA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 1), + mkTopicAssignment(topic2Uuid, 0, 1) + )) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 2), - mkTopicAssignment(topic2Uuid, 2) - ); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 2), + mkTopicAssignment(topic2Uuid, 2) + )) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -450,11 +431,11 @@ public void testReassignmentWhenOnePartitionAddedForTwoConsumersTwoTopics() { ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( + expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic2Uuid, 0, 1) )); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic1Uuid, 2, 3), mkTopicAssignment(topic2Uuid, 2, 3) )); @@ -469,45 +450,40 @@ public void testReassignmentWhenOneConsumerAddedAfterInitialAssignmentWithTwoCon topic1Uuid, topic1Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); topicMetadata.put(topic2Uuid, new TopicMetadata( topic2Uuid, topic2Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1), - mkTopicAssignment(topic2Uuid, 0, 1) - ); - members.put(consumerA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 1), + mkTopicAssignment(topic2Uuid, 0, 1) + )) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 2), - mkTopicAssignment(topic2Uuid, 2) - ); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 2), + mkTopicAssignment(topic2Uuid, 2) + )) )); // Add a new consumer to trigger a re-assignment - members.put(consumerC, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - Collections.emptyMap() + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( @@ -523,15 +499,15 @@ public void testReassignmentWhenOneConsumerAddedAfterInitialAssignmentWithTwoCon ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( + expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic2Uuid, 0) )); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) )); - expectedAssignment.put(consumerC, mkAssignment( + expectedAssignment.put(memberC, mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1) )); @@ -547,45 +523,40 @@ public void testReassignmentWhenOneConsumerAddedAndOnePartitionAfterInitialAssig topic1Uuid, topic1Name, 4, - createPartitionRacks(4) + Collections.emptyMap() )); topicMetadata.put(topic2Uuid, new TopicMetadata( topic2Uuid, topic2Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1), - mkTopicAssignment(topic2Uuid, 0, 1) - ); - members.put(consumerA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForA + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 1), + mkTopicAssignment(topic2Uuid, 0, 1) + )) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 2), - mkTopicAssignment(topic2Uuid, 2) - ); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 2), + mkTopicAssignment(topic2Uuid, 2) + )) )); // Add a new consumer to trigger a re-assignment - members.put(consumerC, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), - Collections.singleton(topic1Uuid), - Collections.emptyMap() + mkSet(topic1Uuid), + Assignment.EMPTY )); GroupSpec groupSpec = new GroupSpecImpl( @@ -601,15 +572,15 @@ public void testReassignmentWhenOneConsumerAddedAndOnePartitionAfterInitialAssig ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( + expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic2Uuid, 0, 1) )); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) )); - expectedAssignment.put(consumerC, mkAssignment( + expectedAssignment.put(memberC, mkAssignment( mkTopicAssignment(topic1Uuid, 3) )); @@ -623,27 +594,26 @@ public void testReassignmentWhenOneConsumerRemovedAfterInitialAssignmentWithTwoC topic1Uuid, topic1Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); topicMetadata.put(topic2Uuid, new TopicMetadata( topic2Uuid, topic2Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); + // Consumer A was removed - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 2), - mkTopicAssignment(topic2Uuid, 2) - ); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 2), + mkTopicAssignment(topic2Uuid, 2) + )) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -659,7 +629,7 @@ public void testReassignmentWhenOneConsumerRemovedAfterInitialAssignmentWithTwoC ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1, 2), mkTopicAssignment(topic2Uuid, 0, 1, 2) )); @@ -674,55 +644,49 @@ public void testReassignmentWhenMultipleSubscriptionsRemovedAfterInitialAssignme topic1Uuid, topic1Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); topicMetadata.put(topic2Uuid, new TopicMetadata( topic2Uuid, topic2Name, 3, - createPartitionRacks(3) + Collections.emptyMap() )); topicMetadata.put(topic3Uuid, new TopicMetadata( topic3Uuid, topic3Name, 2, - createPartitionRacks(2) + Collections.emptyMap() )); // Let initial subscriptions be A -> T1, T2 // B -> T2 // C -> T2, T3 // Change the subscriptions to A -> T1 // B -> T1, T2, T3 // C -> T2 - Map members = new TreeMap<>(); + Map members = new TreeMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1, 2), - mkTopicAssignment(topic2Uuid, 0) - ); - members.put(consumerA, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberA, new MemberSubscriptionSpecImpl( Optional.empty(), - Collections.singleton(topic1Uuid), - currentAssignmentForA + mkSet(topic1Uuid), + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 1, 2), + mkTopicAssignment(topic2Uuid, 0) + )) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic2Uuid, 1) - ); - members.put(consumerB, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberB, new MemberSubscriptionSpecImpl( Optional.empty(), mkSet(topic1Uuid, topic2Uuid, topic3Uuid), - currentAssignmentForB + new Assignment(mkAssignment( + mkTopicAssignment(topic2Uuid, 1) + )) )); - Map> currentAssignmentForC = mkAssignment( - mkTopicAssignment(topic2Uuid, 2), - mkTopicAssignment(topic3Uuid, 0, 1) - ); - members.put(consumerC, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberC, new MemberSubscriptionSpecImpl( Optional.empty(), - Collections.singleton(topic2Uuid), - currentAssignmentForC + mkSet(topic2Uuid), + new Assignment(mkAssignment( + mkTopicAssignment(topic2Uuid, 2), + mkTopicAssignment(topic3Uuid, 0, 1) + )) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -738,15 +702,15 @@ public void testReassignmentWhenMultipleSubscriptionsRemovedAfterInitialAssignme ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( + expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1) )); - expectedAssignment.put(consumerB, mkAssignment( + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 0, 1), mkTopicAssignment(topic3Uuid, 0, 1) )); - expectedAssignment.put(consumerC, mkAssignment( + expectedAssignment.put(memberC, mkAssignment( mkTopicAssignment(topic2Uuid, 2) )); @@ -763,14 +727,4 @@ private void assertAssignment( assertEquals(expectedAssignment.get(memberId), computedAssignmentForMember); } } - - // When rack awareness is enabled for this assignor, rack information can be updated in this method. - private static Map> createPartitionRacks(int numPartitions) { - Map> partitionRacks = new HashMap<>(numPartitions); - Set emptySet = Collections.emptySet(); - for (int i = 0; i < numPartitions; i++) { - partitionRacks.put(i, emptySet); - } - return partitionRacks; - } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java index e2e572b6bf..5c7557cf5e 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java @@ -19,8 +19,9 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.coordinator.group.AssignmentTestUtil; import org.apache.kafka.coordinator.group.MetadataImageBuilder; -import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; import org.apache.kafka.coordinator.group.assignor.GroupSpecImpl; +import org.apache.kafka.coordinator.group.assignor.MemberSubscriptionSpec; +import org.apache.kafka.coordinator.group.assignor.MemberSubscriptionSpecImpl; import org.apache.kafka.coordinator.group.assignor.SubscriptionType; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; @@ -44,7 +45,7 @@ import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpers.newTargetAssignmentRecord; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpersTest.mkMapOfPartitionRacks; import static org.apache.kafka.coordinator.group.assignor.SubscriptionType.HOMOGENEOUS; -import static org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder.createAssignmentMemberSpec; +import static org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder.createMemberSubscriptionSpecImpl; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; @@ -164,22 +165,22 @@ public void prepareMemberAssignment( public TargetAssignmentBuilder.TargetAssignmentResult build() { TopicsImage topicsImage = topicsImageBuilder.build().topics(); // Prepare expected member specs. - Map memberSpecs = new HashMap<>(); + Map memberSubscriptions = new HashMap<>(); // All the existing members are prepared. members.forEach((memberId, member) -> - memberSpecs.put(memberId, createAssignmentMemberSpec( + memberSubscriptions.put(memberId, createMemberSubscriptionSpecImpl( member, targetAssignment.getOrDefault(memberId, Assignment.EMPTY), topicsImage - ) - )); + )) + ); // All the updated are added and all the deleted // members are removed. updatedMembers.forEach((memberId, updatedMemberOrNull) -> { if (updatedMemberOrNull == null) { - memberSpecs.remove(memberId); + memberSubscriptions.remove(memberId); } else { Assignment assignment = targetAssignment.getOrDefault(memberId, Assignment.EMPTY); @@ -191,7 +192,7 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { } } - memberSpecs.put(memberId, createAssignmentMemberSpec( + memberSubscriptions.put(memberId, createMemberSubscriptionSpecImpl( updatedMemberOrNull, assignment, topicsImage @@ -209,10 +210,15 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { SubscriptionType subscriptionType = HOMOGENEOUS; // Prepare the member assignments per topic partition. - Map> invertedTargetAssignment = AssignmentTestUtil.invertedTargetAssignment(memberSpecs); + Map> invertedTargetAssignment = AssignmentTestUtil + .invertedTargetAssignment(memberSubscriptions); // Prepare the expected assignment spec. - GroupSpecImpl groupSpec = new GroupSpecImpl(memberSpecs, subscriptionType, invertedTargetAssignment); + GroupSpecImpl groupSpec = new GroupSpecImpl( + memberSubscriptions, + subscriptionType, + invertedTargetAssignment + ); // We use `any` here to always return an assignment but use `verify` later on // to ensure that the input was correct. @@ -251,7 +257,7 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { } @Test - public void testCreateAssignmentMemberSpec() { + public void testCreateMemberSubscriptionSpecImpl() { Uuid fooTopicId = Uuid.randomUuid(); Uuid barTopicId = Uuid.randomUuid(); TopicsImage topicsImage = new MetadataImageBuilder() @@ -271,18 +277,17 @@ public void testCreateAssignmentMemberSpec() { mkTopicAssignment(barTopicId, 1, 2, 3) )); - AssignmentMemberSpec assignmentMemberSpec = createAssignmentMemberSpec( + MemberSubscriptionSpec subscriptionSpec = createMemberSubscriptionSpecImpl( member, assignment, topicsImage ); - assertEquals(new AssignmentMemberSpec( - Optional.of("instanceId"), + assertEquals(new MemberSubscriptionSpecImpl( Optional.of("rackId"), new TopicIds(mkSet("bar", "foo", "zar"), topicsImage), - assignment.partitions() - ), assignmentMemberSpec); + assignment + ), subscriptionSpec); } @Test diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java index 77a38ab7f4..0b1681e0d1 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java @@ -17,14 +17,15 @@ package org.apache.kafka.jmh.assignor; import org.apache.kafka.common.Uuid; -import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; import org.apache.kafka.coordinator.group.assignor.GroupSpecImpl; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; +import org.apache.kafka.coordinator.group.assignor.MemberSubscriptionSpecImpl; import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; import org.apache.kafka.coordinator.group.assignor.RangeAssignor; import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber; import org.apache.kafka.coordinator.group.assignor.SubscriptionType; +import org.apache.kafka.coordinator.group.consumer.Assignment; import org.apache.kafka.coordinator.group.consumer.TopicIds; import org.apache.kafka.coordinator.group.assignor.UniformAssignor; import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata; @@ -134,7 +135,7 @@ public void setup() { Map topicMetadata = createTopicMetadata(); subscribedTopicDescriber = new SubscribedTopicMetadata(topicMetadata); - createAssignmentSpec(); + createGroupSpec(); partitionAssignor = assignorType.assignor(); @@ -175,8 +176,8 @@ private Map createTopicMetadata() { return topicMetadata; } - private void createAssignmentSpec() { - Map members = new HashMap<>(); + private void createGroupSpec() { + Map members = new HashMap<>(); // In the rebalance case, we will add the last member as a trigger. // This is done to keep the total members count consistent with the input. @@ -215,7 +216,11 @@ private void createAssignmentSpec() { } } - this.groupSpec = new GroupSpecImpl(members, subscriptionType, Collections.emptyMap()); + this.groupSpec = new GroupSpecImpl( + members, + subscriptionType, + Collections.emptyMap() + ); } private Optional rackId(int memberIndex) { @@ -223,18 +228,17 @@ private Optional rackId(int memberIndex) { } private void addMemberSpec( - Map members, + Map members, int memberIndex, Set subscribedTopicIds ) { String memberId = "member" + memberIndex; Optional rackId = rackId(memberIndex); - members.put(memberId, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberId, new MemberSubscriptionSpecImpl( rackId, subscribedTopicIds, - Collections.emptyMap() + Assignment.EMPTY )); } @@ -256,38 +260,40 @@ private void simulateIncrementalRebalance() { Map> invertedTargetAssignment = AssignorBenchmarkUtils.computeInvertedTargetAssignment(initialAssignment); - Map updatedMembers = new HashMap<>(); + Map updatedMemberSpec = new HashMap<>(); - groupSpec.members().forEach((memberId, assignmentMemberSpec) -> { + for (String memberId : groupSpec.memberIds()) { MemberAssignment memberAssignment = members.getOrDefault( memberId, new MemberAssignment(Collections.emptyMap()) ); - updatedMembers.put(memberId, new AssignmentMemberSpec( - assignmentMemberSpec.instanceId(), - assignmentMemberSpec.rackId(), - assignmentMemberSpec.subscribedTopicIds(), - Collections.unmodifiableMap(memberAssignment.targetPartitions()) + updatedMemberSpec.put(memberId, new MemberSubscriptionSpecImpl( + groupSpec.memberSubscription(memberId).rackId(), + groupSpec.memberSubscription(memberId).subscribedTopicIds(), + new Assignment(Collections.unmodifiableMap(memberAssignment.targetPartitions())) )); - }); + } Set subscribedTopicIdsForNewMember; if (subscriptionType == HETEROGENEOUS) { - subscribedTopicIdsForNewMember = updatedMembers.get("member" + (memberCount - 2)).subscribedTopicIds(); + subscribedTopicIdsForNewMember = updatedMemberSpec.get("member" + (memberCount - 2)).subscribedTopicIds(); } else { subscribedTopicIdsForNewMember = new TopicIds(new HashSet<>(allTopicNames), topicsImage); } Optional rackId = rackId(memberCount - 1); - updatedMembers.put("newMember", new AssignmentMemberSpec( - Optional.empty(), + updatedMemberSpec.put("newMember", new MemberSubscriptionSpecImpl( rackId, subscribedTopicIdsForNewMember, - Collections.emptyMap() + Assignment.EMPTY )); - groupSpec = new GroupSpecImpl(updatedMembers, subscriptionType, invertedTargetAssignment); + groupSpec = new GroupSpecImpl( + updatedMemberSpec, + subscriptionType, + invertedTargetAssignment + ); } @Benchmark diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java index 7d67f07d3e..97508ef9f5 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java @@ -17,10 +17,10 @@ package org.apache.kafka.jmh.assignor; import org.apache.kafka.common.Uuid; -import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; import org.apache.kafka.coordinator.group.assignor.GroupSpecImpl; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; +import org.apache.kafka.coordinator.group.assignor.MemberSubscriptionSpecImpl; import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; import org.apache.kafka.coordinator.group.consumer.TopicIds; import org.apache.kafka.coordinator.group.assignor.UniformAssignor; @@ -191,19 +191,22 @@ private Map generateMockInitialTargetAssignmentAndUpdateInve } private void createAssignmentSpec() { - Map members = new HashMap<>(); + Map members = new HashMap<>(); for (int i = 0; i < memberCount - 1; i++) { String memberId = "member" + i; - members.put(memberId, new AssignmentMemberSpec( - Optional.empty(), + members.put(memberId, new MemberSubscriptionSpecImpl( Optional.empty(), new TopicIds(new HashSet<>(allTopicNames), topicsImage), - Collections.emptyMap() + Assignment.EMPTY )); } - groupSpec = new GroupSpecImpl(members, HOMOGENEOUS, Collections.emptyMap()); + groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); } @Benchmark From be15aa4dc2a1bdebf4554e3a3303fec807bad72f Mon Sep 17 00:00:00 2001 From: Igor Soarez Date: Tue, 4 Jun 2024 15:37:20 +0100 Subject: [PATCH 017/128] KAFKA-16583: Handle PartitionChangeRecord without directory IDs (#16118) When PartitionRegistration#merge() reads a PartitionChangeRecord from an older MetadataVersion, with a replica assignment change and without #directories() set, it produces a direcotry assignment of DirectoryId.UNASSIGNED. This is problematic because the MetadataVersion may not yet support directory assignments, leading to a UnwritableMetadataException in PartitionRegistration#toRecord. Since the Controller always sets directories on PartitionChangeRecord if the MetadataVersion supports it, via PartitionChangeBuilder, there's no need for PartitionRegistration#merge() to populate directories upon a replica assignment change. Reviewers: Luke Chen --- .../kafka/server/ReplicaManagerTest.scala | 1 + .../kafka/metadata/PartitionRegistration.java | 29 ++++++++++++------- .../metadata/PartitionRegistrationTest.java | 6 ++-- 3 files changed, 24 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 97ba10d8be..0c055da2a9 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -5970,6 +5970,7 @@ class ReplicaManagerTest { .setPartitionId(0) .setTopicId(FOO_UUID) .setReplicas(util.Arrays.asList(localId, localId + 1, localId + 2)) + .setDirectories(util.Arrays.asList(Uuid.fromString("fKgQ2axkQiuzt4ANqKbPkQ"), DirectoryId.UNASSIGNED, DirectoryId.UNASSIGNED)) .setIsr(util.Arrays.asList(localId, localId + 1)) ) followerMetadataImage = imageFromTopics(followerTopicsDelta.apply()) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java index 72476cf206..4671b2f054 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java @@ -182,8 +182,20 @@ private static List checkDirectories(PartitionChangeRecord record) { return record.directories(); } + private static boolean migratingDirectories(Uuid[] directories) { + if (directories == null) { + return true; + } + for (Uuid directory : directories) { + if (!DirectoryId.MIGRATING.equals(directory)) { + return false; + } + } + return true; + } + private static Uuid[] defaultToMigrating(Uuid[] directories, int numReplicas) { - if (directories == null || directories.length == 0) { + if (migratingDirectories(directories)) { return DirectoryId.migratingArray(numReplicas); } return directories; @@ -228,14 +240,11 @@ private PartitionRegistration(int[] replicas, Uuid[] directories, int[] isr, int public PartitionRegistration merge(PartitionChangeRecord record) { int[] newReplicas = (record.replicas() == null) ? replicas : Replicas.toArray(record.replicas()); - Uuid[] newDirectories; - if (record.directories() != null && !record.directories().isEmpty()) { - newDirectories = Uuid.toArray(checkDirectories(record)); - } else if (record.replicas() != null) { - newDirectories = Uuid.toArray(DirectoryId.createDirectoriesFrom(replicas, directories, record.replicas())); - } else { - newDirectories = directories; - } + Uuid[] newDirectories = defaultToMigrating( + (record.directories() == null) ? + directories : Uuid.toArray(checkDirectories(record)), + newReplicas.length + ); int[] newIsr = (record.isr() == null) ? isr : Replicas.toArray(record.isr()); int[] newRemovingReplicas = (record.removingReplicas() == null) ? removingReplicas : Replicas.toArray(record.removingReplicas()); @@ -257,7 +266,7 @@ public PartitionRegistration merge(PartitionChangeRecord record) { int[] newElr = (record.eligibleLeaderReplicas() == null) ? elr : Replicas.toArray(record.eligibleLeaderReplicas()); int[] newLastKnownElr = (record.lastKnownElr() == null) ? lastKnownElr : Replicas.toArray(record.lastKnownElr()); return new PartitionRegistration(newReplicas, - defaultToMigrating(newDirectories, replicas.length), + newDirectories, newIsr, newRemovingReplicas, newAddingReplicas, diff --git a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java index 8816f2f141..02b7a34623 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java @@ -153,7 +153,8 @@ public void testMergePartitionChangeRecordWithReassignmentData() { PartitionRegistration partition1 = partition0.merge(new PartitionChangeRecord(). setRemovingReplicas(Collections.singletonList(3)). setAddingReplicas(Collections.singletonList(4)). - setReplicas(Arrays.asList(1, 2, 3, 4))); + setReplicas(Arrays.asList(1, 2, 3, 4)). + setDirectories(Arrays.asList(dir1, dir2, dir3, DirectoryId.UNASSIGNED))); assertEquals(new PartitionRegistration.Builder().setReplicas(new int[] {1, 2, 3, 4}). setDirectories(new Uuid[]{dir1, dir2, dir3, DirectoryId.UNASSIGNED}). setIsr(new int[] {1, 2, 3}).setRemovingReplicas(new int[] {3}).setAddingReplicas(new int[] {4}).setLeader(1).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(201).build(), partition1); @@ -161,7 +162,8 @@ public void testMergePartitionChangeRecordWithReassignmentData() { setIsr(Arrays.asList(1, 2, 4)). setRemovingReplicas(Collections.emptyList()). setAddingReplicas(Collections.emptyList()). - setReplicas(Arrays.asList(1, 2, 4))); + setReplicas(Arrays.asList(1, 2, 4)). + setDirectories(Arrays.asList(dir1, dir2, DirectoryId.UNASSIGNED))); assertEquals(new PartitionRegistration.Builder().setReplicas(new int[] {1, 2, 4}). setDirectories(new Uuid[]{dir1, dir2, DirectoryId.UNASSIGNED}). setIsr(new int[] {1, 2, 4}).setLeader(1).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(202).build(), partition2); From b4f17a01e459cf85bbfe964345a473ed06d99dd9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 3 Jun 2024 17:24:48 -0400 Subject: [PATCH 018/128] KAFKA-16525; Dynamic KRaft network manager and channel (#15986) Allow KRaft replicas to send requests to any node (Node) not just the nodes configured in the controller.quorum.voters property. This flexibility is needed so KRaft can implement the controller.quorum.voters configuration, send request to the dynamically changing set of voters and send request to the leader endpoint (Node) discovered through the KRaft RPCs (specially BeginQuorumEpoch request and Fetch response). This was achieved by changing the RequestManager API to accept Node instead of just the replica ID. Internally, the request manager tracks connection state using the Node.idString method to match the connection management used by NetworkClient. The API for RequestManager is also changed so that the ConnectState class is not exposed in the API. This allows the request manager to reclaim heap memory for any connection that is ready. The NetworkChannel was updated to receive the endpoint information (Node) through the outbound raft request (RaftRequent.Outbound). This makes the network channel more flexible as it doesn't need to be configured with the list of all possible endpoints. RaftRequest.Outbound and RaftResponse.Inbound were updated to include the remote node instead of just the remote id. The follower state tracked by KRaft replicas was updated to include both the leader id and the leader's endpoint (Node). In this comment the node value is computed from the set of voters. In future commit this will be updated so that it is sent through KRaft RPCs. For example BeginQuorumEpoch request and Fetch response. Support for configuring controller.quorum.bootstrap.servers was added. This includes changes to KafkaConfig, QuorumConfig, etc. All of the tests using QuorumTestHarness were changed to use the controller.quorum.bootstrap.servers instead of the controller.quorum.voters for the broker configuration. Finally, the node id for the bootstrap server will be decreasing negative numbers starting with -2. Reviewers: Jason Gustafson , Luke Chen , Colin P. McCabe --- checkstyle/import-control.xml | 1 + .../main/scala/kafka/raft/RaftManager.scala | 18 +- .../main/scala/kafka/server/KafkaConfig.scala | 2 + .../scala/kafka/server/KafkaRaftServer.scala | 1 + .../main/scala/kafka/server/KafkaServer.scala | 3 +- .../NodeToControllerChannelManager.scala | 2 +- .../scala/kafka/server/SharedServer.scala | 3 + .../main/scala/kafka/tools/StorageTool.scala | 2 +- .../scala/kafka/tools/TestRaftServer.scala | 1 + .../kafka/testkit/KafkaClusterTestKit.java | 26 +- core/src/test/resources/log4j.properties | 1 - .../kafka/api/PlaintextConsumerTest.scala | 2 +- .../kafka/server/QuorumTestHarness.scala | 14 +- .../scala/unit/kafka/KafkaConfigTest.scala | 4 +- .../unit/kafka/raft/RaftManagerTest.scala | 1 + .../unit/kafka/server/KafkaConfigTest.scala | 20 +- .../kafka/tools/DumpLogSegmentsTest.scala | 4 +- .../org/apache/kafka/raft/ElectionState.java | 18 +- .../org/apache/kafka/raft/FollowerState.java | 36 +- .../kafka/raft/KafkaNetworkChannel.java | 39 +- .../apache/kafka/raft/KafkaRaftClient.java | 233 +++--- .../org/apache/kafka/raft/NetworkChannel.java | 9 +- .../org/apache/kafka/raft/QuorumConfig.java | 68 +- .../org/apache/kafka/raft/QuorumState.java | 62 +- .../org/apache/kafka/raft/RaftRequest.java | 45 +- .../org/apache/kafka/raft/RaftResponse.java | 39 +- .../java/org/apache/kafka/raft/RaftUtil.java | 3 + .../org/apache/kafka/raft/RequestManager.java | 329 ++++++--- .../apache/kafka/raft/internals/VoterSet.java | 73 +- .../apache/kafka/raft/CandidateStateTest.java | 33 +- .../apache/kafka/raft/FollowerStateTest.java | 11 +- .../kafka/raft/KafkaNetworkChannelTest.java | 167 +++-- .../raft/KafkaRaftClientSnapshotTest.java | 110 +-- .../kafka/raft/KafkaRaftClientTest.java | 460 +++++++++--- .../apache/kafka/raft/MockNetworkChannel.java | 26 +- .../apache/kafka/raft/QuorumStateTest.java | 661 +++++++++++------- .../kafka/raft/RaftClientTestContext.java | 72 +- .../kafka/raft/RaftEventSimulationTest.java | 116 ++- .../apache/kafka/raft/RequestManagerTest.java | 238 +++++-- .../KRaftControlRecordStateMachineTest.java | 28 +- .../raft/internals/KafkaRaftMetricsTest.java | 55 +- .../raft/internals/RecordsIteratorTest.java | 4 +- .../raft/internals/VoterSetHistoryTest.java | 18 +- .../kafka/raft/internals/VoterSetTest.java | 78 ++- .../snapshot/RecordsSnapshotWriterTest.java | 7 +- 45 files changed, 2158 insertions(+), 985 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index ab6177961f..2f90548ffa 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -441,6 +441,7 @@ + diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 6bf8bd893b..0c45734593 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -23,6 +23,7 @@ import java.nio.file.Paths import java.util.OptionalInt import java.util.concurrent.CompletableFuture import java.util.{Map => JMap} +import java.util.{Collection => JCollection} import kafka.log.LogManager import kafka.log.UnifiedLog import kafka.server.KafkaConfig @@ -133,7 +134,7 @@ trait RaftManager[T] { def replicatedLog: ReplicatedLog - def voterNode(id: Int, listener: String): Option[Node] + def voterNode(id: Int, listener: ListenerName): Option[Node] } class KafkaRaftManager[T]( @@ -147,6 +148,7 @@ class KafkaRaftManager[T]( metrics: Metrics, threadNamePrefixOpt: Option[String], val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]], + bootstrapServers: JCollection[InetSocketAddress], fatalFaultHandler: FaultHandler ) extends RaftManager[T] with Logging { @@ -185,7 +187,6 @@ class KafkaRaftManager[T]( def startup(): Unit = { client.initialize( controllerQuorumVotersFuture.get(), - config.controllerListenerNames.head, new FileQuorumStateStore(new File(dataDir, FileQuorumStateStore.DEFAULT_FILE_NAME)), metrics ) @@ -228,14 +229,15 @@ class KafkaRaftManager[T]( expirationService, logContext, clusterId, + bootstrapServers, raftConfig ) client } private def buildNetworkChannel(): KafkaNetworkChannel = { - val netClient = buildNetworkClient() - new KafkaNetworkChannel(time, netClient, config.quorumRequestTimeoutMs, threadNamePrefix) + val (listenerName, netClient) = buildNetworkClient() + new KafkaNetworkChannel(time, listenerName, netClient, config.quorumRequestTimeoutMs, threadNamePrefix) } private def createDataDir(): File = { @@ -254,7 +256,7 @@ class KafkaRaftManager[T]( ) } - private def buildNetworkClient(): NetworkClient = { + private def buildNetworkClient(): (ListenerName, NetworkClient) = { val controllerListenerName = new ListenerName(config.controllerListenerNames.head) val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse( controllerListenerName, @@ -292,7 +294,7 @@ class KafkaRaftManager[T]( val reconnectBackoffMsMs = 500 val discoverBrokerVersions = true - new NetworkClient( + val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), clientId, @@ -309,13 +311,15 @@ class KafkaRaftManager[T]( apiVersions, logContext ) + + (controllerListenerName, networkClient) } override def leaderAndEpoch: LeaderAndEpoch = { client.leaderAndEpoch } - override def voterNode(id: Int, listener: String): Option[Node] = { + override def voterNode(id: Int, listener: ListenerName): Option[Node] = { client.voterNode(id, listener).toScala } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c807ce5d6b..51846afcfe 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -439,6 +439,7 @@ object KafkaConfig { /** ********* Raft Quorum Configuration *********/ .define(QuorumConfig.QUORUM_VOTERS_CONFIG, LIST, QuorumConfig.DEFAULT_QUORUM_VOTERS, new QuorumConfig.ControllerQuorumVotersValidator(), HIGH, QuorumConfig.QUORUM_VOTERS_DOC) + .define(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, LIST, QuorumConfig.DEFAULT_QUORUM_BOOTSTRAP_SERVERS, new QuorumConfig.ControllerQuorumBootstrapServersValidator(), HIGH, QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_DOC) .define(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG, INT, QuorumConfig.DEFAULT_QUORUM_ELECTION_TIMEOUT_MS, null, HIGH, QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_DOC) .define(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG, INT, QuorumConfig.DEFAULT_QUORUM_FETCH_TIMEOUT_MS, null, HIGH, QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_DOC) .define(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG, INT, QuorumConfig.DEFAULT_QUORUM_ELECTION_BACKOFF_MAX_MS, null, HIGH, QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_DOC) @@ -1055,6 +1056,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami /** ********* Raft Quorum Configuration *********/ val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG) + val quorumBootstrapServers = getList(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG) val quorumElectionTimeoutMs = getInt(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG) val quorumFetchTimeoutMs = getInt(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG) val quorumElectionBackoffMs = getInt(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG) diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index d3200149f7..ecb757c1a8 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -71,6 +71,7 @@ class KafkaRaftServer( time, metrics, CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), + QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), new StandardFaultHandlerFactory(), ) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 933a5df536..b913b72fc9 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -70,9 +70,9 @@ import java.net.{InetAddress, SocketTimeoutException} import java.nio.file.{Files, Paths} import java.time.Duration import java.util -import java.util.{Optional, OptionalInt, OptionalLong} import java.util.concurrent._ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.{Optional, OptionalInt, OptionalLong} import scala.collection.{Map, Seq} import scala.compat.java8.OptionConverters.RichOptionForJava8 import scala.jdk.CollectionConverters._ @@ -439,6 +439,7 @@ class KafkaServer( metrics, threadNamePrefix, CompletableFuture.completedFuture(quorumVoters), + QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown()) ) quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config) diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index 0017a5876a..a0e4bbbc46 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -112,7 +112,7 @@ class RaftControllerNodeProvider( val saslMechanism: String ) extends ControllerNodeProvider with Logging { - private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName.value()) + private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName) override def getControllerInfo(): ControllerInformation = ControllerInformation(raftManager.leaderAndEpoch.leaderId.asScala.flatMap(idToNode), diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala index 215208f9f6..ea92dd61f5 100644 --- a/core/src/main/scala/kafka/server/SharedServer.scala +++ b/core/src/main/scala/kafka/server/SharedServer.scala @@ -41,6 +41,7 @@ import java.util.Arrays import java.util.Optional import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{CompletableFuture, TimeUnit} +import java.util.{Collection => JCollection} import java.util.{Map => JMap} @@ -94,6 +95,7 @@ class SharedServer( val time: Time, private val _metrics: Metrics, val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]], + val bootstrapServers: JCollection[InetSocketAddress], val faultHandlerFactory: FaultHandlerFactory ) extends Logging { private val logContext: LogContext = new LogContext(s"[SharedServer id=${sharedServerConfig.nodeId}] ") @@ -265,6 +267,7 @@ class SharedServer( metrics, Some(s"kafka-${sharedServerConfig.nodeId}-raft"), // No dash expected at the end controllerQuorumVotersFuture, + bootstrapServers, raftManagerFaultHandler ) raftManager = _raftManager diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index c79548761d..8481f8468b 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -502,7 +502,7 @@ object StorageTool extends Logging { metaPropertiesEnsemble.verify(metaProperties.clusterId(), metaProperties.nodeId(), util.EnumSet.noneOf(classOf[VerificationFlag])) - System.out.println(s"metaPropertiesEnsemble=$metaPropertiesEnsemble") + stream.println(s"metaPropertiesEnsemble=$metaPropertiesEnsemble") val copier = new MetaPropertiesEnsemble.Copier(metaPropertiesEnsemble) if (!(ignoreFormatted || copier.logDirProps().isEmpty)) { val firstLogDir = copier.logDirProps().keySet().iterator().next() diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index d357ad0bd5..0acae6c5dc 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -95,6 +95,7 @@ class TestRaftServer( metrics, Some(threadNamePrefix), CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), + QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), new ProcessTerminatingFaultHandler.Builder().build() ) diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 5365652a5f..94d94dc717 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -239,12 +239,15 @@ public KafkaClusterTestKit build() throws Exception { ThreadUtils.createThreadFactory("kafka-cluster-test-kit-executor-%d", false)); for (ControllerNode node : nodes.controllerNodes().values()) { setupNodeDirectories(baseDirectory, node.metadataDirectory(), Collections.emptyList()); - SharedServer sharedServer = new SharedServer(createNodeConfig(node), - node.initialMetaPropertiesEnsemble(), - Time.SYSTEM, - new Metrics(), - connectFutureManager.future, - faultHandlerFactory); + SharedServer sharedServer = new SharedServer( + createNodeConfig(node), + node.initialMetaPropertiesEnsemble(), + Time.SYSTEM, + new Metrics(), + connectFutureManager.future, + Collections.emptyList(), + faultHandlerFactory + ); ControllerServer controller = null; try { controller = new ControllerServer( @@ -267,13 +270,18 @@ public KafkaClusterTestKit build() throws Exception { jointServers.put(node.id(), sharedServer); } for (BrokerNode node : nodes.brokerNodes().values()) { - SharedServer sharedServer = jointServers.computeIfAbsent(node.id(), - id -> new SharedServer(createNodeConfig(node), + SharedServer sharedServer = jointServers.computeIfAbsent( + node.id(), + id -> new SharedServer( + createNodeConfig(node), node.initialMetaPropertiesEnsemble(), Time.SYSTEM, new Metrics(), connectFutureManager.future, - faultHandlerFactory)); + Collections.emptyList(), + faultHandlerFactory + ) + ); BrokerServer broker = null; try { broker = new BrokerServer(sharedServer); diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index f7fb7364a3..b265ee9cda 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -21,6 +21,5 @@ log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n log4j.logger.kafka=WARN log4j.logger.org.apache.kafka=WARN - # zkclient can be verbose, during debugging it is common to adjust it separately log4j.logger.org.apache.zookeeper=WARN diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 7d32d1bd3e..b4617af150 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -21,12 +21,12 @@ import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.admin.{NewPartitions, NewTopic} import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} -import org.apache.kafka.common.{KafkaException, MetricName, TopicPartition} import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.{InvalidGroupIdException, InvalidTopicException, TimeoutException, WakeupException} import org.apache.kafka.common.header.Headers import org.apache.kafka.common.record.{CompressionType, TimestampType} import org.apache.kafka.common.serialization._ +import org.apache.kafka.common.{KafkaException, MetricName, TopicPartition} import org.apache.kafka.test.{MockConsumerInterceptor, MockProducerInterceptor} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Timeout diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 9f787a1b16..82b5b4cfd1 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -124,12 +124,15 @@ class KRaftQuorumImplementation( metaPropertiesEnsemble.verify(Optional.of(clusterId), OptionalInt.of(config.nodeId), util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)) - val sharedServer = new SharedServer(config, + val sharedServer = new SharedServer( + config, metaPropertiesEnsemble, time, new Metrics(), controllerQuorumVotersFuture, - faultHandlerFactory) + controllerQuorumVotersFuture.get().values(), + faultHandlerFactory + ) var broker: BrokerServer = null try { broker = new BrokerServer(sharedServer) @@ -371,12 +374,15 @@ abstract class QuorumTestHarness extends Logging { metaPropertiesEnsemble.verify(Optional.of(metaProperties.clusterId().get()), OptionalInt.of(nodeId), util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)) - val sharedServer = new SharedServer(config, + val sharedServer = new SharedServer( + config, metaPropertiesEnsemble, Time.SYSTEM, new Metrics(), controllerQuorumVotersFuture, - faultHandlerFactory) + Collections.emptyList(), + faultHandlerFactory + ) var controllerServer: ControllerServer = null try { controllerServer = new ControllerServer( diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 457326cd19..3a1fc2e4bd 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -86,7 +86,7 @@ class KafkaConfigTest { @Test def testBrokerRoleNodeIdValidation(): Unit = { - // Ensure that validation is happening at startup to check that brokers do not use their node.id as a voter in controller.quorum.voters + // Ensure that validation is happening at startup to check that brokers do not use their node.id as a voter in controller.quorum.voters val propertiesFile = new Properties propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") @@ -102,7 +102,7 @@ class KafkaConfigTest { @Test def testControllerRoleNodeIdValidation(): Unit = { - // Ensure that validation is happening at startup to check that controllers use their node.id as a voter in controller.quorum.voters + // Ensure that validation is happening at startup to check that controllers use their node.id as a voter in controller.quorum.voters val propertiesFile = new Properties propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index 3416ffe65b..da9d29304e 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -118,6 +118,7 @@ class RaftManagerTest { new Metrics(Time.SYSTEM), Option.empty, CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), + QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), mock(classOf[FaultHandler]) ) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 933f514df5..266b64560f 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -19,7 +19,7 @@ package kafka.server import java.net.InetSocketAddress import java.util -import java.util.{Collections, Properties} +import java.util.{Arrays, Collections, Properties} import kafka.cluster.EndPoint import kafka.security.authorizer.AclAuthorizer import kafka.utils.TestUtils.assertBadConfigContainingMessage @@ -1032,6 +1032,7 @@ class KafkaConfigTest { // Raft Quorum Configs case QuorumConfig.QUORUM_VOTERS_CONFIG => // ignore string + case QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG => // ignore string case QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -1402,6 +1403,23 @@ class KafkaConfigTest { assertEquals(expectedVoters, addresses) } + @Test + def testParseQuorumBootstrapServers(): Unit = { + val expected = Arrays.asList( + InetSocketAddress.createUnresolved("kafka1", 9092), + InetSocketAddress.createUnresolved("kafka2", 9092) + ) + + val props = TestUtils.createBrokerConfig(0, null) + props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "kafka1:9092,kafka2:9092") + + val addresses = QuorumConfig.parseBootstrapServers( + KafkaConfig.fromProps(props).quorumBootstrapServers + ) + + assertEquals(expected, addresses) + } + @Test def testAcceptsLargeNodeIdForRaftBasedCase(): Unit = { // Generation of Broker IDs is not supported when using Raft-based controller quorums, diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index b8764f5fae..c625dc6e96 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -22,8 +22,8 @@ import java.nio.ByteBuffer import java.util import java.util.Collections import java.util.Optional -import java.util.Arrays import java.util.Properties +import java.util.stream.IntStream import kafka.log.{LogTestUtils, UnifiedLog} import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} import kafka.server.{BrokerTopicStats, KafkaRaftServer} @@ -338,7 +338,7 @@ class DumpLogSegmentsTest { .setLastContainedLogTimestamp(lastContainedLogTimestamp) .setRawSnapshotWriter(metadataLog.createNewSnapshot(new OffsetAndEpoch(0, 0)).get) .setKraftVersion(1) - .setVoterSet(Optional.of(VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)))) + .setVoterSet(Optional.of(VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)))) .build(MetadataRecordSerde.INSTANCE) ) { snapshotWriter => snapshotWriter.append(metadataRecords.asJava) diff --git a/raft/src/main/java/org/apache/kafka/raft/ElectionState.java b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java index 005ff23a4f..825acf7df6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ElectionState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java @@ -30,9 +30,9 @@ * Encapsulate election state stored on disk after every state change. */ final public class ElectionState { - private static int unknownLeaderId = -1; - private static int notVoted = -1; - private static Uuid noVotedDirectoryId = Uuid.ZERO_UUID; + private static final int UNKNOWN_LEADER_ID = -1; + private static final int NOT_VOTED = -1; + private static final Uuid NO_VOTED_DIRECTORY_ID = Uuid.ZERO_UUID; private final int epoch; private final OptionalInt leaderId; @@ -95,7 +95,7 @@ public int leaderId() { } public int leaderIdOrSentinel() { - return leaderId.orElse(unknownLeaderId); + return leaderId.orElse(UNKNOWN_LEADER_ID); } public OptionalInt optionalLeaderId() { @@ -126,7 +126,7 @@ public QuorumStateData toQuorumStateData(short version) { QuorumStateData data = new QuorumStateData() .setLeaderEpoch(epoch) .setLeaderId(leaderIdOrSentinel()) - .setVotedId(votedKey.map(ReplicaKey::id).orElse(notVoted)); + .setVotedId(votedKey.map(ReplicaKey::id).orElse(NOT_VOTED)); if (version == 0) { List dataVoters = voters @@ -135,7 +135,7 @@ public QuorumStateData toQuorumStateData(short version) { .collect(Collectors.toList()); data.setCurrentVoters(dataVoters); } else if (version == 1) { - data.setVotedDirectoryId(votedKey.flatMap(ReplicaKey::directoryId).orElse(noVotedDirectoryId)); + data.setVotedDirectoryId(votedKey.flatMap(ReplicaKey::directoryId).orElse(NO_VOTED_DIRECTORY_ID)); } else { throw new IllegalStateException( String.format( @@ -198,17 +198,17 @@ public static ElectionState withUnknownLeader(int epoch, Set voters) { } public static ElectionState fromQuorumStateData(QuorumStateData data) { - Optional votedDirectoryId = data.votedDirectoryId().equals(noVotedDirectoryId) ? + Optional votedDirectoryId = data.votedDirectoryId().equals(NO_VOTED_DIRECTORY_ID) ? Optional.empty() : Optional.of(data.votedDirectoryId()); - Optional votedKey = data.votedId() == notVoted ? + Optional votedKey = data.votedId() == NOT_VOTED ? Optional.empty() : Optional.of(ReplicaKey.of(data.votedId(), votedDirectoryId)); return new ElectionState( data.leaderEpoch(), - data.leaderId() == unknownLeaderId ? OptionalInt.empty() : OptionalInt.of(data.leaderId()), + data.leaderId() == UNKNOWN_LEADER_ID ? OptionalInt.empty() : OptionalInt.of(data.leaderId()), votedKey, data.currentVoters().stream().map(QuorumStateData.Voter::voterId).collect(Collectors.toSet()) ); diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 49bfaff181..0491689505 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -19,6 +19,7 @@ import java.util.Optional; import java.util.OptionalLong; import java.util.Set; +import org.apache.kafka.common.Node; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; @@ -29,7 +30,7 @@ public class FollowerState implements EpochState { private final int fetchTimeoutMs; private final int epoch; - private final int leaderId; + private final Node leader; private final Set voters; // Used for tracking the expiration of both the Fetch and FetchSnapshot requests private final Timer fetchTimer; @@ -37,14 +38,14 @@ public class FollowerState implements EpochState { /* Used to track the currently fetching snapshot. When fetching snapshot regular * Fetch request are paused */ - private Optional fetchingSnapshot; + private Optional fetchingSnapshot = Optional.empty(); private final Logger log; public FollowerState( Time time, int epoch, - int leaderId, + Node leader, Set voters, Optional highWatermark, int fetchTimeoutMs, @@ -52,17 +53,16 @@ public FollowerState( ) { this.fetchTimeoutMs = fetchTimeoutMs; this.epoch = epoch; - this.leaderId = leaderId; + this.leader = leader; this.voters = voters; this.fetchTimer = time.timer(fetchTimeoutMs); this.highWatermark = highWatermark; - this.fetchingSnapshot = Optional.empty(); this.log = logContext.logger(FollowerState.class); } @Override public ElectionState election() { - return ElectionState.withElectedLeader(epoch, leaderId, voters); + return ElectionState.withElectedLeader(epoch, leader.id(), voters); } @Override @@ -80,8 +80,8 @@ public long remainingFetchTimeMs(long currentTimeMs) { return fetchTimer.remainingMs(); } - public int leaderId() { - return leaderId; + public Node leader() { + return leader; } public boolean hasFetchTimeoutExpired(long currentTimeMs) { @@ -156,7 +156,7 @@ public boolean canGrantVote(ReplicaKey candidateKey, boolean isLogUpToDate) { log.debug( "Rejecting vote request from candidate ({}) since we already have a leader {} in epoch {}", candidateKey, - leaderId(), + leader, epoch ); return false; @@ -164,14 +164,16 @@ public boolean canGrantVote(ReplicaKey candidateKey, boolean isLogUpToDate) { @Override public String toString() { - return "FollowerState(" + - "fetchTimeoutMs=" + fetchTimeoutMs + - ", epoch=" + epoch + - ", leaderId=" + leaderId + - ", voters=" + voters + - ", highWatermark=" + highWatermark + - ", fetchingSnapshot=" + fetchingSnapshot + - ')'; + return String.format( + "FollowerState(fetchTimeoutMs=%d, epoch=%d, leader=%s voters=%s, highWatermark=%s, " + + "fetchingSnapshot=%s)", + fetchTimeoutMs, + epoch, + leader, + voters, + highWatermark, + fetchingSnapshot + ); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java b/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java index 5ec91752cb..f6341b76e7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchSnapshotRequestData; import org.apache.kafka.common.message.VoteRequestData; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; @@ -39,12 +40,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicInteger; @@ -83,9 +81,17 @@ public void sendRequest(RequestAndCompletionHandler request) { private final SendThread requestThread; private final AtomicInteger correlationIdCounter = new AtomicInteger(0); - private final Map endpoints = new HashMap<>(); - public KafkaNetworkChannel(Time time, KafkaClient client, int requestTimeoutMs, String threadNamePrefix) { + private final ListenerName listenerName; + + public KafkaNetworkChannel( + Time time, + ListenerName listenerName, + KafkaClient client, + int requestTimeoutMs, + String threadNamePrefix + ) { + this.listenerName = listenerName; this.requestThread = new SendThread( threadNamePrefix + "-outbound-request-thread", client, @@ -102,23 +108,23 @@ public int newCorrelationId() { @Override public void send(RaftRequest.Outbound request) { - Node node = endpoints.get(request.destinationId()); + Node node = request.destination(); if (node != null) { requestThread.sendRequest(new RequestAndCompletionHandler( - request.createdTimeMs, + request.createdTimeMs(), node, - buildRequest(request.data), + buildRequest(request.data()), response -> sendOnComplete(request, response) )); } else - sendCompleteFuture(request, errorResponse(request.data, Errors.BROKER_NOT_AVAILABLE)); + sendCompleteFuture(request, errorResponse(request.data(), Errors.BROKER_NOT_AVAILABLE)); } private void sendCompleteFuture(RaftRequest.Outbound request, ApiMessage message) { RaftResponse.Inbound response = new RaftResponse.Inbound( - request.correlationId, + request.correlationId(), message, - request.destinationId() + request.destination() ); request.completion.complete(response); } @@ -127,16 +133,16 @@ private void sendOnComplete(RaftRequest.Outbound request, ClientResponse clientR ApiMessage response; if (clientResponse.versionMismatch() != null) { log.error("Request {} failed due to unsupported version error", request, clientResponse.versionMismatch()); - response = errorResponse(request.data, Errors.UNSUPPORTED_VERSION); + response = errorResponse(request.data(), Errors.UNSUPPORTED_VERSION); } else if (clientResponse.authenticationException() != null) { // For now we treat authentication errors as retriable. We use the // `NETWORK_EXCEPTION` error code for lack of a good alternative. // Note that `NodeToControllerChannelManager` will still log the // authentication errors so that users have a chance to fix the problem. log.error("Request {} failed due to authentication error", request, clientResponse.authenticationException()); - response = errorResponse(request.data, Errors.NETWORK_EXCEPTION); + response = errorResponse(request.data(), Errors.NETWORK_EXCEPTION); } else if (clientResponse.wasDisconnected()) { - response = errorResponse(request.data, Errors.BROKER_NOT_AVAILABLE); + response = errorResponse(request.data(), Errors.BROKER_NOT_AVAILABLE); } else { response = clientResponse.responseBody().data(); } @@ -149,9 +155,8 @@ private ApiMessage errorResponse(ApiMessage request, Errors error) { } @Override - public void updateEndpoint(int id, InetSocketAddress address) { - Node node = new Node(id, address.getHostString(), address.getPort()); - endpoints.put(id, node); + public ListenerName listenerName() { + return listenerName; } public void start() { diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 10910c3db7..6a14b37cff 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -37,6 +37,7 @@ import org.apache.kafka.common.message.VoteRequestData; import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; @@ -60,7 +61,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.raft.RequestManager.ConnectionState; import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.BatchMemoryPool; @@ -85,6 +85,7 @@ import org.slf4j.Logger; import java.net.InetSocketAddress; +import java.util.Collection; import java.util.Collections; import java.util.IdentityHashMap; import java.util.Iterator; @@ -100,8 +101,10 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; +import java.util.stream.Collectors; import static java.util.concurrent.CompletableFuture.completedFuture; import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition; @@ -209,6 +212,7 @@ public KafkaRaftClient( ExpirationService expirationService, LogContext logContext, String clusterId, + Collection bootstrapServers, QuorumConfig quorumConfig ) { this( @@ -223,6 +227,7 @@ public KafkaRaftClient( expirationService, MAX_FETCH_WAIT_MS, clusterId, + bootstrapServers, logContext, new Random(), quorumConfig @@ -241,6 +246,7 @@ public KafkaRaftClient( ExpirationService expirationService, int fetchMaxWaitMs, String clusterId, + Collection bootstrapServers, LogContext logContext, Random random, QuorumConfig quorumConfig @@ -262,6 +268,30 @@ public KafkaRaftClient( this.random = random; this.quorumConfig = quorumConfig; this.snapshotCleaner = new RaftMetadataLogCleanerManager(logger, time, 60000, log::maybeClean); + + if (!bootstrapServers.isEmpty()) { + // generate Node objects from network addresses by using decreasing negative ids + AtomicInteger id = new AtomicInteger(-2); + List bootstrapNodes = bootstrapServers + .stream() + .map(address -> + new Node( + id.getAndDecrement(), + address.getHostString(), + address.getPort() + ) + ) + .collect(Collectors.toList()); + + logger.info("Starting request manager with bootstrap servers: {}", bootstrapNodes); + + requestManager = new RequestManager( + bootstrapNodes, + quorumConfig.retryBackoffMs(), + quorumConfig.requestTimeoutMs(), + random + ); + } } private void updateFollowerHighWatermark( @@ -378,12 +408,11 @@ private void maybeFireLeaderChange() { public void initialize( Map voterAddresses, - String listenerName, QuorumStateStore quorumStateStore, Metrics metrics ) { partitionState = new KRaftControlRecordStateMachine( - Optional.of(VoterSet.fromInetSocketAddresses(listenerName, voterAddresses)), + Optional.of(VoterSet.fromInetSocketAddresses(channel.listenerName(), voterAddresses)), log, serde, BufferSupplier.create(), @@ -394,17 +423,35 @@ public void initialize( logger.info("Reading KRaft snapshot and log as part of the initialization"); partitionState.updateState(); - VoterSet lastVoterSet = partitionState.lastVoterSet(); - requestManager = new RequestManager( - lastVoterSet.voterIds(), - quorumConfig.retryBackoffMs(), - quorumConfig.requestTimeoutMs(), - random - ); + if (requestManager == null) { + // The request manager wasn't created using the bootstrap servers + // create it using the voters static configuration + List bootstrapNodes = voterAddresses + .entrySet() + .stream() + .map(entry -> + new Node( + entry.getKey(), + entry.getValue().getHostString(), + entry.getValue().getPort() + ) + ) + .collect(Collectors.toList()); + + logger.info("Starting request manager with static voters: {}", bootstrapNodes); + + requestManager = new RequestManager( + bootstrapNodes, + quorumConfig.retryBackoffMs(), + quorumConfig.requestTimeoutMs(), + random + ); + } quorum = new QuorumState( nodeId, nodeDirectoryId, + channel.listenerName(), partitionState::lastVoterSet, partitionState::lastKraftVersion, quorumConfig.electionTimeoutMs(), @@ -420,10 +467,6 @@ public void initialize( // so there are no unknown voter connections. Report this metric as 0. kafkaRaftMetrics.updateNumUnknownVoterConnections(0); - for (Integer voterId : lastVoterSet.voterIds()) { - channel.updateEndpoint(voterId, lastVoterSet.voterAddress(voterId, listenerName).get()); - } - quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch())); long currentTimeMs = time.milliseconds(); @@ -569,10 +612,10 @@ private void onBecomeFollower(long currentTimeMs) { private void transitionToFollower( int epoch, - int leaderId, + Node leader, long currentTimeMs ) { - quorum.transitionToFollower(epoch, leaderId); + quorum.transitionToFollower(epoch, leader); maybeFireLeaderChange(); onBecomeFollower(currentTimeMs); } @@ -601,7 +644,7 @@ private VoteResponseData buildVoteResponse(Errors partitionLevelError, boolean v private VoteResponseData handleVoteRequest( RaftRequest.Inbound requestMetadata ) { - VoteRequestData request = (VoteRequestData) requestMetadata.data; + VoteRequestData request = (VoteRequestData) requestMetadata.data(); if (!hasValidClusterId(request.clusterId())) { return new VoteResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code()); @@ -652,8 +695,8 @@ private boolean handleVoteResponse( RaftResponse.Inbound responseMetadata, long currentTimeMs ) { - int remoteNodeId = responseMetadata.sourceId(); - VoteResponseData response = (VoteResponseData) responseMetadata.data; + int remoteNodeId = responseMetadata.source().id(); + VoteResponseData response = (VoteResponseData) responseMetadata.data(); Errors topLevelError = Errors.forCode(response.errorCode()); if (topLevelError != Errors.NONE) { return handleTopLevelError(topLevelError, responseMetadata); @@ -751,7 +794,7 @@ private BeginQuorumEpochResponseData handleBeginQuorumEpochRequest( RaftRequest.Inbound requestMetadata, long currentTimeMs ) { - BeginQuorumEpochRequestData request = (BeginQuorumEpochRequestData) requestMetadata.data; + BeginQuorumEpochRequestData request = (BeginQuorumEpochRequestData) requestMetadata.data(); if (!hasValidClusterId(request.clusterId())) { return new BeginQuorumEpochResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code()); @@ -773,7 +816,11 @@ private BeginQuorumEpochResponseData handleBeginQuorumEpochRequest( return buildBeginQuorumEpochResponse(errorOpt.get()); } - maybeTransition(OptionalInt.of(requestLeaderId), requestEpoch, currentTimeMs); + maybeTransition( + partitionState.lastVoterSet().voterNode(requestLeaderId, channel.listenerName()), + requestEpoch, + currentTimeMs + ); return buildBeginQuorumEpochResponse(Errors.NONE); } @@ -781,8 +828,8 @@ private boolean handleBeginQuorumEpochResponse( RaftResponse.Inbound responseMetadata, long currentTimeMs ) { - int remoteNodeId = responseMetadata.sourceId(); - BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) responseMetadata.data; + int remoteNodeId = responseMetadata.source().id(); + BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) responseMetadata.data(); Errors topLevelError = Errors.forCode(response.errorCode()); if (topLevelError != Errors.NONE) { return handleTopLevelError(topLevelError, responseMetadata); @@ -840,7 +887,7 @@ private EndQuorumEpochResponseData handleEndQuorumEpochRequest( RaftRequest.Inbound requestMetadata, long currentTimeMs ) { - EndQuorumEpochRequestData request = (EndQuorumEpochRequestData) requestMetadata.data; + EndQuorumEpochRequestData request = (EndQuorumEpochRequestData) requestMetadata.data(); if (!hasValidClusterId(request.clusterId())) { return new EndQuorumEpochResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code()); @@ -861,11 +908,15 @@ private EndQuorumEpochResponseData handleEndQuorumEpochRequest( if (errorOpt.isPresent()) { return buildEndQuorumEpochResponse(errorOpt.get()); } - maybeTransition(OptionalInt.of(requestLeaderId), requestEpoch, currentTimeMs); + maybeTransition( + partitionState.lastVoterSet().voterNode(requestLeaderId, channel.listenerName()), + requestEpoch, + currentTimeMs + ); if (quorum.isFollower()) { FollowerState state = quorum.followerStateOrThrow(); - if (state.leaderId() == requestLeaderId) { + if (state.leader().id() == requestLeaderId) { List preferredSuccessors = partitionRequest.preferredSuccessors(); long electionBackoffMs = endEpochElectionBackoff(preferredSuccessors); logger.debug("Overriding follower fetch timeout to {} after receiving " + @@ -894,7 +945,7 @@ private boolean handleEndQuorumEpochResponse( RaftResponse.Inbound responseMetadata, long currentTimeMs ) { - EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) responseMetadata.data; + EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) responseMetadata.data(); Errors topLevelError = Errors.forCode(response.errorCode()); if (topLevelError != Errors.NONE) { return handleTopLevelError(topLevelError, responseMetadata); @@ -917,7 +968,7 @@ private boolean handleEndQuorumEpochResponse( return handled.get(); } else if (partitionError == Errors.NONE) { ResignedState resignedState = quorum.resignedStateOrThrow(); - resignedState.acknowledgeResignation(responseMetadata.sourceId()); + resignedState.acknowledgeResignation(responseMetadata.source().id()); return true; } else { return handleUnexpectedError(partitionError, responseMetadata); @@ -998,7 +1049,7 @@ private CompletableFuture handleFetchRequest( RaftRequest.Inbound requestMetadata, long currentTimeMs ) { - FetchRequestData request = (FetchRequestData) requestMetadata.data; + FetchRequestData request = (FetchRequestData) requestMetadata.data(); if (!hasValidClusterId(request.clusterId())) { return completedFuture(new FetchResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code())); @@ -1147,13 +1198,13 @@ private boolean handleFetchResponse( RaftResponse.Inbound responseMetadata, long currentTimeMs ) { - FetchResponseData response = (FetchResponseData) responseMetadata.data; + FetchResponseData response = (FetchResponseData) responseMetadata.data(); Errors topLevelError = Errors.forCode(response.errorCode()); if (topLevelError != Errors.NONE) { return handleTopLevelError(topLevelError, responseMetadata); } - if (!RaftUtil.hasValidTopicPartition(response, log.topicPartition(), log.topicId())) { + if (!hasValidTopicPartition(response, log.topicPartition(), log.topicId())) { return false; } // If the ID is valid, we can set the topic name. @@ -1286,7 +1337,7 @@ private DescribeQuorumResponseData handleDescribeQuorumRequest( RaftRequest.Inbound requestMetadata, long currentTimeMs ) { - DescribeQuorumRequestData describeQuorumRequestData = (DescribeQuorumRequestData) requestMetadata.data; + DescribeQuorumRequestData describeQuorumRequestData = (DescribeQuorumRequestData) requestMetadata.data(); if (!hasValidTopicPartition(describeQuorumRequestData, log.topicPartition())) { return DescribeQuorumRequest.getPartitionLevelErrorResponse( describeQuorumRequestData, Errors.UNKNOWN_TOPIC_OR_PARTITION); @@ -1325,7 +1376,7 @@ private FetchSnapshotResponseData handleFetchSnapshotRequest( RaftRequest.Inbound requestMetadata, long currentTimeMs ) { - FetchSnapshotRequestData data = (FetchSnapshotRequestData) requestMetadata.data; + FetchSnapshotRequestData data = (FetchSnapshotRequestData) requestMetadata.data(); if (!hasValidClusterId(data.clusterId())) { return new FetchSnapshotResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code()); @@ -1429,7 +1480,7 @@ private boolean handleFetchSnapshotResponse( RaftResponse.Inbound responseMetadata, long currentTimeMs ) { - FetchSnapshotResponseData data = (FetchSnapshotResponseData) responseMetadata.data; + FetchSnapshotResponseData data = (FetchSnapshotResponseData) responseMetadata.data(); Errors topLevelError = Errors.forCode(data.errorCode()); if (topLevelError != Errors.NONE) { return handleTopLevelError(topLevelError, responseMetadata); @@ -1593,6 +1644,12 @@ private Optional maybeHandleCommonResponse( int epoch, long currentTimeMs ) { + Optional leader = leaderId.isPresent() ? + partitionState + .lastVoterSet() + .voterNode(leaderId.getAsInt(), channel.listenerName()) : + Optional.empty(); + if (epoch < quorum.epoch() || error == Errors.UNKNOWN_LEADER_EPOCH) { // We have a larger epoch, so the response is no longer relevant return Optional.of(true); @@ -1602,10 +1659,10 @@ private Optional maybeHandleCommonResponse( // The response indicates that the request had a stale epoch, but we need // to validate the epoch from the response against our current state. - maybeTransition(leaderId, epoch, currentTimeMs); + maybeTransition(leader, epoch, currentTimeMs); return Optional.of(true); } else if (epoch == quorum.epoch() - && leaderId.isPresent() + && leader.isPresent() && !quorum.hasLeader()) { // Since we are transitioning to Follower, we will only forward the @@ -1613,7 +1670,7 @@ private Optional maybeHandleCommonResponse( // the request be retried immediately (if needed) after the transition. // This handling allows an observer to discover the leader and append // to the log in the same Fetch request. - transitionToFollower(epoch, leaderId.getAsInt(), currentTimeMs); + transitionToFollower(epoch, leader.get(), currentTimeMs); if (error == Errors.NONE) { return Optional.empty(); } else { @@ -1635,24 +1692,28 @@ private Optional maybeHandleCommonResponse( } private void maybeTransition( - OptionalInt leaderId, + Optional leader, int epoch, long currentTimeMs ) { + OptionalInt leaderId = leader.isPresent() ? + OptionalInt.of(leader.get().id()) : + OptionalInt.empty(); + if (!hasConsistentLeader(epoch, leaderId)) { - throw new IllegalStateException("Received request or response with leader " + leaderId + + throw new IllegalStateException("Received request or response with leader " + leader + " and epoch " + epoch + " which is inconsistent with current leader " + quorum.leaderId() + " and epoch " + quorum.epoch()); } else if (epoch > quorum.epoch()) { - if (leaderId.isPresent()) { - transitionToFollower(epoch, leaderId.getAsInt(), currentTimeMs); + if (leader.isPresent()) { + transitionToFollower(epoch, leader.get(), currentTimeMs); } else { transitionToUnattached(epoch); } - } else if (leaderId.isPresent() && !quorum.hasLeader()) { + } else if (leader.isPresent() && !quorum.hasLeader()) { // The request or response indicates the leader of the current epoch, // which is currently unknown - transitionToFollower(epoch, leaderId.getAsInt(), currentTimeMs); + transitionToFollower(epoch, leader.get(), currentTimeMs); } } @@ -1668,13 +1729,13 @@ private boolean handleTopLevelError(Errors error, RaftResponse.Inbound response) private boolean handleUnexpectedError(Errors error, RaftResponse.Inbound response) { logger.error("Unexpected error {} in {} response: {}", - error, ApiKeys.forId(response.data.apiKey()), response); + error, ApiKeys.forId(response.data().apiKey()), response); return false; } private void handleResponse(RaftResponse.Inbound response, long currentTimeMs) { // The response epoch matches the local epoch, so we can handle the response - ApiKeys apiKey = ApiKeys.forId(response.data.apiKey()); + ApiKeys apiKey = ApiKeys.forId(response.data().apiKey()); final boolean handledSuccessfully; switch (apiKey) { @@ -1702,12 +1763,12 @@ private void handleResponse(RaftResponse.Inbound response, long currentTimeMs) { throw new IllegalArgumentException("Received unexpected response type: " + apiKey); } - ConnectionState connection = requestManager.getOrCreate(response.sourceId()); - if (handledSuccessfully) { - connection.onResponseReceived(response.correlationId); - } else { - connection.onResponseError(response.correlationId, currentTimeMs); - } + requestManager.onResponseResult( + response.source(), + response.correlationId(), + handledSuccessfully, + currentTimeMs + ); } /** @@ -1749,7 +1810,7 @@ private Optional validateLeaderOnlyRequest(int requestEpoch) { } private void handleRequest(RaftRequest.Inbound request, long currentTimeMs) { - ApiKeys apiKey = ApiKeys.forId(request.data.apiKey()); + ApiKeys apiKey = ApiKeys.forId(request.data().apiKey()); final CompletableFuture responseFuture; switch (apiKey) { @@ -1803,8 +1864,7 @@ private void handleInboundMessage(RaftMessage message, long currentTimeMs) { handleRequest(request, currentTimeMs); } else if (message instanceof RaftResponse.Inbound) { RaftResponse.Inbound response = (RaftResponse.Inbound) message; - ConnectionState connection = requestManager.getOrCreate(response.sourceId()); - if (connection.isResponseExpected(response.correlationId)) { + if (requestManager.isResponseExpected(response.source(), response.correlationId())) { handleResponse(response, currentTimeMs); } else { logger.debug("Ignoring response {} since it is no longer needed", response); @@ -1819,25 +1879,23 @@ private void handleInboundMessage(RaftMessage message, long currentTimeMs) { */ private long maybeSendRequest( long currentTimeMs, - int destinationId, + Node destination, Supplier requestSupplier ) { - ConnectionState connection = requestManager.getOrCreate(destinationId); - - if (connection.isBackingOff(currentTimeMs)) { - long remainingBackoffMs = connection.remainingBackoffMs(currentTimeMs); - logger.debug("Connection for {} is backing off for {} ms", destinationId, remainingBackoffMs); + if (requestManager.isBackingOff(destination, currentTimeMs)) { + long remainingBackoffMs = requestManager.remainingBackoffMs(destination, currentTimeMs); + logger.debug("Connection for {} is backing off for {} ms", destination, remainingBackoffMs); return remainingBackoffMs; } - if (connection.isReady(currentTimeMs)) { + if (requestManager.isReady(destination, currentTimeMs)) { int correlationId = channel.newCorrelationId(); ApiMessage request = requestSupplier.get(); RaftRequest.Outbound requestMessage = new RaftRequest.Outbound( correlationId, request, - destinationId, + destination, currentTimeMs ); @@ -1850,20 +1908,19 @@ private long maybeSendRequest( response = new RaftResponse.Inbound( correlationId, errorResponse, - destinationId + destination ); } messageQueue.add(response); }); + requestManager.onRequestSent(destination, correlationId, currentTimeMs); channel.send(requestMessage); logger.trace("Sent outbound request: {}", requestMessage); - connection.onRequestSent(correlationId, currentTimeMs); - return Long.MAX_VALUE; } - return connection.remainingRequestTimeMs(currentTimeMs); + return requestManager.remainingRequestTimeMs(destination, currentTimeMs); } private EndQuorumEpochRequestData buildEndQuorumEpochRequest( @@ -1880,12 +1937,12 @@ private EndQuorumEpochRequestData buildEndQuorumEpochRequest( private long maybeSendRequests( long currentTimeMs, - Set destinationIds, + Set destinations, Supplier requestSupplier ) { long minBackoffMs = Long.MAX_VALUE; - for (Integer destinationId : destinationIds) { - long backoffMs = maybeSendRequest(currentTimeMs, destinationId, requestSupplier); + for (Node destination : destinations) { + long backoffMs = maybeSendRequest(currentTimeMs, destination, requestSupplier); if (backoffMs < minBackoffMs) { minBackoffMs = backoffMs; } @@ -1929,15 +1986,15 @@ private FetchRequestData buildFetchRequest() { } private long maybeSendAnyVoterFetch(long currentTimeMs) { - OptionalInt readyVoterIdOpt = requestManager.findReadyVoter(currentTimeMs); - if (readyVoterIdOpt.isPresent()) { + Optional readyNode = requestManager.findReadyBootstrapServer(currentTimeMs); + if (readyNode.isPresent()) { return maybeSendRequest( currentTimeMs, - readyVoterIdOpt.getAsInt(), + readyNode.get(), this::buildFetchRequest ); } else { - return requestManager.backoffBeforeAvailableVoter(currentTimeMs); + return requestManager.backoffBeforeAvailableBootstrapServer(currentTimeMs); } } @@ -2038,7 +2095,9 @@ private long pollResigned(long currentTimeMs) { ResignedState state = quorum.resignedStateOrThrow(); long endQuorumBackoffMs = maybeSendRequests( currentTimeMs, - state.unackedVoters(), + partitionState + .lastVoterSet() + .voterNodes(state.unackedVoters().stream(), channel.listenerName()), () -> buildEndQuorumEpochRequest(state) ); @@ -2075,7 +2134,9 @@ private long pollLeader(long currentTimeMs) { long timeUntilSend = maybeSendRequests( currentTimeMs, - state.nonAcknowledgingVoters(), + partitionState + .lastVoterSet() + .voterNodes(state.nonAcknowledgingVoters().stream(), channel.listenerName()), this::buildBeginQuorumEpochRequest ); @@ -2090,7 +2151,9 @@ private long maybeSendVoteRequests( if (!state.isVoteRejected()) { return maybeSendRequests( currentTimeMs, - state.unrecordedVoters(), + partitionState + .lastVoterSet() + .voterNodes(state.unrecordedVoters().stream(), channel.listenerName()), this::buildVoteRequest ); } @@ -2163,14 +2226,16 @@ private long pollFollowerAsObserver(FollowerState state, long currentTimeMs) { // If the current leader is backing off due to some failure or if the // request has timed out, then we attempt to send the Fetch to another // voter in order to discover if there has been a leader change. - ConnectionState connection = requestManager.getOrCreate(state.leaderId()); - if (connection.hasRequestTimedOut(currentTimeMs)) { + if (requestManager.hasRequestTimedOut(state.leader(), currentTimeMs)) { + // Once the request has timed out backoff the connection + requestManager.reset(state.leader()); backoffMs = maybeSendAnyVoterFetch(currentTimeMs); - connection.reset(); - } else if (connection.isBackingOff(currentTimeMs)) { + } else if (requestManager.isBackingOff(state.leader(), currentTimeMs)) { backoffMs = maybeSendAnyVoterFetch(currentTimeMs); - } else { + } else if (!requestManager.hasAnyInflightRequest(currentTimeMs)) { backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs); + } else { + backoffMs = requestManager.backoffBeforeAvailableBootstrapServer(currentTimeMs); } return Math.min(backoffMs, state.remainingFetchTimeMs(currentTimeMs)); @@ -2189,7 +2254,7 @@ private long maybeSendFetchOrFetchSnapshot(FollowerState state, long currentTime requestSupplier = this::buildFetchRequest; } - return maybeSendRequest(currentTimeMs, state.leaderId(), requestSupplier); + return maybeSendRequest(currentTimeMs, state.leader(), requestSupplier); } private long pollVoted(long currentTimeMs) { @@ -2549,8 +2614,8 @@ public OptionalLong highWatermark() { } } - public Optional voterNode(int id, String listener) { - return partitionState.lastVoterSet().voterNode(id, listener); + public Optional voterNode(int id, ListenerName listenerName) { + return partitionState.lastVoterSet().voterNode(id, listenerName); } // Visible only for test diff --git a/raft/src/main/java/org/apache/kafka/raft/NetworkChannel.java b/raft/src/main/java/org/apache/kafka/raft/NetworkChannel.java index 6c715b3788..20043ac4f9 100644 --- a/raft/src/main/java/org/apache/kafka/raft/NetworkChannel.java +++ b/raft/src/main/java/org/apache/kafka/raft/NetworkChannel.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.raft; -import java.net.InetSocketAddress; +import org.apache.kafka.common.network.ListenerName; /** * A simple network interface with few assumptions. We do not assume ordering @@ -37,10 +37,11 @@ public interface NetworkChannel extends AutoCloseable { void send(RaftRequest.Outbound request); /** - * Update connection information for the given id. + * The name of listener used when sending requests. + * + * @return the name of the listener */ - void updateEndpoint(int id, InetSocketAddress address); + ListenerName listenerName(); default void close() throws InterruptedException {} - } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java b/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java index 5c9c20b763..d7b18ba084 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java @@ -54,6 +54,13 @@ public class QuorumConfig { "For example: 1@localhost:9092,2@localhost:9093,3@localhost:9094"; public static final List DEFAULT_QUORUM_VOTERS = Collections.emptyList(); + public static final String QUORUM_BOOTSTRAP_SERVERS_CONFIG = QUORUM_PREFIX + "bootstrap.servers"; + public static final String QUORUM_BOOTSTRAP_SERVERS_DOC = "List of endpoints to use for " + + "bootstrapping the cluster metadata. The endpoints are specified in comma-separated list " + + "of {host}:{port} entries. For example: " + + "localhost:9092,localhost:9093,localhost:9094."; + public static final List DEFAULT_QUORUM_BOOTSTRAP_SERVERS = Collections.emptyList(); + public static final String QUORUM_ELECTION_TIMEOUT_MS_CONFIG = QUORUM_PREFIX + "election.timeout.ms"; public static final String QUORUM_ELECTION_TIMEOUT_MS_DOC = "Maximum time in milliseconds to wait " + "without being able to fetch from the leader before triggering a new election"; @@ -163,7 +170,7 @@ private static Map parseVoterConnections( List voterEntries, boolean requireRoutableAddresses ) { - Map voterMap = new HashMap<>(); + Map voterMap = new HashMap<>(voterEntries.size()); for (String voterMapEntry : voterEntries) { String[] idAndAddress = voterMapEntry.split("@"); if (idAndAddress.length != 2) { @@ -173,7 +180,7 @@ private static Map parseVoterConnections( Integer voterId = parseVoterId(idAndAddress[0]); String host = Utils.getHost(idAndAddress[1]); - if (host == null) { + if (host == null || !Utils.validHostPattern(host)) { throw new ConfigException("Failed to parse host name from entry " + voterMapEntry + " for the configuration " + QUORUM_VOTERS_CONFIG + ". Each entry should be in the form `{id}@{host}:{port}`."); @@ -199,6 +206,41 @@ private static Map parseVoterConnections( return voterMap; } + public static List parseBootstrapServers(List bootstrapServers) { + return bootstrapServers + .stream() + .map(QuorumConfig::parseBootstrapServer) + .collect(Collectors.toList()); + } + + private static InetSocketAddress parseBootstrapServer(String bootstrapServer) { + String host = Utils.getHost(bootstrapServer); + if (host == null || !Utils.validHostPattern(host)) { + throw new ConfigException( + String.format( + "Failed to parse host name from {} for the configuration {}. Each " + + "entry should be in the form \"{host}:{port}\"", + bootstrapServer, + QUORUM_BOOTSTRAP_SERVERS_CONFIG + ) + ); + } + + Integer port = Utils.getPort(bootstrapServer); + if (port == null) { + throw new ConfigException( + String.format( + "Failed to parse host port from {} for the configuration {}. Each " + + "entry should be in the form \"{host}:{port}\"", + bootstrapServer, + QUORUM_BOOTSTRAP_SERVERS_CONFIG + ) + ); + } + + return InetSocketAddress.createUnresolved(host, port); + } + public static List quorumVoterStringsToNodes(List voters) { return voterConnectionsToNodes(parseVoterConnections(voters)); } @@ -231,4 +273,26 @@ public String toString() { return "non-empty list"; } } + + public static class ControllerQuorumBootstrapServersValidator implements ConfigDef.Validator { + @Override + public void ensureValid(String name, Object value) { + if (value == null) { + throw new ConfigException(name, null); + } + + @SuppressWarnings("unchecked") + List entries = (List) value; + + // Attempt to parse the connect strings + for (String entry : entries) { + parseBootstrapServer(entry); + } + } + + @Override + public String toString() { + return "non-empty list"; + } + } } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 522b708050..b9b17c5f99 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -25,7 +25,9 @@ import java.util.Random; import java.util.function.Supplier; +import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.internals.BatchAccumulator; @@ -81,6 +83,7 @@ public class QuorumState { private final Time time; private final Logger log; private final QuorumStateStore store; + private final ListenerName listenerName; private final Supplier latestVoterSet; private final Supplier latestKraftVersion; private final Random random; @@ -93,6 +96,7 @@ public class QuorumState { public QuorumState( OptionalInt localId, Uuid localDirectoryId, + ListenerName listenerName, Supplier latestVoterSet, Supplier latestKraftVersion, int electionTimeoutMs, @@ -104,6 +108,7 @@ public QuorumState( ) { this.localId = localId; this.localDirectoryId = localDirectoryId; + this.listenerName = listenerName; this.latestVoterSet = latestVoterSet; this.latestKraftVersion = latestKraftVersion; this.electionTimeoutMs = electionTimeoutMs; @@ -115,16 +120,21 @@ public QuorumState( this.logContext = logContext; } - public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateException { - // We initialize in whatever state we were in on shutdown. If we were a leader - // or candidate, probably an election was held, but we will find out about it - // when we send Vote or BeginEpoch requests. - + private ElectionState readElectionState() { ElectionState election; election = store .readElectionState() .orElseGet(() -> ElectionState.withUnknownLeader(0, latestVoterSet.get().voterIds())); + return election; + } + + public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateException { + // We initialize in whatever state we were in on shutdown. If we were a leader + // or candidate, probably an election was held, but we will find out about it + // when we send Vote or BeginEpoch requests. + ElectionState election = readElectionState(); + final EpochState initialState; if (election.hasVoted() && !localId.isPresent()) { throw new IllegalStateException( @@ -191,10 +201,26 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateE logContext ); } else if (election.hasLeader()) { + /* KAFKA-16529 is going to change this so that the leader is not required to be in the set + * of voters. In other words, don't throw an IllegalStateException if the leader is not in + * the set of voters. + */ + Node leader = latestVoterSet + .get() + .voterNode(election.leaderId(), listenerName) + .orElseThrow(() -> + new IllegalStateException( + String.format( + "Leader %s must be in the voter set %s", + election.leaderId(), + latestVoterSet.get() + ) + ) + ); initialState = new FollowerState( time, election.epoch(), - election.leaderId(), + leader, latestVoterSet.get().voterIds(), Optional.empty(), fetchTimeoutMs, @@ -400,28 +426,24 @@ public void transitionToVoted( /** * Become a follower of an elected leader so that we can begin fetching. */ - public void transitionToFollower( - int epoch, - int leaderId - ) { + public void transitionToFollower(int epoch, Node leader) { int currentEpoch = state.epoch(); - if (localId.isPresent() && leaderId == localId.getAsInt()) { - throw new IllegalStateException("Cannot transition to Follower with leaderId=" + leaderId + - " and epoch=" + epoch + " since it matches the local broker.id=" + localId); + if (localId.isPresent() && leader.id() == localId.getAsInt()) { + throw new IllegalStateException("Cannot transition to Follower with leader " + leader + + " and epoch " + epoch + " since it matches the local broker.id " + localId); } else if (epoch < currentEpoch) { - throw new IllegalStateException("Cannot transition to Follower with leaderId=" + leaderId + - " and epoch=" + epoch + " since the current epoch " + currentEpoch + " is larger"); - } else if (epoch == currentEpoch - && (isFollower() || isLeader())) { - throw new IllegalStateException("Cannot transition to Follower with leaderId=" + leaderId + - " and epoch=" + epoch + " from state " + state); + throw new IllegalStateException("Cannot transition to Follower with leader " + leader + + " and epoch " + epoch + " since the current epoch " + currentEpoch + " is larger"); + } else if (epoch == currentEpoch && (isFollower() || isLeader())) { + throw new IllegalStateException("Cannot transition to Follower with leader " + leader + + " and epoch " + epoch + " from state " + state); } durableTransitionTo( new FollowerState( time, epoch, - leaderId, + leader, latestVoterSet.get().voterIds(), state.highWatermark(), fetchTimeoutMs, diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftRequest.java b/raft/src/main/java/org/apache/kafka/raft/RaftRequest.java index 28e63c14ce..bf590f56ab 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftRequest.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftRequest.java @@ -17,13 +17,14 @@ package org.apache.kafka.raft; import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.Node; import java.util.concurrent.CompletableFuture; public abstract class RaftRequest implements RaftMessage { - protected final int correlationId; - protected final ApiMessage data; - protected final long createdTimeMs; + private final int correlationId; + private final ApiMessage data; + private final long createdTimeMs; public RaftRequest(int correlationId, ApiMessage data, long createdTimeMs) { this.correlationId = correlationId; @@ -45,7 +46,7 @@ public long createdTimeMs() { return createdTimeMs; } - public static class Inbound extends RaftRequest { + public final static class Inbound extends RaftRequest { public final CompletableFuture completion = new CompletableFuture<>(); public Inbound(int correlationId, ApiMessage data, long createdTimeMs) { @@ -54,35 +55,37 @@ public Inbound(int correlationId, ApiMessage data, long createdTimeMs) { @Override public String toString() { - return "InboundRequest(" + - "correlationId=" + correlationId + - ", data=" + data + - ", createdTimeMs=" + createdTimeMs + - ')'; + return String.format( + "InboundRequest(correlationId=%d, data=%s, createdTimeMs=%d)", + correlationId(), + data(), + createdTimeMs() + ); } } - public static class Outbound extends RaftRequest { - private final int destinationId; + public final static class Outbound extends RaftRequest { + private final Node destination; public final CompletableFuture completion = new CompletableFuture<>(); - public Outbound(int correlationId, ApiMessage data, int destinationId, long createdTimeMs) { + public Outbound(int correlationId, ApiMessage data, Node destination, long createdTimeMs) { super(correlationId, data, createdTimeMs); - this.destinationId = destinationId; + this.destination = destination; } - public int destinationId() { - return destinationId; + public Node destination() { + return destination; } @Override public String toString() { - return "OutboundRequest(" + - "correlationId=" + correlationId + - ", data=" + data + - ", createdTimeMs=" + createdTimeMs + - ", destinationId=" + destinationId + - ')'; + return String.format( + "OutboundRequest(correlationId=%d, data=%s, createdTimeMs=%d, destination=%s)", + correlationId(), + data(), + createdTimeMs(), + destination + ); } } } diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftResponse.java b/raft/src/main/java/org/apache/kafka/raft/RaftResponse.java index 71101a63bf..9c5047ca92 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftResponse.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftResponse.java @@ -16,11 +16,12 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.Node; import org.apache.kafka.common.protocol.ApiMessage; public abstract class RaftResponse implements RaftMessage { - protected final int correlationId; - protected final ApiMessage data; + private final int correlationId; + private final ApiMessage data; protected RaftResponse(int correlationId, ApiMessage data) { this.correlationId = correlationId; @@ -37,39 +38,41 @@ public ApiMessage data() { return data; } - public static class Inbound extends RaftResponse { - private final int sourceId; + public final static class Inbound extends RaftResponse { + private final Node source; - public Inbound(int correlationId, ApiMessage data, int sourceId) { + public Inbound(int correlationId, ApiMessage data, Node source) { super(correlationId, data); - this.sourceId = sourceId; + this.source = source; } - public int sourceId() { - return sourceId; + public Node source() { + return source; } @Override public String toString() { - return "InboundResponse(" + - "correlationId=" + correlationId + - ", data=" + data + - ", sourceId=" + sourceId + - ')'; + return String.format( + "InboundResponse(correlationId=%d, data=%s, source=%s)", + correlationId(), + data(), + source + ); } } - public static class Outbound extends RaftResponse { + public final static class Outbound extends RaftResponse { public Outbound(int requestId, ApiMessage data) { super(requestId, data); } @Override public String toString() { - return "OutboundResponse(" + - "correlationId=" + correlationId + - ", data=" + data + - ')'; + return String.format( + "OutboundResponse(correlationId=%d, data=%s)", + correlationId(), + data() + ); } } } diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java index 9ff03617e6..86a47eff1c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.message.FetchSnapshotResponseData; import org.apache.kafka.common.message.VoteRequestData; import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.protocol.ApiKeys; @@ -48,6 +49,8 @@ public static ApiMessage errorResponse(ApiKeys apiKey, Errors error) { return new EndQuorumEpochResponseData().setErrorCode(error.code()); case FETCH: return new FetchResponseData().setErrorCode(error.code()); + case FETCH_SNAPSHOT: + return new FetchSnapshotResponseData().setErrorCode(error.code()); default: throw new IllegalArgumentException("Received response for unexpected request type: " + apiKey); } diff --git a/raft/src/main/java/org/apache/kafka/raft/RequestManager.java b/raft/src/main/java/org/apache/kafka/raft/RequestManager.java index 5a5cb003c2..dfdaf9d193 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RequestManager.java +++ b/raft/src/main/java/org/apache/kafka/raft/RequestManager.java @@ -17,96 +17,288 @@ package org.apache.kafka.raft; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; -import java.util.List; +import java.util.Iterator; import java.util.Map; -import java.util.OptionalInt; +import java.util.Optional; import java.util.OptionalLong; import java.util.Random; -import java.util.Set; +import org.apache.kafka.common.Node; +/** + * The request manager keeps tracks of the connection with remote replicas. + * + * When sending a request update this type by calling {@code onRequestSent(Node, long, long)}. When + * the RPC returns a response, update this manager with {@code onResponseResult(Node, long, boolean, long)}. + * + * Connections start in the ready state ({@code isReady(Node, long)} returns true). + * + * When a request times out or completes successfully the collection will transition back to the + * ready state. + * + * When a request completes with an error it still transition to the backoff state until + * {@code retryBackoffMs}. + */ public class RequestManager { - private final Map connections = new HashMap<>(); - private final List voters = new ArrayList<>(); + private final Map connections = new HashMap<>(); + private final ArrayList bootstrapServers; private final int retryBackoffMs; private final int requestTimeoutMs; private final Random random; - public RequestManager(Set voterIds, - int retryBackoffMs, - int requestTimeoutMs, - Random random) { - + public RequestManager( + Collection bootstrapServers, + int retryBackoffMs, + int requestTimeoutMs, + Random random + ) { + this.bootstrapServers = new ArrayList<>(bootstrapServers); this.retryBackoffMs = retryBackoffMs; this.requestTimeoutMs = requestTimeoutMs; - this.voters.addAll(voterIds); this.random = random; + } - for (Integer voterId: voterIds) { - ConnectionState connection = new ConnectionState(voterId); - connections.put(voterId, connection); + /** + * Returns true if there any connection with pending requests. + * + * This is useful for satisfying the invariant that there is only one pending Fetch request. + * If there are more than one pending fetch request, it is possible for the follower to write + * the same offset twice. + * + * @param currentTimeMs the current time + * @return true if the request manager is tracking at least one request + */ + public boolean hasAnyInflightRequest(long currentTimeMs) { + boolean result = false; + + Iterator iterator = connections.values().iterator(); + while (iterator.hasNext()) { + ConnectionState connection = iterator.next(); + if (connection.hasRequestTimedOut(currentTimeMs)) { + // Mark the node as ready after request timeout + iterator.remove(); + } else if (connection.isBackoffComplete(currentTimeMs)) { + // Mark the node as ready after completed backoff + iterator.remove(); + } else if (connection.hasInflightRequest(currentTimeMs)) { + // If there is at least one inflight request, it is enough + // to stop checking the rest of the connections + result = true; + break; + } } - } - public ConnectionState getOrCreate(int id) { - return connections.computeIfAbsent(id, key -> new ConnectionState(id)); + return result; } - public OptionalInt findReadyVoter(long currentTimeMs) { - int startIndex = random.nextInt(voters.size()); - OptionalInt res = OptionalInt.empty(); - for (int i = 0; i < voters.size(); i++) { - int index = (startIndex + i) % voters.size(); - Integer voterId = voters.get(index); - ConnectionState connection = connections.get(voterId); - boolean isReady = connection.isReady(currentTimeMs); + /** + * Returns a random bootstrap node that is ready to receive a request. + * + * This method doesn't return a node if there is at least one request pending. In general this + * method is used to send Fetch requests. Fetch requests have the invariant that there can + * only be one pending Fetch request for the LEO. + * + * @param currentTimeMs the current time + * @return a random ready bootstrap node + */ + public Optional findReadyBootstrapServer(long currentTimeMs) { + // Check that there are no infilght requests accross any of the known nodes not just + // the bootstrap servers + if (hasAnyInflightRequest(currentTimeMs)) { + return Optional.empty(); + } - if (isReady) { - res = OptionalInt.of(voterId); - } else if (connection.inFlightCorrelationId.isPresent()) { - res = OptionalInt.empty(); + int startIndex = random.nextInt(bootstrapServers.size()); + Optional result = Optional.empty(); + for (int i = 0; i < bootstrapServers.size(); i++) { + int index = (startIndex + i) % bootstrapServers.size(); + Node node = bootstrapServers.get(index); + + if (isReady(node, currentTimeMs)) { + result = Optional.of(node); break; } } - return res; + + return result; } - public long backoffBeforeAvailableVoter(long currentTimeMs) { - long minBackoffMs = Long.MAX_VALUE; - for (Integer voterId : voters) { - ConnectionState connection = connections.get(voterId); - if (connection.isReady(currentTimeMs)) { - return 0L; + /** + * Computes the amount of time needed to wait before a bootstrap server is ready for a Fetch + * request. + * + * If there is a connection with a pending request it returns the amount of time to wait until + * the request times out. + * + * Returns zero, if there are no pending request and at least one of the boorstrap servers is + * ready. + * + * If all of the bootstrap servers are backing off and there are no pending requests, return + * the minimum amount of time until a bootstrap server becomes ready. + * + * @param currentTimeMs the current time + * @return the amount of time to wait until bootstrap server can accept a Fetch request + */ + public long backoffBeforeAvailableBootstrapServer(long currentTimeMs) { + long minBackoffMs = retryBackoffMs; + + Iterator iterator = connections.values().iterator(); + while (iterator.hasNext()) { + ConnectionState connection = iterator.next(); + if (connection.hasRequestTimedOut(currentTimeMs)) { + // Mark the node as ready after request timeout + iterator.remove(); + } else if (connection.isBackoffComplete(currentTimeMs)) { + // Mark the node as ready after completed backoff + iterator.remove(); + } else if (connection.hasInflightRequest(currentTimeMs)) { + // There can be at most one inflight fetch request + return connection.remainingRequestTimeMs(currentTimeMs); } else if (connection.isBackingOff(currentTimeMs)) { minBackoffMs = Math.min(minBackoffMs, connection.remainingBackoffMs(currentTimeMs)); - } else { - minBackoffMs = Math.min(minBackoffMs, connection.remainingRequestTimeMs(currentTimeMs)); } } + + // There are no inflight fetch requests so check if there is a ready bootstrap server + for (Node node : bootstrapServers) { + if (isReady(node, currentTimeMs)) { + return 0L; + } + } + + // There are no ready bootstrap servers and inflight fetch requests, return the backoff return minBackoffMs; } + public boolean hasRequestTimedOut(Node node, long timeMs) { + ConnectionState state = connections.get(node.idString()); + if (state == null) { + return false; + } + + return state.hasRequestTimedOut(timeMs); + } + + public boolean isReady(Node node, long timeMs) { + ConnectionState state = connections.get(node.idString()); + if (state == null) { + return true; + } + + boolean ready = state.isReady(timeMs); + if (ready) { + reset(node); + } + + return ready; + } + + public boolean isBackingOff(Node node, long timeMs) { + ConnectionState state = connections.get(node.idString()); + if (state == null) { + return false; + } + + return state.isBackingOff(timeMs); + } + + public long remainingRequestTimeMs(Node node, long timeMs) { + ConnectionState state = connections.get(node.idString()); + if (state == null) { + return 0; + } + + return state.remainingRequestTimeMs(timeMs); + } + + public long remainingBackoffMs(Node node, long timeMs) { + ConnectionState state = connections.get(node.idString()); + if (state == null) { + return 0; + } + + return state.remainingBackoffMs(timeMs); + } + + public boolean isResponseExpected(Node node, long correlationId) { + ConnectionState state = connections.get(node.idString()); + if (state == null) { + return false; + } + + return state.isResponseExpected(correlationId); + } + + /** + * Updates the manager when a response is received. + * + * @param node the source of the response + * @param correlationId the correlation id of the response + * @param success true if the request was successful, false otherwise + * @param timeMs the current time + */ + public void onResponseResult(Node node, long correlationId, boolean success, long timeMs) { + if (isResponseExpected(node, correlationId)) { + if (success) { + // Mark the connection as ready by reseting it + reset(node); + } else { + // Backoff the connection + connections.get(node.idString()).onResponseError(correlationId, timeMs); + } + } + } + + /** + * Updates the manager when a request is sent. + * + * @param node the destination of the request + * @param correlationId the correlation id of the request + * @param timeMs the current time + */ + public void onRequestSent(Node node, long correlationId, long timeMs) { + ConnectionState state = connections.computeIfAbsent( + node.idString(), + key -> new ConnectionState(node, retryBackoffMs, requestTimeoutMs) + ); + + state.onRequestSent(correlationId, timeMs); + } + + public void reset(Node node) { + connections.remove(node.idString()); + } + public void resetAll() { - for (ConnectionState connectionState : connections.values()) - connectionState.reset(); + connections.clear(); } private enum State { - AWAITING_REQUEST, + AWAITING_RESPONSE, BACKING_OFF, READY } - public class ConnectionState { - private final long id; + private final static class ConnectionState { + private final Node node; + private final int retryBackoffMs; + private final int requestTimeoutMs; + private State state = State.READY; private long lastSendTimeMs = 0L; private long lastFailTimeMs = 0L; private OptionalLong inFlightCorrelationId = OptionalLong.empty(); - public ConnectionState(long id) { - this.id = id; + private ConnectionState( + Node node, + int retryBackoffMs, + int requestTimeoutMs + ) { + this.node = node; + this.retryBackoffMs = retryBackoffMs; + this.requestTimeoutMs = requestTimeoutMs; } private boolean isBackoffComplete(long timeMs) { @@ -114,11 +306,7 @@ private boolean isBackoffComplete(long timeMs) { } boolean hasRequestTimedOut(long timeMs) { - return state == State.AWAITING_REQUEST && timeMs >= lastSendTimeMs + requestTimeoutMs; - } - - public long id() { - return id; + return state == State.AWAITING_RESPONSE && timeMs >= lastSendTimeMs + requestTimeoutMs; } boolean isReady(long timeMs) { @@ -136,8 +324,8 @@ boolean isBackingOff(long timeMs) { } } - boolean hasInflightRequest(long timeMs) { - if (state != State.AWAITING_REQUEST) { + private boolean hasInflightRequest(long timeMs) { + if (state != State.AWAITING_RESPONSE) { return false; } else { return !hasRequestTimedOut(timeMs); @@ -174,41 +362,22 @@ void onResponseError(long correlationId, long timeMs) { }); } - void onResponseReceived(long correlationId) { - inFlightCorrelationId.ifPresent(inflightRequestId -> { - if (inflightRequestId == correlationId) { - state = State.READY; - inFlightCorrelationId = OptionalLong.empty(); - } - }); - } - void onRequestSent(long correlationId, long timeMs) { lastSendTimeMs = timeMs; inFlightCorrelationId = OptionalLong.of(correlationId); - state = State.AWAITING_REQUEST; - } - - /** - * Ignore in-flight requests or backoff and become available immediately. This is used - * when there is a state change which usually means in-flight requests are obsolete - * and we need to send new requests. - */ - void reset() { - state = State.READY; - inFlightCorrelationId = OptionalLong.empty(); + state = State.AWAITING_RESPONSE; } @Override public String toString() { - return "ConnectionState(" + - "id=" + id + - ", state=" + state + - ", lastSendTimeMs=" + lastSendTimeMs + - ", lastFailTimeMs=" + lastFailTimeMs + - ", inFlightCorrelationId=" + inFlightCorrelationId + - ')'; + return String.format( + "ConnectionState(node=%s, state=%s, lastSendTimeMs=%d, lastFailTimeMs=%d, inFlightCorrelationId=%d)", + node, + state, + lastSendTimeMs, + lastFailTimeMs, + inFlightCorrelationId + ); } } - } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index 16662e06ee..3ab41f5788 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -28,11 +28,13 @@ import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.message.VotersRecord; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; /** @@ -55,15 +57,41 @@ final public class VoterSet { } /** - * Returns the socket address for a given voter at a given listener. + * Returns the node information for all the given voter ids and listener. * - * @param voter the id of the voter - * @param listener the name of the listener - * @return the socket address if it exists, otherwise {@code Optional.empty()} + * @param voterIds the ids of the voters + * @param listenerName the name of the listener + * @return the node information for all of the voter ids + * @throws IllegalArgumentException if there are missing endpoints */ - public Optional voterAddress(int voter, String listener) { - return Optional.ofNullable(voters.get(voter)) - .flatMap(voterNode -> voterNode.address(listener)); + public Set voterNodes(Stream voterIds, ListenerName listenerName) { + return voterIds + .map(voterId -> + voterNode(voterId, listenerName).orElseThrow(() -> + new IllegalArgumentException( + String.format( + "Unable to find endpoint for voter %d and listener %s in %s", + voterId, + listenerName, + voters + ) + ) + ) + ) + .collect(Collectors.toSet()); + } + + /** + * Returns the node information for a given voter id and listener. + * + * @param voterId the id of the voter + * @param listenerName the name of the listener + * @return the node information if it exists, otherwise {@code Optional.empty()} + */ + public Optional voterNode(int voterId, ListenerName listenerName) { + return Optional.ofNullable(voters.get(voterId)) + .flatMap(voterNode -> voterNode.address(listenerName)) + .map(address -> new Node(voterId, address.getHostString(), address.getPort())); } /** @@ -166,7 +194,7 @@ public VotersRecord toVotersRecord(short version) { .stream() .map(entry -> new VotersRecord.Endpoint() - .setName(entry.getKey()) + .setName(entry.getKey().value()) .setHost(entry.getValue().getHostString()) .setPort(entry.getValue().getPort()) ) @@ -247,12 +275,12 @@ public String toString() { public final static class VoterNode { private final ReplicaKey voterKey; - private final Map listeners; + private final Map listeners; private final SupportedVersionRange supportedKRaftVersion; VoterNode( ReplicaKey voterKey, - Map listeners, + Map listeners, SupportedVersionRange supportedKRaftVersion ) { this.voterKey = voterKey; @@ -264,7 +292,7 @@ public ReplicaKey voterKey() { return voterKey; } - Map listeners() { + Map listeners() { return listeners; } @@ -273,7 +301,7 @@ SupportedVersionRange supportedKRaftVersion() { } - Optional address(String listener) { + Optional address(ListenerName listener) { return Optional.ofNullable(listeners.get(listener)); } @@ -323,9 +351,12 @@ public static VoterSet fromVotersRecord(VotersRecord voters) { directoryId = Optional.empty(); } - Map listeners = new HashMap<>(voter.endpoints().size()); + Map listeners = new HashMap<>(voter.endpoints().size()); for (VotersRecord.Endpoint endpoint : voter.endpoints()) { - listeners.put(endpoint.name(), InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())); + listeners.put( + ListenerName.normalised(endpoint.name()), + InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port()) + ); } voterNodes.put( @@ -351,7 +382,7 @@ public static VoterSet fromVotersRecord(VotersRecord voters) { * @param voters the socket addresses by voter id * @return the voter set */ - public static VoterSet fromInetSocketAddresses(String listener, Map voters) { + public static VoterSet fromInetSocketAddresses(ListenerName listener, Map voters) { Map voterNodes = voters .entrySet() .stream() @@ -368,16 +399,4 @@ public static VoterSet fromInetSocketAddresses(String listener, Map voterNode(int id, String listener) { - VoterNode voterNode = voters.get(id); - if (voterNode == null) { - return Optional.empty(); - } - InetSocketAddress address = voterNode.listeners.get(listener); - if (address == null) { - return Optional.empty(); - } - return Optional.of(new Node(id, address.getHostString(), address.getPort())); - } } diff --git a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index 524a93fa1d..9aa5eeab49 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -26,11 +26,10 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.Map; import java.util.Optional; +import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -60,7 +59,7 @@ private CandidateState newCandidateState(VoterSet voters) { @Test public void testSingleNodeQuorum() { - CandidateState state = newCandidateState(voterSetWithLocal(Collections.emptyList())); + CandidateState state = newCandidateState(voterSetWithLocal(IntStream.empty())); assertTrue(state.isVoteGranted()); assertFalse(state.isVoteRejected()); assertEquals(Collections.emptySet(), state.unrecordedVoters()); @@ -70,7 +69,7 @@ public void testSingleNodeQuorum() { public void testTwoNodeQuorumVoteRejected() { int otherNodeId = 1; CandidateState state = newCandidateState( - voterSetWithLocal(Collections.singletonList(otherNodeId)) + voterSetWithLocal(IntStream.of(otherNodeId)) ); assertFalse(state.isVoteGranted()); assertFalse(state.isVoteRejected()); @@ -84,7 +83,7 @@ public void testTwoNodeQuorumVoteRejected() { public void testTwoNodeQuorumVoteGranted() { int otherNodeId = 1; CandidateState state = newCandidateState( - voterSetWithLocal(Collections.singletonList(otherNodeId)) + voterSetWithLocal(IntStream.of(otherNodeId)) ); assertFalse(state.isVoteGranted()); assertFalse(state.isVoteRejected()); @@ -100,7 +99,7 @@ public void testThreeNodeQuorumVoteGranted() { int node1 = 1; int node2 = 2; CandidateState state = newCandidateState( - voterSetWithLocal(Arrays.asList(node1, node2)) + voterSetWithLocal(IntStream.of(node1, node2)) ); assertFalse(state.isVoteGranted()); assertFalse(state.isVoteRejected()); @@ -120,7 +119,7 @@ public void testThreeNodeQuorumVoteRejected() { int node1 = 1; int node2 = 2; CandidateState state = newCandidateState( - voterSetWithLocal(Arrays.asList(node1, node2)) + voterSetWithLocal(IntStream.of(node1, node2)) ); assertFalse(state.isVoteGranted()); assertFalse(state.isVoteRejected()); @@ -139,7 +138,7 @@ public void testThreeNodeQuorumVoteRejected() { public void testCannotRejectVoteFromLocalId() { int otherNodeId = 1; CandidateState state = newCandidateState( - voterSetWithLocal(Collections.singletonList(otherNodeId)) + voterSetWithLocal(IntStream.of(otherNodeId)) ); assertThrows( IllegalArgumentException.class, @@ -151,7 +150,7 @@ public void testCannotRejectVoteFromLocalId() { public void testCannotChangeVoteGrantedToRejected() { int otherNodeId = 1; CandidateState state = newCandidateState( - voterSetWithLocal(Collections.singletonList(otherNodeId)) + voterSetWithLocal(IntStream.of(otherNodeId)) ); assertTrue(state.recordGrantedVote(otherNodeId)); assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(otherNodeId)); @@ -162,7 +161,7 @@ public void testCannotChangeVoteGrantedToRejected() { public void testCannotChangeVoteRejectedToGranted() { int otherNodeId = 1; CandidateState state = newCandidateState( - voterSetWithLocal(Collections.singletonList(otherNodeId)) + voterSetWithLocal(IntStream.of(otherNodeId)) ); assertTrue(state.recordRejectedVote(otherNodeId)); assertThrows(IllegalArgumentException.class, () -> state.recordGrantedVote(otherNodeId)); @@ -172,7 +171,7 @@ public void testCannotChangeVoteRejectedToGranted() { @Test public void testCannotGrantOrRejectNonVoters() { int nonVoterId = 1; - CandidateState state = newCandidateState(voterSetWithLocal(Collections.emptyList())); + CandidateState state = newCandidateState(voterSetWithLocal(IntStream.empty())); assertThrows(IllegalArgumentException.class, () -> state.recordGrantedVote(nonVoterId)); assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(nonVoterId)); } @@ -181,7 +180,7 @@ public void testCannotGrantOrRejectNonVoters() { public void testIdempotentGrant() { int otherNodeId = 1; CandidateState state = newCandidateState( - voterSetWithLocal(Collections.singletonList(otherNodeId)) + voterSetWithLocal(IntStream.of(otherNodeId)) ); assertTrue(state.recordGrantedVote(otherNodeId)); assertFalse(state.recordGrantedVote(otherNodeId)); @@ -191,7 +190,7 @@ public void testIdempotentGrant() { public void testIdempotentReject() { int otherNodeId = 1; CandidateState state = newCandidateState( - voterSetWithLocal(Collections.singletonList(otherNodeId)) + voterSetWithLocal(IntStream.of(otherNodeId)) ); assertTrue(state.recordRejectedVote(otherNodeId)); assertFalse(state.recordRejectedVote(otherNodeId)); @@ -201,7 +200,7 @@ public void testIdempotentReject() { @ValueSource(booleans = {true, false}) public void testGrantVote(boolean isLogUpToDate) { CandidateState state = newCandidateState( - voterSetWithLocal(Arrays.asList(1, 2, 3)) + voterSetWithLocal(IntStream.of(1, 2, 3)) ); assertFalse(state.canGrantVote(ReplicaKey.of(0, Optional.empty()), isLogUpToDate)); @@ -212,7 +211,7 @@ public void testGrantVote(boolean isLogUpToDate) { @Test public void testElectionState() { - VoterSet voters = voterSetWithLocal(Arrays.asList(1, 2, 3)); + VoterSet voters = voterSetWithLocal(IntStream.of(1, 2, 3)); CandidateState state = newCandidateState(voters); assertEquals( ElectionState.withVotedCandidate( @@ -228,11 +227,11 @@ public void testElectionState() { public void testInvalidVoterSet() { assertThrows( IllegalArgumentException.class, - () -> newCandidateState(VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true))) + () -> newCandidateState(VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true))) ); } - private VoterSet voterSetWithLocal(Collection remoteVoters) { + private VoterSet voterSetWithLocal(IntStream remoteVoters) { Map voterMap = VoterSetTest.voterMap(remoteVoters, true); voterMap.put(localNode.voterKey().id(), localNode); diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index 1894472fa3..ab699159d0 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.Node; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -38,7 +39,7 @@ public class FollowerStateTest { private final LogContext logContext = new LogContext(); private final int epoch = 5; private final int fetchTimeoutMs = 15000; - int leaderId = 3; + private final Node leader = new Node(3, "mock-host-3", 1234); private FollowerState newFollowerState( Set voters, @@ -47,7 +48,7 @@ private FollowerState newFollowerState( return new FollowerState( time, epoch, - leaderId, + leader, voters, highWatermark, fetchTimeoutMs, @@ -96,4 +97,10 @@ public void testGrantVote(boolean isLogUpToDate) { assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)); } + @Test + public void testLeaderNode() { + FollowerState state = newFollowerState(Utils.mkSet(0, 1, 2), Optional.empty()); + + assertEquals(leader, state.leader()); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java index 4a580a124b..2455990e77 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java @@ -26,7 +26,10 @@ import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.message.FetchSnapshotRequestData; +import org.apache.kafka.common.message.FetchSnapshotResponseData; import org.apache.kafka.common.message.VoteResponseData; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; @@ -39,6 +42,8 @@ import org.apache.kafka.common.requests.EndQuorumEpochResponse; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; +import org.apache.kafka.common.requests.FetchSnapshotRequest; +import org.apache.kafka.common.requests.FetchSnapshotResponse; import org.apache.kafka.common.requests.VoteRequest; import org.apache.kafka.common.requests.VoteResponse; import org.apache.kafka.common.utils.MockTime; @@ -47,8 +52,8 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; -import java.net.InetSocketAddress; import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutionException; @@ -80,7 +85,8 @@ public void update(Time time, MockClient.MetadataUpdate update) { } ApiKeys.VOTE, ApiKeys.BEGIN_QUORUM_EPOCH, ApiKeys.END_QUORUM_EPOCH, - ApiKeys.FETCH + ApiKeys.FETCH, + ApiKeys.FETCH_SNAPSHOT ); private final int requestTimeoutMs = 30000; @@ -88,35 +94,40 @@ public void update(Time time, MockClient.MetadataUpdate update) { } private final MockClient client = new MockClient(time, new StubMetadataUpdater()); private final TopicPartition topicPartition = new TopicPartition("topic", 0); private final Uuid topicId = Uuid.randomUuid(); - private final KafkaNetworkChannel channel = new KafkaNetworkChannel(time, client, requestTimeoutMs, "test-raft"); + private final KafkaNetworkChannel channel = new KafkaNetworkChannel( + time, + ListenerName.normalised("NAME"), + client, + requestTimeoutMs, + "test-raft" + ); + + private Node nodeWithId(boolean withId) { + int id = withId ? 2 : -2; + return new Node(id, "127.0.0.1", 9092); + } @BeforeEach public void setupSupportedApis() { - List supportedApis = RAFT_APIS.stream().map( - ApiVersionsResponse::toApiVersion).collect(Collectors.toList()); + List supportedApis = RAFT_APIS + .stream() + .map(ApiVersionsResponse::toApiVersion) + .collect(Collectors.toList()); client.setNodeApiVersions(NodeApiVersions.create(supportedApis)); } - @Test - public void testSendToUnknownDestination() throws ExecutionException, InterruptedException { - int destinationId = 2; - assertBrokerNotAvailable(destinationId); - } - - @Test - public void testSendToBlackedOutDestination() throws ExecutionException, InterruptedException { - int destinationId = 2; - Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); - client.backoff(destinationNode, 500); - assertBrokerNotAvailable(destinationId); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testSendToBlackedOutDestination(boolean withDestinationId) throws ExecutionException, InterruptedException { + Node destination = nodeWithId(withDestinationId); + client.backoff(destination, 500); + assertBrokerNotAvailable(destination); } @Test public void testWakeupClientOnSend() throws InterruptedException, ExecutionException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); client.enableBlockingUntilWakeup(1); @@ -132,7 +143,7 @@ public void testWakeupClientOnSend() throws InterruptedException, ExecutionExcep client.prepareResponseFrom(response, destinationNode, false); ioThread.start(); - RaftRequest.Outbound request = sendTestRequest(ApiKeys.FETCH, destinationId); + RaftRequest.Outbound request = sendTestRequest(ApiKeys.FETCH, destinationNode); ioThread.join(); assertResponseCompleted(request, Errors.INVALID_REQUEST); @@ -142,12 +153,11 @@ public void testWakeupClientOnSend() throws InterruptedException, ExecutionExcep public void testSendAndDisconnect() throws ExecutionException, InterruptedException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); for (ApiKeys apiKey : RAFT_APIS) { AbstractResponse response = buildResponse(buildTestErrorResponse(apiKey, Errors.INVALID_REQUEST)); client.prepareResponseFrom(response, destinationNode, true); - sendAndAssertErrorResponse(apiKey, destinationId, Errors.BROKER_NOT_AVAILABLE); + sendAndAssertErrorResponse(apiKey, destinationNode, Errors.BROKER_NOT_AVAILABLE); } } @@ -155,35 +165,33 @@ public void testSendAndDisconnect() throws ExecutionException, InterruptedExcept public void testSendAndFailAuthentication() throws ExecutionException, InterruptedException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); for (ApiKeys apiKey : RAFT_APIS) { client.createPendingAuthenticationError(destinationNode, 100); - sendAndAssertErrorResponse(apiKey, destinationId, Errors.NETWORK_EXCEPTION); + sendAndAssertErrorResponse(apiKey, destinationNode, Errors.NETWORK_EXCEPTION); // reset to clear backoff time client.reset(); } } - private void assertBrokerNotAvailable(int destinationId) throws ExecutionException, InterruptedException { + private void assertBrokerNotAvailable(Node destination) throws ExecutionException, InterruptedException { for (ApiKeys apiKey : RAFT_APIS) { - sendAndAssertErrorResponse(apiKey, destinationId, Errors.BROKER_NOT_AVAILABLE); + sendAndAssertErrorResponse(apiKey, destination, Errors.BROKER_NOT_AVAILABLE); } } - @Test - public void testSendAndReceiveOutboundRequest() throws ExecutionException, InterruptedException { - int destinationId = 2; - Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testSendAndReceiveOutboundRequest(boolean withDestinationId) throws ExecutionException, InterruptedException { + Node destination = nodeWithId(withDestinationId); for (ApiKeys apiKey : RAFT_APIS) { Errors expectedError = Errors.INVALID_REQUEST; AbstractResponse response = buildResponse(buildTestErrorResponse(apiKey, expectedError)); - client.prepareResponseFrom(response, destinationNode); + client.prepareResponseFrom(response, destination); System.out.println("api key " + apiKey + ", response " + response); - sendAndAssertErrorResponse(apiKey, destinationId, expectedError); + sendAndAssertErrorResponse(apiKey, destination, expectedError); } } @@ -191,11 +199,10 @@ public void testSendAndReceiveOutboundRequest() throws ExecutionException, Inter public void testUnsupportedVersionError() throws ExecutionException, InterruptedException { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); for (ApiKeys apiKey : RAFT_APIS) { client.prepareUnsupportedVersionResponse(request -> request.apiKey() == apiKey); - sendAndAssertErrorResponse(apiKey, destinationId, Errors.UNSUPPORTED_VERSION); + sendAndAssertErrorResponse(apiKey, destinationNode, Errors.UNSUPPORTED_VERSION); } } @@ -204,8 +211,7 @@ public void testUnsupportedVersionError() throws ExecutionException, Interrupted public void testFetchRequestDowngrade(short version) { int destinationId = 2; Node destinationNode = new Node(destinationId, "127.0.0.1", 9092); - channel.updateEndpoint(destinationId, new InetSocketAddress(destinationNode.host(), destinationNode.port())); - sendTestRequest(ApiKeys.FETCH, destinationId); + sendTestRequest(ApiKeys.FETCH, destinationNode); channel.pollOnce(); assertEquals(1, client.requests().size()); @@ -220,27 +226,39 @@ public void testFetchRequestDowngrade(short version) { } } - private RaftRequest.Outbound sendTestRequest(ApiKeys apiKey, int destinationId) { + private RaftRequest.Outbound sendTestRequest(ApiKeys apiKey, Node destination) { int correlationId = channel.newCorrelationId(); long createdTimeMs = time.milliseconds(); ApiMessage apiRequest = buildTestRequest(apiKey); - RaftRequest.Outbound request = new RaftRequest.Outbound(correlationId, apiRequest, destinationId, createdTimeMs); + RaftRequest.Outbound request = new RaftRequest.Outbound( + correlationId, + apiRequest, + destination, + createdTimeMs + ); channel.send(request); return request; } - private void assertResponseCompleted(RaftRequest.Outbound request, Errors expectedError) throws ExecutionException, InterruptedException { + private void assertResponseCompleted( + RaftRequest.Outbound request, + Errors expectedError + ) throws ExecutionException, InterruptedException { assertTrue(request.completion.isDone()); RaftResponse.Inbound response = request.completion.get(); - assertEquals(request.destinationId(), response.sourceId()); - assertEquals(request.correlationId, response.correlationId); - assertEquals(request.data.apiKey(), response.data.apiKey()); - assertEquals(expectedError, extractError(response.data)); + assertEquals(request.destination(), response.source()); + assertEquals(request.correlationId(), response.correlationId()); + assertEquals(request.data().apiKey(), response.data().apiKey()); + assertEquals(expectedError, extractError(response.data())); } - private void sendAndAssertErrorResponse(ApiKeys apiKey, int destinationId, Errors error) throws ExecutionException, InterruptedException { - RaftRequest.Outbound request = sendTestRequest(apiKey, destinationId); + private void sendAndAssertErrorResponse( + ApiKeys apiKey, + Node destination, + Errors error + ) throws ExecutionException, InterruptedException { + RaftRequest.Outbound request = sendTestRequest(apiKey, destination); channel.pollOnce(); assertResponseCompleted(request, error); } @@ -252,12 +270,20 @@ private ApiMessage buildTestRequest(ApiKeys key) { switch (key) { case BEGIN_QUORUM_EPOCH: return BeginQuorumEpochRequest.singletonRequest(topicPartition, clusterId, leaderEpoch, leaderId); + case END_QUORUM_EPOCH: - return EndQuorumEpochRequest.singletonRequest(topicPartition, clusterId, leaderId, leaderEpoch, - Collections.singletonList(2)); + return EndQuorumEpochRequest.singletonRequest( + topicPartition, + clusterId, + leaderId, + leaderEpoch, + Collections.singletonList(2) + ); + case VOTE: int lastEpoch = 4; return VoteRequest.singletonRequest(topicPartition, clusterId, leaderEpoch, leaderId, lastEpoch, 329); + case FETCH: FetchRequestData request = RaftUtil.singletonFetchRequest(topicPartition, topicId, fetchPartition -> { fetchPartition @@ -267,6 +293,21 @@ private ApiMessage buildTestRequest(ApiKeys key) { }); request.setReplicaState(new FetchRequestData.ReplicaState().setReplicaId(1)); return request; + + case FETCH_SNAPSHOT: + return FetchSnapshotRequest.singleton( + clusterId, + 1, + topicPartition, + snapshotPartition -> snapshotPartition + .setCurrentLeaderEpoch(5) + .setSnapshotId(new FetchSnapshotRequestData.SnapshotId() + .setEpoch(4) + .setEndOffset(323) + ) + .setPosition(10) + ); + default: throw new AssertionError("Unexpected api " + key); } @@ -282,6 +323,8 @@ private ApiMessage buildTestErrorResponse(ApiKeys key, Errors error) { return VoteResponse.singletonResponse(error, topicPartition, Errors.NONE, 1, 5, false); case FETCH: return new FetchResponseData().setErrorCode(error.code()); + case FETCH_SNAPSHOT: + return new FetchSnapshotResponseData().setErrorCode(error.code()); default: throw new AssertionError("Unexpected api " + key); } @@ -289,28 +332,36 @@ private ApiMessage buildTestErrorResponse(ApiKeys key, Errors error) { private Errors extractError(ApiMessage response) { short code; - if (response instanceof BeginQuorumEpochResponseData) + if (response instanceof BeginQuorumEpochResponseData) { code = ((BeginQuorumEpochResponseData) response).errorCode(); - else if (response instanceof EndQuorumEpochResponseData) + } else if (response instanceof EndQuorumEpochResponseData) { code = ((EndQuorumEpochResponseData) response).errorCode(); - else if (response instanceof FetchResponseData) + } else if (response instanceof FetchResponseData) { code = ((FetchResponseData) response).errorCode(); - else if (response instanceof VoteResponseData) + } else if (response instanceof VoteResponseData) { code = ((VoteResponseData) response).errorCode(); - else + } else if (response instanceof FetchSnapshotResponseData) { + code = ((FetchSnapshotResponseData) response).errorCode(); + } else { throw new IllegalArgumentException("Unexpected type for responseData: " + response); + } + return Errors.forCode(code); } private AbstractResponse buildResponse(ApiMessage responseData) { - if (responseData instanceof VoteResponseData) + if (responseData instanceof VoteResponseData) { return new VoteResponse((VoteResponseData) responseData); - if (responseData instanceof BeginQuorumEpochResponseData) + } else if (responseData instanceof BeginQuorumEpochResponseData) { return new BeginQuorumEpochResponse((BeginQuorumEpochResponseData) responseData); - if (responseData instanceof EndQuorumEpochResponseData) + } else if (responseData instanceof EndQuorumEpochResponseData) { return new EndQuorumEpochResponse((EndQuorumEpochResponseData) responseData); - if (responseData instanceof FetchResponseData) + } else if (responseData instanceof FetchResponseData) { return new FetchResponse((FetchResponseData) responseData); - throw new IllegalArgumentException("Unexpected type for responseData: " + responseData); + } else if (responseData instanceof FetchSnapshotResponseData) { + return new FetchSnapshotResponse((FetchSnapshotResponseData) responseData); + } else { + throw new IllegalArgumentException("Unexpected type for responseData: " + responseData); + } } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 3fcbec4229..299fa819d5 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -153,8 +153,8 @@ public void testFollowerListenerNotified(boolean entireLog) throws Exception { RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch()); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, localLogEndOffset, Errors.NONE) ); @@ -195,8 +195,8 @@ public void testSecondListenerNotified(boolean entireLog) throws Exception { RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch()); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, localLogEndOffset, Errors.NONE) ); @@ -1032,8 +1032,8 @@ public void testFetchResponseWithInvalidSnapshotId() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, invalidEpoch, 200L) ); @@ -1049,8 +1049,8 @@ public void testFetchResponseWithInvalidSnapshotId() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, invalidEndOffset, 200L) ); @@ -1091,8 +1091,8 @@ public void testFetchResponseWithSnapshotId() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) ); @@ -1116,8 +1116,8 @@ public void testFetchResponseWithSnapshotId() throws Exception { } context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), fetchSnapshotResponse( context.metadataPartition, epoch, @@ -1162,8 +1162,8 @@ public void testFetchSnapshotResponsePartialData() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) ); @@ -1190,8 +1190,8 @@ public void testFetchSnapshotResponsePartialData() throws Exception { sendingBuffer.limit(sendingBuffer.limit() / 2); context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), fetchSnapshotResponse( context.metadataPartition, epoch, @@ -1219,8 +1219,8 @@ public void testFetchSnapshotResponsePartialData() throws Exception { sendingBuffer.position(Math.toIntExact(request.position())); context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), fetchSnapshotResponse( context.metadataPartition, epoch, @@ -1265,8 +1265,8 @@ public void testFetchSnapshotResponseMissingSnapshot() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) ); @@ -1284,8 +1284,8 @@ public void testFetchSnapshotResponseMissingSnapshot() throws Exception { // Reply with a snapshot not found error context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), FetchSnapshotResponse.singleton( context.metadataPartition, responsePartitionSnapshot -> { @@ -1323,8 +1323,8 @@ public void testFetchSnapshotResponseFromNewerEpochNotLeader() throws Exception context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, firstLeaderId, snapshotId, 200L) ); @@ -1342,8 +1342,8 @@ public void testFetchSnapshotResponseFromNewerEpochNotLeader() throws Exception // Reply with new leader response context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), FetchSnapshotResponse.singleton( context.metadataPartition, responsePartitionSnapshot -> { @@ -1380,8 +1380,8 @@ public void testFetchSnapshotResponseFromNewerEpochLeader() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) ); @@ -1399,8 +1399,8 @@ public void testFetchSnapshotResponseFromNewerEpochLeader() throws Exception { // Reply with new leader epoch context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), FetchSnapshotResponse.singleton( context.metadataPartition, responsePartitionSnapshot -> { @@ -1437,8 +1437,8 @@ public void testFetchSnapshotResponseFromOlderEpoch() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) ); @@ -1456,8 +1456,8 @@ public void testFetchSnapshotResponseFromOlderEpoch() throws Exception { // Reply with unknown leader epoch context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), FetchSnapshotResponse.singleton( context.metadataPartition, responsePartitionSnapshot -> { @@ -1504,8 +1504,8 @@ public void testFetchSnapshotResponseWithInvalidId() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) ); @@ -1523,8 +1523,8 @@ public void testFetchSnapshotResponseWithInvalidId() throws Exception { // Reply with an invalid snapshot id endOffset context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), FetchSnapshotResponse.singleton( context.metadataPartition, responsePartitionSnapshot -> { @@ -1550,8 +1550,8 @@ public void testFetchSnapshotResponseWithInvalidId() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) ); @@ -1570,8 +1570,8 @@ public void testFetchSnapshotResponseWithInvalidId() throws Exception { // Reply with an invalid snapshot id epoch context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), FetchSnapshotResponse.singleton( context.metadataPartition, responsePartitionSnapshot -> { @@ -1614,8 +1614,8 @@ public void testFetchSnapshotResponseToNotFollower() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); context.deliverResponse( - fetchRequest.correlationId, - fetchRequest.destinationId(), + fetchRequest.correlationId(), + fetchRequest.destination(), snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) ); @@ -1642,8 +1642,8 @@ public void testFetchSnapshotResponseToNotFollower() throws Exception { // Send the response late context.deliverResponse( - snapshotRequest.correlationId, - snapshotRequest.destinationId(), + snapshotRequest.correlationId(), + snapshotRequest.destination(), FetchSnapshotResponse.singleton( context.metadataPartition, responsePartitionSnapshot -> { @@ -1805,14 +1805,17 @@ public void testCreateSnapshotAsFollowerWithInvalidSnapshotId() throws Exception // Poll for our first fetch request context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(voters.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); // The response does not advance the high watermark List records1 = Arrays.asList("a", "b", "c"); MemoryRecords batch1 = context.buildBatch(0L, 3, records1); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(epoch, leaderId, batch1, 0L, Errors.NONE)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, leaderId, batch1, 0L, Errors.NONE) + ); context.client.poll(); // 2) The high watermark must be larger than or equal to the snapshotId's endOffset @@ -1827,13 +1830,16 @@ public void testCreateSnapshotAsFollowerWithInvalidSnapshotId() throws Exception // The high watermark advances to be larger than log.endOffsetForEpoch(3), to test the case 3 context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(voters.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, epoch, 3L, 3); List records2 = Arrays.asList("d", "e", "f"); MemoryRecords batch2 = context.buildBatch(3L, 4, records2); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(epoch, leaderId, batch2, 6L, Errors.NONE)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, leaderId, batch2, 6L, Errors.NONE) + ); context.client.poll(); assertEquals(6L, context.client.highWatermark().getAsLong()); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index c531e5860a..049b648d88 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -51,6 +51,7 @@ import org.mockito.Mockito; import java.io.IOException; +import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -62,6 +63,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import static java.util.Collections.singletonList; import static org.apache.kafka.raft.RaftClientTestContext.Builder.DEFAULT_ELECTION_TIMEOUT_MS; @@ -274,8 +276,12 @@ public void testInitializeAsResignedLeaderFromStateStore() throws Exception { assertThrows(NotLeaderException.class, () -> context.client.scheduleAppend(epoch, Arrays.asList("a", "b"))); context.pollUntilRequest(); - int correlationId = context.assertSentEndQuorumEpochRequest(epoch, 1); - context.deliverResponse(correlationId, 1, context.endEpochResponse(epoch, OptionalInt.of(localId))); + RaftRequest.Outbound request = context.assertSentEndQuorumEpochRequest(epoch, 1); + context.deliverResponse( + request.correlationId(), + request.destination(), + context.endEpochResponse(epoch, OptionalInt.of(localId)) + ); context.client.poll(); context.time.sleep(context.electionTimeoutMs()); @@ -389,14 +395,17 @@ public void testEndQuorumEpochRetriesWhileResigned() throws Exception { // Respond to one of the requests so that we can verify that no additional // request to this node is sent. RaftRequest.Outbound endEpochOutbound = requests.get(0); - context.deliverResponse(endEpochOutbound.correlationId, endEpochOutbound.destinationId(), - context.endEpochResponse(epoch, OptionalInt.of(localId))); + context.deliverResponse( + endEpochOutbound.correlationId(), + endEpochOutbound.destination(), + context.endEpochResponse(epoch, OptionalInt.of(localId)) + ); context.client.poll(); assertEquals(Collections.emptyList(), context.channel.drainSendQueue()); // Now sleep for the request timeout and verify that we get only one // retried request from the voter that hasn't responded yet. - int nonRespondedId = requests.get(1).destinationId(); + int nonRespondedId = requests.get(1).destination().id(); context.time.sleep(6000); context.pollUntilRequest(); List retries = context.collectEndQuorumRequests( @@ -573,7 +582,7 @@ public void testElectionTimeoutAfterUserInitiatedResign() throws Exception { context.pollUntil(context.client.quorum()::isResigned); context.pollUntilRequest(); - int correlationId = context.assertSentEndQuorumEpochRequest(resignedEpoch, otherNodeId); + RaftRequest.Outbound request = context.assertSentEndQuorumEpochRequest(resignedEpoch, otherNodeId); EndQuorumEpochResponseData response = EndQuorumEpochResponse.singletonResponse( Errors.NONE, @@ -583,7 +592,7 @@ public void testElectionTimeoutAfterUserInitiatedResign() throws Exception { localId ); - context.deliverResponse(correlationId, otherNodeId, response); + context.deliverResponse(request.correlationId(), request.destination(), response); context.client.poll(); // We do not resend `EndQuorumRequest` once the other voter has acknowledged it. @@ -644,11 +653,14 @@ public void testCannotResignIfObserver() throws Exception { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(voters.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, 0, 0L, 0); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH) + ); context.client.poll(); context.assertElectedLeader(epoch, leaderId); @@ -686,8 +698,12 @@ public void testInitializeAsCandidateAndBecomeLeader() throws Exception { context.pollUntilRequest(); context.assertVotedCandidate(1, localId); - int correlationId = context.assertSentVoteRequest(1, 0, 0L, 1); - context.deliverResponse(correlationId, otherNodeId, context.voteResponse(true, Optional.empty(), 1)); + RaftRequest.Outbound request = context.assertSentVoteRequest(1, 0, 0L, 1); + context.deliverResponse( + request.correlationId(), + request.destination(), + context.voteResponse(true, Optional.empty(), 1) + ); // Become leader after receiving the vote context.pollUntil(() -> context.log.endOffset().offset == 1L); @@ -726,8 +742,12 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree() throws Excep context.pollUntilRequest(); context.assertVotedCandidate(1, localId); - int correlationId = context.assertSentVoteRequest(1, 0, 0L, 2); - context.deliverResponse(correlationId, firstNodeId, context.voteResponse(true, Optional.empty(), 1)); + RaftRequest.Outbound request = context.assertSentVoteRequest(1, 0, 0L, 2); + context.deliverResponse( + request.correlationId(), + request.destination(), + context.voteResponse(true, Optional.empty(), 1) + ); // Become leader after receiving the vote context.pollUntil(() -> context.log.endOffset().offset == 1L); @@ -1102,19 +1122,27 @@ public void testVoteRequestTimeout() throws Exception { context.pollUntilRequest(); context.assertVotedCandidate(epoch, localId); - int correlationId = context.assertSentVoteRequest(epoch, 0, 0L, 1); + RaftRequest.Outbound request = context.assertSentVoteRequest(epoch, 0, 0L, 1); context.time.sleep(context.requestTimeoutMs()); context.client.poll(); - int retryCorrelationId = context.assertSentVoteRequest(epoch, 0, 0L, 1); + RaftRequest.Outbound retryRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); // We will ignore the timed out response if it arrives late - context.deliverResponse(correlationId, otherNodeId, context.voteResponse(true, Optional.empty(), 1)); + context.deliverResponse( + request.correlationId(), + request.destination(), + context.voteResponse(true, Optional.empty(), 1) + ); context.client.poll(); context.assertVotedCandidate(epoch, localId); // Become leader after receiving the retry response - context.deliverResponse(retryCorrelationId, otherNodeId, context.voteResponse(true, Optional.empty(), 1)); + context.deliverResponse( + retryRequest.correlationId(), + retryRequest.destination(), + context.voteResponse(true, Optional.empty(), 1) + ); context.client.poll(); context.assertElectedLeader(epoch, localId); } @@ -1338,8 +1366,12 @@ public void testRetryElection() throws Exception { context.assertVotedCandidate(epoch, localId); // Quorum size is two. If the other member rejects, then we need to schedule a revote. - int correlationId = context.assertSentVoteRequest(epoch, 0, 0L, 1); - context.deliverResponse(correlationId, otherNodeId, context.voteResponse(false, Optional.empty(), 1)); + RaftRequest.Outbound request = context.assertSentVoteRequest(epoch, 0, 0L, 1); + context.deliverResponse( + request.correlationId(), + request.destination(), + context.voteResponse(false, Optional.empty(), 1) + ); context.client.poll(); @@ -1434,11 +1466,14 @@ public void testInitializeObserverNoPreviousState() throws Exception { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(voters.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, 0, 0L, 0); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH) + ); context.client.poll(); context.assertElectedLeader(epoch, leaderId); @@ -1450,27 +1485,39 @@ public void testObserverQuorumDiscoveryFailure() throws Exception { int leaderId = 1; int epoch = 5; Set voters = Utils.mkSet(leaderId); + List bootstrapServers = voters + .stream() + .map(RaftClientTestContext::mockAddress) + .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withBootstrapServers(bootstrapServers) + .build(); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(context.bootstrapIds.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, 0, 0L, 0); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(-1, -1, MemoryRecords.EMPTY, -1, Errors.UNKNOWN_SERVER_ERROR)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(-1, -1, MemoryRecords.EMPTY, -1, Errors.UNKNOWN_SERVER_ERROR) + ); context.client.poll(); context.time.sleep(context.retryBackoffMs); context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(context.bootstrapIds.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, 0, 0L, 0); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH) + ); context.client.poll(); context.assertElectedLeader(epoch, leaderId); @@ -1483,27 +1530,169 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout() throws Exception { int otherNodeId = 2; int epoch = 5; Set voters = Utils.mkSet(leaderId, otherNodeId); + List bootstrapServers = voters + .stream() + .map(RaftClientTestContext::mockAddress) + .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withBootstrapServers(bootstrapServers) + .build(); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(context.bootstrapIds.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, 0, 0L, 0); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH)); - context.client.poll(); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH) + ); + context.client.poll(); context.assertElectedLeader(epoch, leaderId); + context.time.sleep(context.fetchTimeoutMs); context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertNotEquals(leaderId, fetchRequest.destination().id()); + assertTrue(context.bootstrapIds.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); } + @Test + public void testObserverHandleRetryFetchtToBootstrapServer() throws Exception { + // This test tries to check that KRaft is able to handle a retrying Fetch request to + // a boostrap server after a Fetch request to the leader. + + int localId = 0; + int leaderId = 1; + int otherNodeId = 2; + int epoch = 5; + Set voters = Utils.mkSet(leaderId, otherNodeId); + List bootstrapServers = voters + .stream() + .map(RaftClientTestContext::mockAddress) + .collect(Collectors.toList()); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withBootstrapServers(bootstrapServers) + .build(); + + // Expect a fetch request to one of the bootstrap servers + context.pollUntilRequest(); + RaftRequest.Outbound discoveryFetchRequest = context.assertSentFetchRequest(); + assertFalse(voters.contains(discoveryFetchRequest.destination().id())); + assertTrue(context.bootstrapIds.contains(discoveryFetchRequest.destination().id())); + context.assertFetchRequestData(discoveryFetchRequest, 0, 0L, 0); + + // Send a response with the leader and epoch + context.deliverResponse( + discoveryFetchRequest.correlationId(), + discoveryFetchRequest.destination(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH) + ); + + context.client.poll(); + context.assertElectedLeader(epoch, leaderId); + + // Expect a fetch request to the leader + context.pollUntilRequest(); + RaftRequest.Outbound toLeaderFetchRequest = context.assertSentFetchRequest(); + assertEquals(leaderId, toLeaderFetchRequest.destination().id()); + context.assertFetchRequestData(toLeaderFetchRequest, epoch, 0L, 0); + + context.time.sleep(context.requestTimeoutMs()); + + // After the fetch timeout expect a request to a bootstrap server + context.pollUntilRequest(); + RaftRequest.Outbound retryToBootstrapServerFetchRequest = context.assertSentFetchRequest(); + assertFalse(voters.contains(retryToBootstrapServerFetchRequest.destination().id())); + assertTrue(context.bootstrapIds.contains(retryToBootstrapServerFetchRequest.destination().id())); + context.assertFetchRequestData(retryToBootstrapServerFetchRequest, epoch, 0L, 0); + + // Deliver the delayed responses from the leader + Records records = context.buildBatch(0L, 3, Arrays.asList("a", "b")); + context.deliverResponse( + toLeaderFetchRequest.correlationId(), + toLeaderFetchRequest.destination(), + context.fetchResponse(epoch, leaderId, records, 0L, Errors.NONE) + ); + + context.client.poll(); + + // Deliver the same delayed responses from the bootstrap server and assume that it is the leader + records = context.buildBatch(0L, 3, Arrays.asList("a", "b")); + context.deliverResponse( + retryToBootstrapServerFetchRequest.correlationId(), + retryToBootstrapServerFetchRequest.destination(), + context.fetchResponse(epoch, leaderId, records, 0L, Errors.NONE) + ); + + // This poll should not fail when handling the duplicate response from the bootstrap server + context.client.poll(); + } + + @Test + public void testObserverHandleRetryFetchToLeader() throws Exception { + // This test tries to check that KRaft is able to handle a retrying Fetch request to + // the leader after a Fetch request to the bootstrap server. + + int localId = 0; + int leaderId = 1; + int otherNodeId = 2; + int epoch = 5; + Set voters = Utils.mkSet(leaderId, otherNodeId); + List bootstrapServers = voters + .stream() + .map(RaftClientTestContext::mockAddress) + .collect(Collectors.toList()); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withBootstrapServers(bootstrapServers) + .build(); + + // Expect a fetch request to one of the bootstrap servers + context.pollUntilRequest(); + RaftRequest.Outbound discoveryFetchRequest = context.assertSentFetchRequest(); + assertFalse(voters.contains(discoveryFetchRequest.destination().id())); + assertTrue(context.bootstrapIds.contains(discoveryFetchRequest.destination().id())); + context.assertFetchRequestData(discoveryFetchRequest, 0, 0L, 0); + + // Send a response with the leader and epoch + context.deliverResponse( + discoveryFetchRequest.correlationId(), + discoveryFetchRequest.destination(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH) + ); + + context.client.poll(); + context.assertElectedLeader(epoch, leaderId); + + // Expect a fetch request to the leader + context.pollUntilRequest(); + RaftRequest.Outbound toLeaderFetchRequest = context.assertSentFetchRequest(); + assertEquals(leaderId, toLeaderFetchRequest.destination().id()); + context.assertFetchRequestData(toLeaderFetchRequest, epoch, 0L, 0); + + context.time.sleep(context.requestTimeoutMs()); + + // After the fetch timeout expect a request to a bootstrap server + context.pollUntilRequest(); + RaftRequest.Outbound retryToBootstrapServerFetchRequest = context.assertSentFetchRequest(); + assertFalse(voters.contains(retryToBootstrapServerFetchRequest.destination().id())); + assertTrue(context.bootstrapIds.contains(retryToBootstrapServerFetchRequest.destination().id())); + context.assertFetchRequestData(retryToBootstrapServerFetchRequest, epoch, 0L, 0); + + // At this point toLeaderFetchRequest has timed out but retryToBootstrapServerFetchRequest + // is still waiting for a response. + // Confirm that no new fetch request has been sent + context.client.poll(); + assertFalse(context.channel.hasSentRequests()); + } + @Test public void testInvalidFetchRequest() throws Exception { int localId = 0; @@ -1828,7 +2017,7 @@ public void testFetchResponseIgnoredAfterBecomingCandidate() throws Exception { // Wait until we have a Fetch inflight to the leader context.pollUntilRequest(); - int fetchCorrelationId = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0L, 0); // Now await the fetch timeout and become a candidate context.time.sleep(context.fetchTimeoutMs); @@ -1837,8 +2026,11 @@ public void testFetchResponseIgnoredAfterBecomingCandidate() throws Exception { // The fetch response from the old leader returns, but it should be ignored Records records = context.buildBatch(0L, 3, Arrays.asList("a", "b")); - context.deliverResponse(fetchCorrelationId, otherNodeId, - context.fetchResponse(epoch, otherNodeId, records, 0L, Errors.NONE)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, otherNodeId, records, 0L, Errors.NONE) + ); context.client.poll(); assertEquals(0, context.log.endOffset().offset); @@ -1862,7 +2054,7 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader() thr // Wait until we have a Fetch inflight to the leader context.pollUntilRequest(); - int fetchCorrelationId = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0L, 0); // Now receive a BeginEpoch from `voter3` context.deliverRequest(context.beginEpochRequest(epoch + 1, voter3)); @@ -1872,7 +2064,11 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader() thr // The fetch response from the old leader returns, but it should be ignored Records records = context.buildBatch(0L, 3, Arrays.asList("a", "b")); FetchResponseData response = context.fetchResponse(epoch, voter2, records, 0L, Errors.NONE); - context.deliverResponse(fetchCorrelationId, voter2, response); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + response + ); context.client.poll(); assertEquals(0, context.log.endOffset().offset); @@ -1909,10 +2105,18 @@ public void testVoteResponseIgnoredAfterBecomingFollower() throws Exception { // The vote requests now return and should be ignored VoteResponseData voteResponse1 = context.voteResponse(false, Optional.empty(), epoch); - context.deliverResponse(voteRequests.get(0).correlationId, voter2, voteResponse1); + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + voteResponse1 + ); VoteResponseData voteResponse2 = context.voteResponse(false, Optional.of(voter3), epoch); - context.deliverResponse(voteRequests.get(1).correlationId, voter3, voteResponse2); + context.deliverResponse( + voteRequests.get(1).correlationId(), + voteRequests.get(1).destination(), + voteResponse2 + ); context.client.poll(); context.assertElectedLeader(epoch, voter3); @@ -1925,31 +2129,43 @@ public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError() throws E int otherNodeId = 2; int epoch = 5; Set voters = Utils.mkSet(leaderId, otherNodeId); + List bootstrapServers = voters + .stream() + .map(RaftClientTestContext::mockAddress) + .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withBootstrapServers(bootstrapServers) + .build(); context.discoverLeaderAsObserver(leaderId, epoch); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest1 = context.assertSentFetchRequest(); - assertEquals(leaderId, fetchRequest1.destinationId()); + assertEquals(leaderId, fetchRequest1.destination().id()); context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0); - context.deliverResponse(fetchRequest1.correlationId, fetchRequest1.destinationId(), - context.fetchResponse(epoch, -1, MemoryRecords.EMPTY, -1, Errors.BROKER_NOT_AVAILABLE)); + context.deliverResponse( + fetchRequest1.correlationId(), + fetchRequest1.destination(), + context.fetchResponse(epoch, -1, MemoryRecords.EMPTY, -1, Errors.BROKER_NOT_AVAILABLE) + ); context.pollUntilRequest(); // We should retry the Fetch against the other voter since the original // voter connection will be backing off. RaftRequest.Outbound fetchRequest2 = context.assertSentFetchRequest(); - assertNotEquals(leaderId, fetchRequest2.destinationId()); - assertTrue(voters.contains(fetchRequest2.destinationId())); + assertNotEquals(leaderId, fetchRequest2.destination().id()); + assertTrue(context.bootstrapIds.contains(fetchRequest2.destination().id())); context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0); - Errors error = fetchRequest2.destinationId() == leaderId ? + Errors error = fetchRequest2.destination().id() == leaderId ? Errors.NONE : Errors.NOT_LEADER_OR_FOLLOWER; - context.deliverResponse(fetchRequest2.correlationId, fetchRequest2.destinationId(), - context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, error)); + context.deliverResponse( + fetchRequest2.correlationId(), + fetchRequest2.destination(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, error) + ); context.client.poll(); context.assertElectedLeader(epoch, leaderId); @@ -1962,14 +2178,20 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout() throws Exception int otherNodeId = 2; int epoch = 5; Set voters = Utils.mkSet(leaderId, otherNodeId); + List bootstrapServers = voters + .stream() + .map(RaftClientTestContext::mockAddress) + .collect(Collectors.toList()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withBootstrapServers(bootstrapServers) + .build(); context.discoverLeaderAsObserver(leaderId, epoch); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest1 = context.assertSentFetchRequest(); - assertEquals(leaderId, fetchRequest1.destinationId()); + assertEquals(leaderId, fetchRequest1.destination().id()); context.assertFetchRequestData(fetchRequest1, epoch, 0L, 0); context.time.sleep(context.requestTimeoutMs()); @@ -1978,12 +2200,15 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout() throws Exception // We should retry the Fetch against the other voter since the original // voter connection will be backing off. RaftRequest.Outbound fetchRequest2 = context.assertSentFetchRequest(); - assertNotEquals(leaderId, fetchRequest2.destinationId()); - assertTrue(voters.contains(fetchRequest2.destinationId())); + assertNotEquals(leaderId, fetchRequest2.destination().id()); + assertTrue(context.bootstrapIds.contains(fetchRequest2.destination().id())); context.assertFetchRequestData(fetchRequest2, epoch, 0L, 0); - context.deliverResponse(fetchRequest2.correlationId, fetchRequest2.destinationId(), - context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH)); + context.deliverResponse( + fetchRequest2.correlationId(), + fetchRequest2.destination(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH) + ); context.client.poll(); context.assertElectedLeader(epoch, leaderId); @@ -2273,10 +2498,14 @@ public void testFollowerReplication() throws Exception { context.pollUntilRequest(); - int fetchQuorumCorrelationId = context.assertSentFetchRequest(epoch, 0L, 0); + RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0); Records records = context.buildBatch(0L, 3, Arrays.asList("a", "b")); FetchResponseData response = context.fetchResponse(epoch, otherNodeId, records, 0L, Errors.NONE); - context.deliverResponse(fetchQuorumCorrelationId, otherNodeId, response); + context.deliverResponse( + fetchQuorumRequest.correlationId(), + fetchQuorumRequest.destination(), + response + ); context.client.poll(); assertEquals(2L, context.log.endOffset().offset); @@ -2297,10 +2526,19 @@ public void testEmptyRecordSetInFetchResponse() throws Exception { // Receive an empty fetch response context.pollUntilRequest(); - int fetchQuorumCorrelationId = context.assertSentFetchRequest(epoch, 0L, 0); - FetchResponseData fetchResponse = context.fetchResponse(epoch, otherNodeId, - MemoryRecords.EMPTY, 0L, Errors.NONE); - context.deliverResponse(fetchQuorumCorrelationId, otherNodeId, fetchResponse); + RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0); + FetchResponseData fetchResponse = context.fetchResponse( + epoch, + otherNodeId, + MemoryRecords.EMPTY, + 0L, + Errors.NONE + ); + context.deliverResponse( + fetchQuorumRequest.correlationId(), + fetchQuorumRequest.destination(), + fetchResponse + ); context.client.poll(); assertEquals(0L, context.log.endOffset().offset); assertEquals(OptionalLong.of(0L), context.client.highWatermark()); @@ -2308,20 +2546,32 @@ public void testEmptyRecordSetInFetchResponse() throws Exception { // Receive some records in the next poll, but do not advance high watermark context.pollUntilRequest(); Records records = context.buildBatch(0L, epoch, Arrays.asList("a", "b")); - fetchQuorumCorrelationId = context.assertSentFetchRequest(epoch, 0L, 0); - fetchResponse = context.fetchResponse(epoch, otherNodeId, - records, 0L, Errors.NONE); - context.deliverResponse(fetchQuorumCorrelationId, otherNodeId, fetchResponse); + fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0); + fetchResponse = context.fetchResponse(epoch, otherNodeId, records, 0L, Errors.NONE); + context.deliverResponse( + fetchQuorumRequest.correlationId(), + fetchQuorumRequest.destination(), + fetchResponse + ); context.client.poll(); assertEquals(2L, context.log.endOffset().offset); assertEquals(OptionalLong.of(0L), context.client.highWatermark()); // The next fetch response is empty, but should still advance the high watermark context.pollUntilRequest(); - fetchQuorumCorrelationId = context.assertSentFetchRequest(epoch, 2L, epoch); - fetchResponse = context.fetchResponse(epoch, otherNodeId, - MemoryRecords.EMPTY, 2L, Errors.NONE); - context.deliverResponse(fetchQuorumCorrelationId, otherNodeId, fetchResponse); + fetchQuorumRequest = context.assertSentFetchRequest(epoch, 2L, epoch); + fetchResponse = context.fetchResponse( + epoch, + otherNodeId, + MemoryRecords.EMPTY, + 2L, + Errors.NONE + ); + context.deliverResponse( + fetchQuorumRequest.correlationId(), + fetchQuorumRequest.destination(), + fetchResponse + ); context.client.poll(); assertEquals(2L, context.log.endOffset().offset); assertEquals(OptionalLong.of(2L), context.client.highWatermark()); @@ -2454,11 +2704,11 @@ public void testFollowerLogReconciliation() throws Exception { context.pollUntilRequest(); - int correlationId = context.assertSentFetchRequest(epoch, 3L, lastEpoch); + RaftRequest.Outbound request = context.assertSentFetchRequest(epoch, 3L, lastEpoch); FetchResponseData response = context.divergingFetchResponse(epoch, otherNodeId, 2L, lastEpoch, 1L); - context.deliverResponse(correlationId, otherNodeId, response); + context.deliverResponse(request.correlationId(), request.destination(), response); // Poll again to complete truncation context.client.poll(); @@ -2530,10 +2780,14 @@ public void testClusterAuthorizationFailedInFetch() throws Exception { context.pollUntilRequest(); - int correlationId = context.assertSentFetchRequest(epoch, 0, 0); + RaftRequest.Outbound request = context.assertSentFetchRequest(epoch, 0, 0); FetchResponseData response = new FetchResponseData() .setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code()); - context.deliverResponse(correlationId, otherNodeId, response); + context.deliverResponse( + request.correlationId(), + request.destination(), + response + ); assertThrows(ClusterAuthorizationException.class, context.client::poll); } @@ -2553,11 +2807,11 @@ public void testClusterAuthorizationFailedInBeginQuorumEpoch() throws Exception context.expectAndGrantVotes(epoch); context.pollUntilRequest(); - int correlationId = context.assertSentBeginQuorumEpochRequest(epoch, 1); + RaftRequest.Outbound request = context.assertSentBeginQuorumEpochRequest(epoch, 1); BeginQuorumEpochResponseData response = new BeginQuorumEpochResponseData() .setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code()); - context.deliverResponse(correlationId, otherNodeId, response); + context.deliverResponse(request.correlationId(), request.destination(), response); assertThrows(ClusterAuthorizationException.class, context.client::poll); } @@ -2577,11 +2831,11 @@ public void testClusterAuthorizationFailedInVote() throws Exception { context.pollUntilRequest(); context.assertVotedCandidate(epoch, localId); - int correlationId = context.assertSentVoteRequest(epoch, 0, 0L, 1); + RaftRequest.Outbound request = context.assertSentVoteRequest(epoch, 0, 0L, 1); VoteResponseData response = new VoteResponseData() .setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code()); - context.deliverResponse(correlationId, otherNodeId, response); + context.deliverResponse(request.correlationId(), request.destination(), response); assertThrows(ClusterAuthorizationException.class, context.client::poll); } @@ -2597,11 +2851,11 @@ public void testClusterAuthorizationFailedInEndQuorumEpoch() throws Exception { context.client.shutdown(5000); context.pollUntilRequest(); - int correlationId = context.assertSentEndQuorumEpochRequest(epoch, otherNodeId); + RaftRequest.Outbound request = context.assertSentEndQuorumEpochRequest(epoch, otherNodeId); EndQuorumEpochResponseData response = new EndQuorumEpochResponseData() .setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code()); - context.deliverResponse(correlationId, otherNodeId, response); + context.deliverResponse(request.correlationId(), request.destination(), response); assertThrows(ClusterAuthorizationException.class, context.client::poll); } @@ -2810,14 +3064,17 @@ public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances() th // Poll for our first fetch request context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(voters.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); // The response does not advance the high watermark List records1 = Arrays.asList("a", "b", "c"); MemoryRecords batch1 = context.buildBatch(0L, 3, records1); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(epoch, otherNodeId, batch1, 0L, Errors.NONE)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, otherNodeId, batch1, 0L, Errors.NONE) + ); context.client.poll(); // The listener should not have seen any data @@ -2828,14 +3085,17 @@ public void testHandleCommitCallbackFiresAfterFollowerHighWatermarkAdvances() th // Now look for the next fetch request context.pollUntilRequest(); fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + assertTrue(voters.contains(fetchRequest.destination().id())); context.assertFetchRequestData(fetchRequest, epoch, 3L, 3); // The high watermark advances to include the first batch we fetched List records2 = Arrays.asList("d", "e", "f"); MemoryRecords batch2 = context.buildBatch(3L, 3, records2); - context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - context.fetchResponse(epoch, otherNodeId, batch2, 3L, Errors.NONE)); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, otherNodeId, batch2, 3L, Errors.NONE) + ); context.client.poll(); // The listener should have seen only the data from the first batch @@ -3012,21 +3272,30 @@ public void testObserverFetchWithNoLocalId() throws Exception { // This is designed for tooling/debugging use cases. Set voters = Utils.mkSet(1, 2); + List bootstrapServers = voters + .stream() + .map(RaftClientTestContext::mockAddress) + .collect(Collectors.toList()); + RaftClientTestContext context = new RaftClientTestContext.Builder(OptionalInt.empty(), voters) + .withBootstrapServers(bootstrapServers) .build(); // First fetch discovers the current leader and epoch context.pollUntilRequest(); RaftRequest.Outbound fetchRequest1 = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest1.destinationId())); + assertTrue(context.bootstrapIds.contains(fetchRequest1.destination().id())); context.assertFetchRequestData(fetchRequest1, 0, 0L, 0); int leaderEpoch = 5; int leaderId = 1; - context.deliverResponse(fetchRequest1.correlationId, fetchRequest1.destinationId(), - context.fetchResponse(5, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH)); + context.deliverResponse( + fetchRequest1.correlationId(), + fetchRequest1.destination(), + context.fetchResponse(5, leaderId, MemoryRecords.EMPTY, 0L, Errors.FENCED_LEADER_EPOCH) + ); context.client.poll(); context.assertElectedLeader(leaderEpoch, leaderId); @@ -3034,13 +3303,16 @@ public void testObserverFetchWithNoLocalId() throws Exception { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest2 = context.assertSentFetchRequest(); - assertEquals(leaderId, fetchRequest2.destinationId()); + assertEquals(leaderId, fetchRequest2.destination().id()); context.assertFetchRequestData(fetchRequest2, leaderEpoch, 0L, 0); List records = Arrays.asList("a", "b", "c"); MemoryRecords batch1 = context.buildBatch(0L, 3, records); - context.deliverResponse(fetchRequest2.correlationId, fetchRequest2.destinationId(), - context.fetchResponse(leaderEpoch, leaderId, batch1, 0L, Errors.NONE)); + context.deliverResponse( + fetchRequest2.correlationId(), + fetchRequest2.destination(), + context.fetchResponse(leaderEpoch, leaderId, batch1, 0L, Errors.NONE) + ); context.client.poll(); assertEquals(3L, context.log.endOffset().offset); assertEquals(3, context.log.lastFetchedEpoch()); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java b/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java index 379290240e..f9c3efee02 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java @@ -16,31 +16,29 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; -import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; public class MockNetworkChannel implements NetworkChannel { private final AtomicInteger correlationIdCounter; - private final Set nodeCache; private final List sendQueue = new ArrayList<>(); private final Map awaitingResponse = new HashMap<>(); + private final ListenerName listenerName = ListenerName.normalised("CONTROLLER"); - public MockNetworkChannel(AtomicInteger correlationIdCounter, Set destinationIds) { + public MockNetworkChannel(AtomicInteger correlationIdCounter) { this.correlationIdCounter = correlationIdCounter; - this.nodeCache = destinationIds; } - public MockNetworkChannel(Set destinationIds) { - this(new AtomicInteger(0), destinationIds); + public MockNetworkChannel() { + this(new AtomicInteger(0)); } @Override @@ -50,16 +48,12 @@ public int newCorrelationId() { @Override public void send(RaftRequest.Outbound request) { - if (!nodeCache.contains(request.destinationId())) { - throw new IllegalArgumentException("Attempted to send to destination " + - request.destinationId() + ", but its address is not yet known"); - } sendQueue.add(request); } @Override - public void updateEndpoint(int id, InetSocketAddress address) { - // empty + public ListenerName listenerName() { + return listenerName; } public List drainSendQueue() { @@ -72,7 +66,7 @@ public List drainSentRequests(Optional apiKeyFilt while (iterator.hasNext()) { RaftRequest.Outbound request = iterator.next(); if (!apiKeyFilter.isPresent() || request.data().apiKey() == apiKeyFilter.get().id) { - awaitingResponse.put(request.correlationId, request); + awaitingResponse.put(request.correlationId(), request); requests.add(request); iterator.remove(); } @@ -80,17 +74,15 @@ public List drainSentRequests(Optional apiKeyFilt return requests; } - public boolean hasSentRequests() { return !sendQueue.isEmpty(); } public void mockReceive(RaftResponse.Inbound response) { - RaftRequest.Outbound request = awaitingResponse.get(response.correlationId); + RaftRequest.Outbound request = awaitingResponse.get(response.correlationId()); if (request == null) { throw new IllegalStateException("Received response for a request which is not being awaited"); } request.completion.complete(response); } - } diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 08acba1047..94567dce66 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -16,12 +16,14 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.ReplicaKey; +import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.raft.internals.VoterSetTest; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -29,10 +31,13 @@ import java.io.UncheckedIOException; import java.util.Collections; +import java.util.Map; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; +import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -54,19 +59,16 @@ public class QuorumStateTest { private final MockableRandom random = new MockableRandom(1L); private final BatchAccumulator accumulator = Mockito.mock(BatchAccumulator.class); - private QuorumState buildQuorumState(Set voters, short kraftVersion) { - return buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); - } - private QuorumState buildQuorumState( OptionalInt localId, - Set voters, + VoterSet voterSet, short kraftVersion ) { return new QuorumState( localId, localDirectoryId, - () -> VoterSetTest.voterSet(VoterSetTest.voterMap(voters, false)), + VoterSetTest.DEFAULT_LISTENER_NAME, + () -> voterSet, () -> kraftVersion, electionTimeoutMs, fetchTimeoutMs, @@ -77,10 +79,47 @@ private QuorumState buildQuorumState( ); } + private QuorumState initializeEmptyState(VoterSet voters, short kraftVersion) { + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + store.writeElectionState(ElectionState.withUnknownLeader(0, voters.voterIds()), kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + return state; + } + + private Set persistedVoters(Set voters, short kraftVersion) { + if (kraftVersion == 1) { + return Collections.emptySet(); + } + + return voters; + } + + private ReplicaKey persistedVotedKey(ReplicaKey replicaKey, short kraftVersion) { + if (kraftVersion == 1) { + return replicaKey; + } + + return ReplicaKey.of(replicaKey.id(), Optional.empty()); + } + + private VoterSet localStandaloneVoterSet() { + return VoterSetTest.voterSet( + Collections.singletonMap(localId, VoterSetTest.voterNode(localVoterKey)) + ); + } + + private VoterSet localWithRemoteVoterSet(IntStream remoteIds, short kraftVersion) { + boolean withDirectoryId = kraftVersion > 0; + Map voters = VoterSetTest.voterMap(remoteIds, withDirectoryId); + voters.put(localId, VoterSetTest.voterNode(localVoterKey)); + + return VoterSetTest.voterSet(voters); + } + @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testInitializePrimordialEpoch(short kraftVersion) { - Set voters = Utils.mkSet(localId); + VoterSet voters = localStandaloneVoterSet(); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -98,13 +137,13 @@ public void testInitializeAsUnattached(short kraftVersion) { int node1 = 1; int node2 = 2; int epoch = 5; - Set voters = Utils.mkSet(localId, node1, node2); - store.writeElectionState(ElectionState.withUnknownLeader(epoch, voters), kraftVersion); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + store.writeElectionState(ElectionState.withUnknownLeader(epoch, voters.voterIds()), kraftVersion); int jitterMs = 2500; random.mockNextInt(jitterMs); - QuorumState state = buildQuorumState(voters, kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); assertTrue(state.isUnattached()); @@ -120,47 +159,44 @@ public void testInitializeAsFollower(short kraftVersion) { int node1 = 1; int node2 = 2; int epoch = 5; - Set voters = Utils.mkSet(localId, node1, node2); - store.writeElectionState(ElectionState.withElectedLeader(epoch, node1, voters), kraftVersion); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + store.writeElectionState(ElectionState.withElectedLeader(epoch, node1, voters.voterIds()), kraftVersion); - QuorumState state = buildQuorumState(voters, kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isFollower()); assertEquals(epoch, state.epoch()); FollowerState followerState = state.followerStateOrThrow(); assertEquals(epoch, followerState.epoch()); - assertEquals(node1, followerState.leaderId()); + assertEquals(node1, followerState.leader().id()); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testInitializeAsVoted(short kraftVersion) { - int node1 = 1; - Optional node1DirectoryId = Optional.of(Uuid.randomUuid()); - int node2 = 2; + ReplicaKey nodeKey1 = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Optional.of(Uuid.randomUuid())); + int epoch = 5; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2)); store.writeElectionState( - ElectionState.withVotedCandidate(epoch, ReplicaKey.of(node1, node1DirectoryId), voters), + ElectionState.withVotedCandidate(epoch, nodeKey1, voters.voterIds()), kraftVersion ); int jitterMs = 2500; random.mockNextInt(jitterMs); - QuorumState state = buildQuorumState(voters, kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isVoted()); assertEquals(epoch, state.epoch()); VotedState votedState = state.votedStateOrThrow(); assertEquals(epoch, votedState.epoch()); - assertEquals( - ReplicaKey.of(node1, persistedDirectoryId(node1DirectoryId, kraftVersion)), - votedState.votedKey() - ); + assertEquals(persistedVotedKey(nodeKey1, kraftVersion), votedState.votedKey()); assertEquals( electionTimeoutMs + jitterMs, @@ -174,18 +210,18 @@ public void testInitializeAsResignedCandidate(short kraftVersion) { int node1 = 1; int node2 = 2; int epoch = 5; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); ElectionState election = ElectionState.withVotedCandidate( epoch, localVoterKey, - voters + voters.voterIds() ); store.writeElectionState(election, kraftVersion); int jitterMs = 2500; random.mockNextInt(jitterMs); - QuorumState state = buildQuorumState(voters, kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isCandidate()); assertEquals(epoch, state.epoch()); @@ -193,7 +229,7 @@ public void testInitializeAsResignedCandidate(short kraftVersion) { CandidateState candidateState = state.candidateStateOrThrow(); assertEquals(epoch, candidateState.epoch()); assertEquals( - ElectionState.withVotedCandidate(epoch, localVoterKey, voters), + ElectionState.withVotedCandidate(epoch, localVoterKey, voters.voterIds()), candidateState.election() ); assertEquals(Utils.mkSet(node1, node2), candidateState.unrecordedVoters()); @@ -211,8 +247,8 @@ public void testInitializeAsResignedLeader(short kraftVersion) { int node1 = 1; int node2 = 2; int epoch = 5; - Set voters = Utils.mkSet(localId, node1, node2); - ElectionState election = ElectionState.withElectedLeader(epoch, localId, voters); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + ElectionState election = ElectionState.withElectedLeader(epoch, localId, voters.voterIds()); store.writeElectionState(election, kraftVersion); // If we were previously a leader, we will start as resigned in order to ensure @@ -223,7 +259,7 @@ public void testInitializeAsResignedLeader(short kraftVersion) { int jitterMs = 2500; random.mockNextInt(jitterMs); - QuorumState state = buildQuorumState(voters, kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertFalse(state.isLeader()); assertEquals(epoch, state.epoch()); @@ -241,7 +277,7 @@ public void testInitializeAsResignedLeader(short kraftVersion) { public void testCandidateToCandidate(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -285,7 +321,7 @@ public void testCandidateToCandidate(short kraftVersion) { public void testCandidateToResigned(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -301,7 +337,7 @@ public void testCandidateToResigned(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testCandidateToLeader(short kraftVersion) { - Set voters = Utils.mkSet(localId); + VoterSet voters = localStandaloneVoterSet(); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -320,7 +356,7 @@ public void testCandidateToLeader(short kraftVersion) { @ValueSource(shorts = {0, 1}) public void testCandidateToLeaderWithoutGrantedVote(short kraftVersion) { int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); @@ -336,16 +372,23 @@ public void testCandidateToLeaderWithoutGrantedVote(short kraftVersion) { @ValueSource(shorts = {0, 1}) public void testCandidateToFollower(short kraftVersion) { int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); - state.transitionToFollower(5, otherNodeId); + state.transitionToFollower(5, voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get()); assertEquals(5, state.epoch()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( - Optional.of(ElectionState.withElectedLeader(5, otherNodeId, persistedVoters(voters, kraftVersion))), + Optional.of( + ElectionState.withElectedLeader( + 5, + otherNodeId, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -354,7 +397,7 @@ public void testCandidateToFollower(short kraftVersion) { @ValueSource(shorts = {0, 1}) public void testCandidateToUnattached(short kraftVersion) { int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); @@ -363,7 +406,12 @@ public void testCandidateToUnattached(short kraftVersion) { assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); assertEquals( - Optional.of(ElectionState.withUnknownLeader(5, persistedVoters(voters, kraftVersion))), + Optional.of( + ElectionState.withUnknownLeader( + 5, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -371,10 +419,8 @@ public void testCandidateToUnattached(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testCandidateToVoted(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); @@ -390,11 +436,8 @@ public void testCandidateToVoted(short kraftVersion) { Optional.of( ElectionState.withVotedCandidate( 5, - ReplicaKey.of( - otherNodeId, - persistedDirectoryId(otherNodeDirectoryId, kraftVersion) - ), - persistedVoters(voters, kraftVersion)) + persistedVotedKey(otherNodeKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion)) ), store.readElectionState() ); @@ -403,27 +446,28 @@ public void testCandidateToVoted(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testCandidateToAnyStateLowerEpoch(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); state.transitionToCandidate(); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(4, otherNodeKey)); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(4, otherNodeId)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ) + ); assertEquals(6, state.epoch()); assertEquals( Optional.of( ElectionState.withVotedCandidate( 6, - ReplicaKey.of( - localId, - persistedDirectoryId(Optional.of(localDirectoryId), kraftVersion) - ), - persistedVoters(voters, kraftVersion) + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) ) ), store.readElectionState() @@ -433,7 +477,7 @@ public void testCandidateToAnyStateLowerEpoch(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testLeaderToLeader(short kraftVersion) { - Set voters = Utils.mkSet(localId); + VoterSet voters = localStandaloneVoterSet(); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -451,7 +495,7 @@ public void testLeaderToLeader(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testLeaderToResigned(short kraftVersion) { - Set voters = Utils.mkSet(localId); + VoterSet voters = localStandaloneVoterSet(); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -464,8 +508,10 @@ public void testLeaderToResigned(short kraftVersion) { state.transitionToResigned(Collections.singletonList(localId)); assertTrue(state.isResigned()); ResignedState resignedState = state.resignedStateOrThrow(); - assertEquals(ElectionState.withElectedLeader(1, localId, voters), - resignedState.election()); + assertEquals( + ElectionState.withElectedLeader(1, localId, voters.voterIds()), + resignedState.election() + ); assertEquals(1, resignedState.epoch()); assertEquals(Collections.emptySet(), resignedState.unackedVoters()); } @@ -473,7 +519,7 @@ public void testLeaderToResigned(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testLeaderToCandidate(short kraftVersion) { - Set voters = Utils.mkSet(localId); + VoterSet voters = localStandaloneVoterSet(); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -492,19 +538,25 @@ public void testLeaderToCandidate(short kraftVersion) { @ValueSource(shorts = {0, 1}) public void testLeaderToFollower(short kraftVersion) { int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.transitionToLeader(0L, accumulator); - state.transitionToFollower(5, otherNodeId); + state.transitionToFollower(5, voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get()); assertEquals(5, state.epoch()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( - Optional.of(ElectionState.withElectedLeader(5, otherNodeId, persistedVoters(voters, kraftVersion))), + Optional.of( + ElectionState.withElectedLeader( + 5, + otherNodeId, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -513,7 +565,7 @@ public void testLeaderToFollower(short kraftVersion) { @ValueSource(shorts = {0, 1}) public void testLeaderToUnattached(short kraftVersion) { int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); @@ -523,7 +575,12 @@ public void testLeaderToUnattached(short kraftVersion) { assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); assertEquals( - Optional.of(ElectionState.withUnknownLeader(5, persistedVoters(voters, kraftVersion))), + Optional.of( + ElectionState.withUnknownLeader( + 5, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -531,14 +588,12 @@ public void testLeaderToUnattached(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testLeaderToVoted(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); state.transitionToLeader(0L, accumulator); state.transitionToVoted(5, otherNodeKey); @@ -552,11 +607,8 @@ public void testLeaderToVoted(short kraftVersion) { Optional.of( ElectionState.withVotedCandidate( 5, - ReplicaKey.of( - otherNodeId, - persistedDirectoryId(otherNodeDirectoryId, kraftVersion) - ), - persistedVoters(voters, kraftVersion) + persistedVotedKey(otherNodeKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) ) ), store.readElectionState() @@ -566,22 +618,32 @@ public void testLeaderToVoted(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testLeaderToAnyStateLowerEpoch(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); state.transitionToLeader(0L, accumulator); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(4, otherNodeKey)); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(4, otherNodeId)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ) + ); assertEquals(6, state.epoch()); assertEquals( - Optional.of(ElectionState.withElectedLeader(6, localId, persistedVoters(voters, kraftVersion))), + Optional.of( + ElectionState.withElectedLeader( + 6, + localId, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -589,26 +651,28 @@ public void testLeaderToAnyStateLowerEpoch(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testCannotFollowOrVoteForSelf(short kraftVersion) { - Set voters = Utils.mkSet(localId); + VoterSet voters = localStandaloneVoterSet(); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(0, localId)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 0, + voters.voterNode(localId, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ) + ); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(0, localVoterKey)); } @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToLeaderOrResigned(short kraftVersion) { - int leaderId = 1; + ReplicaKey leaderKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); int epoch = 5; - Set voters = Utils.mkSet(localId, leaderId); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leaderKey)); store.writeElectionState( - ElectionState.withVotedCandidate( - epoch, - ReplicaKey.of(leaderId, Optional.empty()), - voters - ), + ElectionState.withVotedCandidate(epoch, leaderKey, voters.voterIds()), kraftVersion ); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -621,10 +685,8 @@ public void testUnattachedToLeaderOrResigned(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToVotedSameEpoch(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); @@ -641,11 +703,8 @@ public void testUnattachedToVotedSameEpoch(short kraftVersion) { Optional.of( ElectionState.withVotedCandidate( 5, - ReplicaKey.of( - otherNodeId, - persistedDirectoryId(otherNodeDirectoryId, kraftVersion) - ), - persistedVoters(voters, kraftVersion) + persistedVotedKey(otherNodeKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) ) ), store.readElectionState() @@ -659,10 +718,8 @@ public void testUnattachedToVotedSameEpoch(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToVotedHigherEpoch(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); @@ -676,11 +733,8 @@ public void testUnattachedToVotedHigherEpoch(short kraftVersion) { Optional.of( ElectionState.withVotedCandidate( 8, - ReplicaKey.of( - otherNodeId, - persistedDirectoryId(otherNodeDirectoryId, kraftVersion) - ), - persistedVoters(voters, kraftVersion) + persistedVotedKey(otherNodeKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) ) ), store.readElectionState() @@ -690,8 +744,8 @@ public void testUnattachedToVotedHigherEpoch(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToCandidate(short kraftVersion) { - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); @@ -710,8 +764,8 @@ public void testUnattachedToCandidate(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToUnattached(short kraftVersion) { - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); @@ -731,52 +785,74 @@ public void testUnattachedToUnattached(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToFollowerSameEpoch(short kraftVersion) { - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); - state.transitionToFollower(5, otherNodeId); + state.transitionToFollower( + 5, + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); assertTrue(state.isFollower()); FollowerState followerState = state.followerStateOrThrow(); assertEquals(5, followerState.epoch()); - assertEquals(otherNodeId, followerState.leaderId()); + assertEquals( + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME), + Optional.of(followerState.leader()) + ); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToFollowerHigherEpoch(short kraftVersion) { - int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); - state.transitionToFollower(8, otherNodeId); + state.transitionToFollower( + 8, + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); assertTrue(state.isFollower()); FollowerState followerState = state.followerStateOrThrow(); assertEquals(8, followerState.epoch()); - assertEquals(otherNodeId, followerState.leaderId()); + assertEquals( + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME), + Optional.of(followerState.leader()) + ); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testUnattachedToAnyStateLowerEpoch(short kraftVersion) { - int otherNodeId = 1; - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, Optional.empty()); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(4, otherNodeKey)); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(4, otherNodeId)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ) + ); assertEquals(5, state.epoch()); assertEquals( - Optional.of(ElectionState.withUnknownLeader(5, persistedVoters(voters, kraftVersion))), + Optional.of( + ElectionState.withUnknownLeader( + 5, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -786,7 +862,7 @@ public void testUnattachedToAnyStateLowerEpoch(short kraftVersion) { public void testVotedToInvalidLeaderOrResigned(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); @@ -799,7 +875,7 @@ public void testVotedToInvalidLeaderOrResigned(short kraftVersion) { public void testVotedToCandidate(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); @@ -819,7 +895,7 @@ public void testVotedToCandidate(short kraftVersion) { public void testVotedToVotedSameEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); @@ -839,17 +915,29 @@ public void testVotedToVotedSameEpoch(short kraftVersion) { public void testVotedToFollowerSameEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); - state.transitionToFollower(5, node2); + state.transitionToFollower( + 5, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); FollowerState followerState = state.followerStateOrThrow(); assertEquals(5, followerState.epoch()); - assertEquals(node2, followerState.leaderId()); assertEquals( - Optional.of(ElectionState.withElectedLeader(5, node2, persistedVoters(voters, kraftVersion))), + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME), + Optional.of(followerState.leader()) + ); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 5, + node2, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -859,17 +947,29 @@ public void testVotedToFollowerSameEpoch(short kraftVersion) { public void testVotedToFollowerHigherEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); - state.transitionToFollower(8, node2); + state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); FollowerState followerState = state.followerStateOrThrow(); assertEquals(8, followerState.epoch()); - assertEquals(node2, followerState.leaderId()); assertEquals( - Optional.of(ElectionState.withElectedLeader(8, node2, persistedVoters(voters, kraftVersion))), + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get(), + followerState.leader() + ); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 8, + node2, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -879,7 +979,7 @@ public void testVotedToFollowerHigherEpoch(short kraftVersion) { public void testVotedToUnattachedSameEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty())); @@ -890,7 +990,7 @@ public void testVotedToUnattachedSameEpoch(short kraftVersion) { @ValueSource(shorts = {0, 1}) public void testVotedToUnattachedHigherEpoch(short kraftVersion) { int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToVoted(5, ReplicaKey.of(otherNodeId, Optional.empty())); @@ -910,26 +1010,27 @@ public void testVotedToUnattachedHigherEpoch(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testVotedToAnyStateLowerEpoch(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToVoted(5, otherNodeKey); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(4, otherNodeKey)); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(4, otherNodeId)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ) + ); assertEquals(5, state.epoch()); assertEquals( Optional.of( ElectionState.withVotedCandidate( 5, - ReplicaKey.of( - otherNodeId, - persistedDirectoryId(otherNodeDirectoryId, kraftVersion) - ), - persistedVoters(voters, kraftVersion) + persistedVotedKey(otherNodeKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) ) ), store.readElectionState() @@ -941,18 +1042,42 @@ public void testVotedToAnyStateLowerEpoch(short kraftVersion) { public void testFollowerToFollowerSameEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(8, node2); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(8, node1)); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(8, node2)); + state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 8, + voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ) + ); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ) + ); FollowerState followerState = state.followerStateOrThrow(); assertEquals(8, followerState.epoch()); - assertEquals(node2, followerState.leaderId()); assertEquals( - Optional.of(ElectionState.withElectedLeader(8, node2, persistedVoters(voters, kraftVersion))), + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME), + Optional.of(followerState.leader()) + ); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 8, + node2, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -962,17 +1087,32 @@ public void testFollowerToFollowerSameEpoch(short kraftVersion) { public void testFollowerToFollowerHigherEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(8, node2); - state.transitionToFollower(9, node1); + state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); + state.transitionToFollower( + 9, + voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); FollowerState followerState = state.followerStateOrThrow(); assertEquals(9, followerState.epoch()); - assertEquals(node1, followerState.leaderId()); assertEquals( - Optional.of(ElectionState.withElectedLeader(9, node1, persistedVoters(voters, kraftVersion))), + voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME), + Optional.of(followerState.leader()) + ); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 9, + node1, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -982,10 +1122,13 @@ public void testFollowerToFollowerHigherEpoch(short kraftVersion) { public void testFollowerToLeaderOrResigned(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(8, node2); + state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @@ -995,10 +1138,13 @@ public void testFollowerToLeaderOrResigned(short kraftVersion) { public void testFollowerToCandidate(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(8, node2); + state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); @@ -1015,10 +1161,13 @@ public void testFollowerToCandidate(short kraftVersion) { public void testFollowerToUnattachedSameEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(8, node2); + state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8)); } @@ -1027,10 +1176,13 @@ public void testFollowerToUnattachedSameEpoch(short kraftVersion) { public void testFollowerToUnattachedHigherEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(8, node2); + state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); @@ -1047,10 +1199,13 @@ public void testFollowerToUnattachedHigherEpoch(short kraftVersion) { public void testFollowerToVotedSameEpoch(short kraftVersion) { int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(8, node2); + state.transitionToFollower( + 8, + voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); assertThrows( IllegalStateException.class, @@ -1069,24 +1224,26 @@ public void testFollowerToVotedSameEpoch(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testFollowerToVotedHigherEpoch(short kraftVersion) { - int node1 = 1; - Optional node1DirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey node1Key = ReplicaKey.of(node1, node1DirectoryId); - int node2 = 2; - Set voters = Utils.mkSet(localId, node1, node2); + ReplicaKey nodeKey1 = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + ReplicaKey nodeKey2 = ReplicaKey.of(2, Optional.of(Uuid.randomUuid())); + + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(8, node2); + state.transitionToFollower( + 8, + voters.voterNode(nodeKey2.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToVoted(9, node1Key); + state.transitionToVoted(9, nodeKey1); assertTrue(state.isVoted()); VotedState votedState = state.votedStateOrThrow(); assertEquals(9, votedState.epoch()); - assertEquals(node1Key, votedState.votedKey()); + assertEquals(nodeKey1, votedState.votedKey()); assertEquals(electionTimeoutMs + jitterMs, votedState.remainingElectionTimeMs(time.milliseconds())); @@ -1096,19 +1253,34 @@ public void testFollowerToVotedHigherEpoch(short kraftVersion) { @ValueSource(shorts = {0, 1}) public void testFollowerToAnyStateLowerEpoch(short kraftVersion) { int otherNodeId = 1; - Set voters = Utils.mkSet(localId, otherNodeId); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower(5, otherNodeId); + state.transitionToFollower( + 5, + voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows( IllegalStateException.class, () -> state.transitionToVoted(4, ReplicaKey.of(otherNodeId, Optional.empty())) ); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(4, otherNodeId)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ) + ); assertEquals(5, state.epoch()); assertEquals( - Optional.of(ElectionState.withElectedLeader(5, otherNodeId, persistedVoters(voters, kraftVersion))), + Optional.of( + ElectionState.withElectedLeader( + 5, + otherNodeId, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), store.readElectionState() ); } @@ -1117,10 +1289,8 @@ public void testFollowerToAnyStateLowerEpoch(short kraftVersion) { @ValueSource(shorts = {0, 1}) public void testCanBecomeFollowerOfNonVoter(short kraftVersion) { int otherNodeId = 1; - int nonVoterId = 2; - Optional nonVoterDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey nonVoterKey = ReplicaKey.of(nonVoterId, nonVoterDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey nonVoterKey = ReplicaKey.of(2, Optional.of(Uuid.randomUuid())); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -1133,15 +1303,22 @@ public void testCanBecomeFollowerOfNonVoter(short kraftVersion) { assertEquals(nonVoterKey, votedState.votedKey()); // Transition to follower - state.transitionToFollower(4, nonVoterId); - assertEquals(new LeaderAndEpoch(OptionalInt.of(nonVoterId), 4), state.leaderAndEpoch()); + Node nonVoterNode = new Node(nonVoterKey.id(), "non-voter-host", 1234); + state.transitionToFollower(4, nonVoterNode); + assertEquals( + new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4), + state.leaderAndEpoch() + ); } @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testObserverCannotBecomeCandidateOrLeader(short kraftVersion) { + boolean withDirectoryId = kraftVersion > 0; int otherNodeId = 1; - Set voters = Utils.mkSet(otherNodeId); + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(otherNodeId), withDirectoryId) + ); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isObserver()); @@ -1152,10 +1329,8 @@ public void testObserverCannotBecomeCandidateOrLeader(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testObserverWithIdCanVote(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -1172,14 +1347,20 @@ public void testObserverWithIdCanVote(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testObserverFollowerToUnattached(short kraftVersion) { + boolean withDirectoryId = kraftVersion > 0; int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(node1, node2); + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(node1, node2), withDirectoryId) + ); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isObserver()); - state.transitionToFollower(2, node1); + state.transitionToFollower( + 2, + voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); state.transitionToUnattached(3); assertTrue(state.isUnattached()); UnattachedState unattachedState = state.unattachedStateOrThrow(); @@ -1192,19 +1373,25 @@ public void testObserverFollowerToUnattached(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testObserverUnattachedToFollower(short kraftVersion) { + boolean withDirectoryId = kraftVersion > 0; int node1 = 1; int node2 = 2; - Set voters = Utils.mkSet(node1, node2); + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(node1, node2), withDirectoryId) + ); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isObserver()); state.transitionToUnattached(2); - state.transitionToFollower(3, node1); + state.transitionToFollower(3, voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); assertTrue(state.isFollower()); FollowerState followerState = state.followerStateOrThrow(); assertEquals(3, followerState.epoch()); - assertEquals(node1, followerState.leaderId()); + assertEquals( + voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME), + Optional.of(followerState.leader()) + ); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @@ -1214,7 +1401,11 @@ public void testInitializeWithCorruptedStore(short kraftVersion) { QuorumStateStore stateStore = Mockito.mock(QuorumStateStore.class); Mockito.doThrow(UncheckedIOException.class).when(stateStore).readElectionState(); - QuorumState state = buildQuorumState(Utils.mkSet(localId), kraftVersion); + QuorumState state = buildQuorumState( + OptionalInt.of(localId), + localStandaloneVoterSet(), + kraftVersion + ); int epoch = 2; state.initialize(new OffsetAndEpoch(0L, epoch)); @@ -1226,10 +1417,8 @@ public void testInitializeWithCorruptedStore(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testHasRemoteLeader(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); assertFalse(state.hasRemoteLeader()); @@ -1237,7 +1426,7 @@ public void testHasRemoteLeader(short kraftVersion) { state.transitionToCandidate(); assertFalse(state.hasRemoteLeader()); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); state.transitionToLeader(0L, accumulator); assertFalse(state.hasRemoteLeader()); @@ -1247,20 +1436,24 @@ public void testHasRemoteLeader(short kraftVersion) { state.transitionToVoted(state.epoch() + 1, otherNodeKey); assertFalse(state.hasRemoteLeader()); - state.transitionToFollower(state.epoch() + 1, otherNodeId); + state.transitionToFollower( + state.epoch() + 1, + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); assertTrue(state.hasRemoteLeader()); } @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testHighWatermarkRetained(short kraftVersion) { - int otherNodeId = 1; - Optional otherNodeDirectoryId = Optional.of(Uuid.randomUuid()); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeId, otherNodeDirectoryId); - Set voters = Utils.mkSet(localId, otherNodeId); + ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.transitionToFollower(5, otherNodeId); + state.transitionToFollower( + 5, + voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() + ); FollowerState followerState = state.followerStateOrThrow(); followerState.updateHighWatermark(OptionalLong.of(10L)); @@ -1278,7 +1471,7 @@ public void testHighWatermarkRetained(short kraftVersion) { assertEquals(highWatermark, state.highWatermark()); CandidateState candidateState = state.candidateStateOrThrow(); - candidateState.recordGrantedVote(otherNodeId); + candidateState.recordGrantedVote(otherNodeKey.id()); assertTrue(candidateState.isVoteGranted()); state.transitionToLeader(10L, accumulator); @@ -1288,7 +1481,11 @@ public void testHighWatermarkRetained(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testInitializeWithEmptyLocalId(short kraftVersion) { - QuorumState state = buildQuorumState(OptionalInt.empty(), Utils.mkSet(0, 1), kraftVersion); + boolean withDirectoryId = kraftVersion > 0; + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(0, 1), withDirectoryId) + ); + QuorumState state = buildQuorumState(OptionalInt.empty(), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); assertTrue(state.isObserver()); @@ -1301,7 +1498,7 @@ public void testInitializeWithEmptyLocalId(short kraftVersion) { ); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); - state.transitionToFollower(1, 1); + state.transitionToFollower(1, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); assertTrue(state.isFollower()); state.transitionToUnattached(2); @@ -1311,15 +1508,18 @@ public void testInitializeWithEmptyLocalId(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void testNoLocalIdInitializationFailsIfElectionStateHasVotedCandidate(short kraftVersion) { + boolean withDirectoryId = kraftVersion > 0; int epoch = 5; int votedId = 1; - Set voters = Utils.mkSet(0, votedId); + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(0, votedId), withDirectoryId) + ); store.writeElectionState( ElectionState.withVotedCandidate( epoch, ReplicaKey.of(votedId, Optional.empty()), - voters + voters.voterIds() ), kraftVersion ); @@ -1327,27 +1527,4 @@ public void testNoLocalIdInitializationFailsIfElectionStateHasVotedCandidate(sho QuorumState state2 = buildQuorumState(OptionalInt.empty(), voters, kraftVersion); assertThrows(IllegalStateException.class, () -> state2.initialize(new OffsetAndEpoch(0, 0))); } - - private QuorumState initializeEmptyState(Set voters, short kraftVersion) { - QuorumState state = buildQuorumState(voters, kraftVersion); - store.writeElectionState(ElectionState.withUnknownLeader(0, voters), kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - return state; - } - - private Set persistedVoters(Set voters, short kraftVersion) { - if (kraftVersion == 1) { - return Collections.emptySet(); - } - - return voters; - } - - private Optional persistedDirectoryId(Optional directoryId, short kraftVersion) { - if (kraftVersion == 1) { - return directoryId; - } - - return Optional.empty(); - } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 03ab95ffce..8d6b9c1cad 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; @@ -79,6 +80,7 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.apache.kafka.raft.LeaderState.CHECK_QUORUM_TIMEOUT_FACTOR; import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition; @@ -114,6 +116,7 @@ public final class RaftClientTestContext { final MockTime time; final MockListener listener; final Set voters; + final Set bootstrapIds; private final List sentResponses = new ArrayList<>(); @@ -146,6 +149,7 @@ public static final class Builder { private int electionTimeoutMs = DEFAULT_ELECTION_TIMEOUT_MS; private int appendLingerMs = DEFAULT_APPEND_LINGER_MS; private MemoryPool memoryPool = MemoryPool.NONE; + private List bootstrapServers = Collections.emptyList(); public Builder(int localId, Set voters) { this(OptionalInt.of(localId), voters); @@ -240,9 +244,14 @@ Builder withRequestTimeoutMs(int requestTimeoutMs) { return this; } + Builder withBootstrapServers(List bootstrapServers) { + this.bootstrapServers = bootstrapServers; + return this; + } + public RaftClientTestContext build() throws IOException { Metrics metrics = new Metrics(time); - MockNetworkChannel channel = new MockNetworkChannel(voters); + MockNetworkChannel channel = new MockNetworkChannel(); MockListener listener = new MockListener(localId); Map voterAddressMap = voters .stream() @@ -269,6 +278,7 @@ public RaftClientTestContext build() throws IOException { new MockExpirationService(time), FETCH_MAX_WAIT_MS, clusterId.toString(), + bootstrapServers, logContext, random, quorumConfig @@ -277,7 +287,6 @@ public RaftClientTestContext build() throws IOException { client.register(listener); client.initialize( voterAddressMap, - "CONTROLLER", quorumStateStore, metrics ); @@ -292,6 +301,11 @@ public RaftClientTestContext build() throws IOException { time, quorumStateStore, voters, + IntStream + .iterate(-2, id -> id - 1) + .limit(bootstrapServers.size()) + .boxed() + .collect(Collectors.toSet()), metrics, listener ); @@ -314,6 +328,7 @@ private RaftClientTestContext( MockTime time, QuorumStateStore quorumStateStore, Set voters, + Set bootstrapIds, Metrics metrics, MockListener listener ) { @@ -326,6 +341,7 @@ private RaftClientTestContext( this.time = time; this.quorumStateStore = quorumStateStore; this.voters = voters; + this.bootstrapIds = bootstrapIds; this.metrics = metrics; this.listener = listener; } @@ -417,7 +433,7 @@ void expectAndGrantVotes(int epoch) throws Exception { for (RaftRequest.Outbound request : voteRequests) { VoteResponseData voteResponse = voteResponse(true, Optional.empty(), epoch); - deliverResponse(request.correlationId, request.destinationId(), voteResponse); + deliverResponse(request.correlationId(), request.destination(), voteResponse); } client.poll(); @@ -432,7 +448,7 @@ private void expectBeginEpoch(int epoch) throws Exception { pollUntilRequest(); for (RaftRequest.Outbound request : collectBeginEpochRequests(epoch)) { BeginQuorumEpochResponseData beginEpochResponse = beginEpochResponse(epoch, localIdOrThrow()); - deliverResponse(request.correlationId, request.destinationId(), beginEpochResponse); + deliverResponse(request.correlationId(), request.destination(), beginEpochResponse); } client.poll(); } @@ -519,10 +535,10 @@ void assertSentDescribeQuorumResponse( assertEquals(expectedResponse, response); } - int assertSentVoteRequest(int epoch, int lastEpoch, long lastEpochOffset, int numVoteReceivers) { + RaftRequest.Outbound assertSentVoteRequest(int epoch, int lastEpoch, long lastEpochOffset, int numVoteReceivers) { List voteRequests = collectVoteRequests(epoch, lastEpoch, lastEpochOffset); assertEquals(numVoteReceivers, voteRequests.size()); - return voteRequests.iterator().next().correlationId(); + return voteRequests.iterator().next(); } void assertSentVoteResponse(Errors error) { @@ -590,14 +606,14 @@ void deliverRequest(ApiMessage request) { client.handle(inboundRequest); } - void deliverResponse(int correlationId, int sourceId, ApiMessage response) { - channel.mockReceive(new RaftResponse.Inbound(correlationId, response, sourceId)); + void deliverResponse(int correlationId, Node source, ApiMessage response) { + channel.mockReceive(new RaftResponse.Inbound(correlationId, response, source)); } - int assertSentBeginQuorumEpochRequest(int epoch, int numBeginEpochRequests) { + RaftRequest.Outbound assertSentBeginQuorumEpochRequest(int epoch, int numBeginEpochRequests) { List requests = collectBeginEpochRequests(epoch); assertEquals(numBeginEpochRequests, requests.size()); - return requests.get(0).correlationId; + return requests.get(0); } private List drainSentResponses( @@ -607,7 +623,7 @@ private List drainSentResponses( Iterator iterator = sentResponses.iterator(); while (iterator.hasNext()) { RaftResponse.Outbound response = iterator.next(); - if (response.data.apiKey() == apiKey.id) { + if (response.data().apiKey() == apiKey.id) { res.add(response); iterator.remove(); } @@ -646,11 +662,14 @@ void assertSentBeginQuorumEpochResponse( assertEquals(partitionError, Errors.forCode(partitionResponse.errorCode())); } - int assertSentEndQuorumEpochRequest(int epoch, int destinationId) { + RaftRequest.Outbound assertSentEndQuorumEpochRequest(int epoch, int destinationId) { List endQuorumRequests = collectEndQuorumRequests( - epoch, Collections.singleton(destinationId), Optional.empty()); + epoch, + Collections.singleton(destinationId), + Optional.empty() + ); assertEquals(1, endQuorumRequests.size()); - return endQuorumRequests.get(0).correlationId(); + return endQuorumRequests.get(0); } void assertSentEndQuorumEpochResponse( @@ -690,7 +709,7 @@ RaftRequest.Outbound assertSentFetchRequest() { return sentRequests.get(0); } - int assertSentFetchRequest( + RaftRequest.Outbound assertSentFetchRequest( int epoch, long fetchOffset, int lastFetchedEpoch @@ -700,7 +719,7 @@ int assertSentFetchRequest( RaftRequest.Outbound raftRequest = sentMessages.get(0); assertFetchRequestData(raftRequest, epoch, fetchOffset, lastFetchedEpoch); - return raftRequest.correlationId(); + return raftRequest; } FetchResponseData.PartitionData assertSentFetchPartitionResponse() { @@ -708,7 +727,7 @@ FetchResponseData.PartitionData assertSentFetchPartitionResponse() { assertEquals( 1, sentMessages.size(), "Found unexpected sent messages " + sentMessages); RaftResponse.Outbound raftMessage = sentMessages.get(0); - assertEquals(ApiKeys.FETCH.id, raftMessage.data.apiKey()); + assertEquals(ApiKeys.FETCH.id, raftMessage.data().apiKey()); FetchResponseData response = (FetchResponseData) raftMessage.data(); assertEquals(Errors.NONE, Errors.forCode(response.errorCode())); @@ -723,7 +742,7 @@ void assertSentFetchPartitionResponse(Errors topLevelError) { assertEquals( 1, sentMessages.size(), "Found unexpected sent messages " + sentMessages); RaftResponse.Outbound raftMessage = sentMessages.get(0); - assertEquals(ApiKeys.FETCH.id, raftMessage.data.apiKey()); + assertEquals(ApiKeys.FETCH.id, raftMessage.data().apiKey()); FetchResponseData response = (FetchResponseData) raftMessage.data(); assertEquals(topLevelError, Errors.forCode(response.errorCode())); } @@ -811,7 +830,7 @@ List collectEndQuorumRequests( assertEquals(preferredSuccessors, partitionRequest.preferredSuccessors()); }); - collectedDestinationIdSet.add(raftMessage.destinationId()); + collectedDestinationIdSet.add(raftMessage.destination().id()); endQuorumRequests.add(raftMessage); } } @@ -825,11 +844,18 @@ void discoverLeaderAsObserver( ) throws Exception { pollUntilRequest(); RaftRequest.Outbound fetchRequest = assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destinationId())); + int destinationId = fetchRequest.destination().id(); + assertTrue( + voters.contains(destinationId) || bootstrapIds.contains(destinationId), + String.format("id %d is not in sets %s or %s", destinationId, voters, bootstrapIds) + ); assertFetchRequestData(fetchRequest, 0, 0L, 0); - deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), - fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.NONE)); + deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.NONE) + ); client.poll(); assertElectedLeader(epoch, leaderId); } @@ -850,7 +876,7 @@ private List collectBeginEpochRequests(int epoch) { return requests; } - private static InetSocketAddress mockAddress(int id) { + public static InetSocketAddress mockAddress(int id) { return new InetSocketAddress("localhost", 9990 + id); } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index f52ee371f4..4896571c22 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -21,6 +21,7 @@ import net.jqwik.api.Property; import net.jqwik.api.Tag; import net.jqwik.api.constraints.IntRange; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.memory.MemoryPool; @@ -45,6 +46,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -59,7 +61,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; -import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -189,7 +190,7 @@ void canRecoverAfterAllNodesKilled( // they are able to elect a leader and continue making progress cluster.killAll(); - Iterator nodeIdsIterator = cluster.nodes().iterator(); + Iterator nodeIdsIterator = cluster.nodeIds().iterator(); for (int i = 0; i < cluster.majoritySize(); i++) { Integer nodeId = nodeIdsIterator.next(); cluster.start(nodeId); @@ -224,7 +225,7 @@ void canElectNewLeaderAfterOldLeaderPartitionedAway( ); router.filter(leaderId, new DropAllTraffic()); - Set nonPartitionedNodes = new HashSet<>(cluster.nodes()); + Set nonPartitionedNodes = new HashSet<>(cluster.nodeIds()); nonPartitionedNodes.remove(leaderId); scheduler.runUntil(() -> cluster.allReachedHighWatermark(20, nonPartitionedNodes)); @@ -252,11 +253,17 @@ void canMakeProgressIfMajorityIsReachable( // Partition the nodes into two sets. Nodes are reachable within each set, // but the two sets cannot communicate with each other. We should be able // to make progress even if an election is needed in the larger set. - router.filter(0, new DropOutboundRequestsFrom(Utils.mkSet(2, 3, 4))); - router.filter(1, new DropOutboundRequestsFrom(Utils.mkSet(2, 3, 4))); - router.filter(2, new DropOutboundRequestsFrom(Utils.mkSet(0, 1))); - router.filter(3, new DropOutboundRequestsFrom(Utils.mkSet(0, 1))); - router.filter(4, new DropOutboundRequestsFrom(Utils.mkSet(0, 1))); + router.filter( + 0, + new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(2, 3, 4))) + ); + router.filter( + 1, + new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(2, 3, 4))) + ); + router.filter(2, new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(0, 1)))); + router.filter(3, new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(0, 1)))); + router.filter(4, new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(0, 1)))); long partitionLogEndOffset = cluster.maxLogEndOffset(); scheduler.runUntil(() -> cluster.anyReachedHighWatermark(2 * partitionLogEndOffset)); @@ -374,7 +381,7 @@ private void schedulePolling(EventScheduler scheduler, int pollIntervalMs, int pollJitterMs) { int delayMs = 0; - for (int nodeId : cluster.nodes()) { + for (int nodeId : cluster.nodeIds()) { scheduler.schedule(() -> cluster.pollIfRunning(nodeId), delayMs, pollIntervalMs, pollJitterMs); delayMs++; } @@ -527,25 +534,37 @@ private static class Cluster { final AtomicInteger correlationIdCounter = new AtomicInteger(); final MockTime time = new MockTime(); final Uuid clusterId = Uuid.randomUuid(); - final Set voters = new HashSet<>(); + final Map voters = new HashMap<>(); final Map nodes = new HashMap<>(); final Map running = new HashMap<>(); private Cluster(int numVoters, int numObservers, Random random) { this.random = random; - int nodeId = 0; - for (; nodeId < numVoters; nodeId++) { - voters.add(nodeId); + for (int nodeId = 0; nodeId < numVoters; nodeId++) { + voters.put( + nodeId, + new Node(nodeId, String.format("host-node-%d", nodeId), 1234) + ); nodes.put(nodeId, new PersistentState(nodeId)); } - for (; nodeId < numVoters + numObservers; nodeId++) { + for (int nodeIdDelta = 0; nodeIdDelta < numObservers; nodeIdDelta++) { + int nodeId = numVoters + nodeIdDelta; nodes.put(nodeId, new PersistentState(nodeId)); } } - Set nodes() { + Set endpointsFromIds(Set nodeIds) { + return voters + .values() + .stream() + .filter(node -> nodeIds.contains(node.id())) + .map(Cluster::nodeAddress) + .collect(Collectors.toSet()); + } + + Set nodeIds() { return nodes.keySet(); } @@ -710,18 +729,19 @@ void killAndDeletePersistentState(int nodeId) { nodes.put(nodeId, new PersistentState(nodeId)); } - private static InetSocketAddress nodeAddress(int id) { - return new InetSocketAddress("localhost", 9990 + id); + private static InetSocketAddress nodeAddress(Node node) { + return InetSocketAddress.createUnresolved(node.host(), node.port()); } void start(int nodeId) { LogContext logContext = new LogContext("[Node " + nodeId + "] "); PersistentState persistentState = nodes.get(nodeId); - MockNetworkChannel channel = new MockNetworkChannel(correlationIdCounter, voters); + MockNetworkChannel channel = new MockNetworkChannel(correlationIdCounter); MockMessageQueue messageQueue = new MockMessageQueue(); Map voterAddressMap = voters + .values() .stream() - .collect(Collectors.toMap(Function.identity(), Cluster::nodeAddress)); + .collect(Collectors.toMap(Node::id, Cluster::nodeAddress)); QuorumConfig quorumConfig = new QuorumConfig( REQUEST_TIMEOUT_MS, @@ -750,6 +770,7 @@ void start(int nodeId) { new MockExpirationService(time), FETCH_MAX_WAIT_MS, clusterId.toString(), + Collections.emptyList(), logContext, random, quorumConfig @@ -808,7 +829,6 @@ void initialize(Map voterAddresses, Metrics metrics) client.register(counter); client.initialize( voterAddresses, - "CONTROLLER", store, metrics ); @@ -847,9 +867,11 @@ public String toString() { private static class InflightRequest { final int sourceId; + final Node destination; - private InflightRequest(int sourceId) { + private InflightRequest(int sourceId, Node destination) { this.sourceId = sourceId; + this.destination = destination; } } @@ -884,11 +906,15 @@ public boolean acceptOutbound(RaftMessage message) { } } - private static class DropOutboundRequestsFrom implements NetworkFilter { - - private final Set unreachable; + private static class DropOutboundRequestsTo implements NetworkFilter { + private final Set unreachable; - private DropOutboundRequestsFrom(Set unreachable) { + /** + * This network filter drops any outbound message sent to the {@code unreachable} nodes. + * + * @param unreachable the set of destination address which are not reachable + */ + private DropOutboundRequestsTo(Set unreachable) { this.unreachable = unreachable; } @@ -897,11 +923,25 @@ public boolean acceptInbound(RaftMessage message) { return true; } + /** + * Returns if the message should be sent to the destination. + * + * Returns false when outbound request messages contains a destination {@code Node} that + * matches the set of unreaable {@code InetSocketAddress}. Note that the {@code Node.id()} + * and {@code Node.rack()} are not compared. + * + * @param message the raft message + * @return true if the message should be delivered, otherwise false + */ @Override public boolean acceptOutbound(RaftMessage message) { if (message instanceof RaftRequest.Outbound) { RaftRequest.Outbound request = (RaftRequest.Outbound) message; - return !unreachable.contains(request.destinationId()); + InetSocketAddress destination = InetSocketAddress.createUnresolved( + request.destination().host(), + request.destination().port() + ); + return !unreachable.contains(destination); } return true; } @@ -955,7 +995,7 @@ private MajorityReachedHighWatermark(Cluster cluster) { public void verify() { cluster.leaderHighWatermark().ifPresent(highWatermark -> { long numReachedHighWatermark = cluster.nodes.entrySet().stream() - .filter(entry -> cluster.voters.contains(entry.getKey())) + .filter(entry -> cluster.voters.containsKey(entry.getKey())) .filter(entry -> entry.getValue().log.endOffset().offset >= highWatermark) .count(); assertTrue( @@ -1194,19 +1234,19 @@ void deliver(int senderId, RaftRequest.Outbound outbound) { return; int correlationId = outbound.correlationId(); - int destinationId = outbound.destinationId(); + Node destination = outbound.destination(); RaftRequest.Inbound inbound = new RaftRequest.Inbound(correlationId, outbound.data(), cluster.time.milliseconds()); - if (!filters.get(destinationId).acceptInbound(inbound)) + if (!filters.get(destination.id()).acceptInbound(inbound)) return; - cluster.nodeIfRunning(destinationId).ifPresent(node -> { - inflight.put(correlationId, new InflightRequest(senderId)); + cluster.nodeIfRunning(destination.id()).ifPresent(node -> { + inflight.put(correlationId, new InflightRequest(senderId, destination)); inbound.completion.whenComplete((response, exception) -> { - if (response != null && filters.get(destinationId).acceptOutbound(response)) { - deliver(destinationId, response); + if (response != null && filters.get(destination.id()).acceptOutbound(response)) { + deliver(response); } }); @@ -1214,11 +1254,17 @@ void deliver(int senderId, RaftRequest.Outbound outbound) { }); } - void deliver(int senderId, RaftResponse.Outbound outbound) { + void deliver(RaftResponse.Outbound outbound) { int correlationId = outbound.correlationId(); - RaftResponse.Inbound inbound = new RaftResponse.Inbound(correlationId, outbound.data(), senderId); InflightRequest inflightRequest = inflight.remove(correlationId); + RaftResponse.Inbound inbound = new RaftResponse.Inbound( + correlationId, + outbound.data(), + // The source of the response is the destination of the request + inflightRequest.destination + ); + if (!filters.get(inflightRequest.sourceId).acceptInbound(inbound)) return; diff --git a/raft/src/test/java/org/apache/kafka/raft/RequestManagerTest.java b/raft/src/test/java/org/apache/kafka/raft/RequestManagerTest.java index e6e2f7cf0a..326222e18c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RequestManagerTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RequestManagerTest.java @@ -16,14 +16,20 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.Node; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Utils; import org.junit.jupiter.api.Test; +import java.util.List; +import java.util.Optional; import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertNotEquals; public class RequestManagerTest { private final MockTime time = new MockTime(); @@ -33,105 +39,247 @@ public class RequestManagerTest { @Test public void testResetAllConnections() { + Node node1 = new Node(1, "mock-host-1", 4321); + Node node2 = new Node(2, "mock-host-2", 4321); + RequestManager cache = new RequestManager( - Utils.mkSet(1, 2, 3), + makeBootstrapList(3), retryBackoffMs, requestTimeoutMs, - random); + random + ); // One host has an inflight request - RequestManager.ConnectionState connectionState1 = cache.getOrCreate(1); - connectionState1.onRequestSent(1, time.milliseconds()); - assertFalse(connectionState1.isReady(time.milliseconds())); + cache.onRequestSent(node1, 1, time.milliseconds()); + assertFalse(cache.isReady(node1, time.milliseconds())); // Another is backing off - RequestManager.ConnectionState connectionState2 = cache.getOrCreate(2); - connectionState2.onRequestSent(2, time.milliseconds()); - connectionState2.onResponseError(2, time.milliseconds()); - assertFalse(connectionState2.isReady(time.milliseconds())); + cache.onRequestSent(node2, 2, time.milliseconds()); + cache.onResponseResult(node2, 2, false, time.milliseconds()); + assertFalse(cache.isReady(node2, time.milliseconds())); cache.resetAll(); // Now both should be ready - assertTrue(connectionState1.isReady(time.milliseconds())); - assertTrue(connectionState2.isReady(time.milliseconds())); + assertTrue(cache.isReady(node1, time.milliseconds())); + assertTrue(cache.isReady(node2, time.milliseconds())); } @Test public void testBackoffAfterFailure() { + Node node = new Node(1, "mock-host-1", 4321); + RequestManager cache = new RequestManager( - Utils.mkSet(1, 2, 3), + makeBootstrapList(3), retryBackoffMs, requestTimeoutMs, - random); + random + ); - RequestManager.ConnectionState connectionState = cache.getOrCreate(1); - assertTrue(connectionState.isReady(time.milliseconds())); + assertTrue(cache.isReady(node, time.milliseconds())); long correlationId = 1; - connectionState.onRequestSent(correlationId, time.milliseconds()); - assertFalse(connectionState.isReady(time.milliseconds())); + cache.onRequestSent(node, correlationId, time.milliseconds()); + assertFalse(cache.isReady(node, time.milliseconds())); - connectionState.onResponseError(correlationId, time.milliseconds()); - assertFalse(connectionState.isReady(time.milliseconds())); + cache.onResponseResult(node, correlationId, false, time.milliseconds()); + assertFalse(cache.isReady(node, time.milliseconds())); time.sleep(retryBackoffMs); - assertTrue(connectionState.isReady(time.milliseconds())); + assertTrue(cache.isReady(node, time.milliseconds())); } @Test public void testSuccessfulResponse() { + Node node = new Node(1, "mock-host-1", 4321); + RequestManager cache = new RequestManager( - Utils.mkSet(1, 2, 3), + makeBootstrapList(3), retryBackoffMs, requestTimeoutMs, - random); - - RequestManager.ConnectionState connectionState = cache.getOrCreate(1); + random + ); long correlationId = 1; - connectionState.onRequestSent(correlationId, time.milliseconds()); - assertFalse(connectionState.isReady(time.milliseconds())); - connectionState.onResponseReceived(correlationId); - assertTrue(connectionState.isReady(time.milliseconds())); + cache.onRequestSent(node, correlationId, time.milliseconds()); + assertFalse(cache.isReady(node, time.milliseconds())); + cache.onResponseResult(node, correlationId, true, time.milliseconds()); + assertTrue(cache.isReady(node, time.milliseconds())); } @Test public void testIgnoreUnexpectedResponse() { + Node node = new Node(1, "mock-host-1", 4321); + RequestManager cache = new RequestManager( - Utils.mkSet(1, 2, 3), + makeBootstrapList(3), retryBackoffMs, requestTimeoutMs, - random); - - RequestManager.ConnectionState connectionState = cache.getOrCreate(1); + random + ); long correlationId = 1; - connectionState.onRequestSent(correlationId, time.milliseconds()); - assertFalse(connectionState.isReady(time.milliseconds())); - connectionState.onResponseReceived(correlationId + 1); - assertFalse(connectionState.isReady(time.milliseconds())); + cache.onRequestSent(node, correlationId, time.milliseconds()); + assertFalse(cache.isReady(node, time.milliseconds())); + cache.onResponseResult(node, correlationId + 1, true, time.milliseconds()); + assertFalse(cache.isReady(node, time.milliseconds())); } @Test public void testRequestTimeout() { + Node node = new Node(1, "mock-host-1", 4321); + RequestManager cache = new RequestManager( - Utils.mkSet(1, 2, 3), + makeBootstrapList(3), retryBackoffMs, requestTimeoutMs, - random); - - RequestManager.ConnectionState connectionState = cache.getOrCreate(1); + random + ); long correlationId = 1; - connectionState.onRequestSent(correlationId, time.milliseconds()); - assertFalse(connectionState.isReady(time.milliseconds())); + cache.onRequestSent(node, correlationId, time.milliseconds()); + assertFalse(cache.isReady(node, time.milliseconds())); time.sleep(requestTimeoutMs - 1); - assertFalse(connectionState.isReady(time.milliseconds())); + assertFalse(cache.isReady(node, time.milliseconds())); time.sleep(1); - assertTrue(connectionState.isReady(time.milliseconds())); + assertTrue(cache.isReady(node, time.milliseconds())); } + @Test + public void testRequestToBootstrapList() { + List bootstrapList = makeBootstrapList(2); + RequestManager cache = new RequestManager( + bootstrapList, + retryBackoffMs, + requestTimeoutMs, + random + ); + + // Find a ready node with the starting state + Node bootstrapNode1 = cache.findReadyBootstrapServer(time.milliseconds()).get(); + assertTrue( + bootstrapList.contains(bootstrapNode1), + String.format("%s is not in %s", bootstrapNode1, bootstrapList) + ); + assertEquals(0, cache.backoffBeforeAvailableBootstrapServer(time.milliseconds())); + + // Send a request and check the cache state + cache.onRequestSent(bootstrapNode1, 1, time.milliseconds()); + assertEquals( + Optional.empty(), + cache.findReadyBootstrapServer(time.milliseconds()) + ); + assertEquals(requestTimeoutMs, cache.backoffBeforeAvailableBootstrapServer(time.milliseconds())); + + // Fail the request + time.sleep(100); + cache.onResponseResult(bootstrapNode1, 1, false, time.milliseconds()); + Node bootstrapNode2 = cache.findReadyBootstrapServer(time.milliseconds()).get(); + assertNotEquals(bootstrapNode1, bootstrapNode2); + assertEquals(0, cache.backoffBeforeAvailableBootstrapServer(time.milliseconds())); + + // Send a request to the second node and check the state + cache.onRequestSent(bootstrapNode2, 2, time.milliseconds()); + assertEquals( + Optional.empty(), + cache.findReadyBootstrapServer(time.milliseconds()) + ); + assertEquals(requestTimeoutMs, cache.backoffBeforeAvailableBootstrapServer(time.milliseconds())); + + + // Fail the second request before the request timeout + time.sleep(retryBackoffMs - 1); + cache.onResponseResult(bootstrapNode2, 2, false, time.milliseconds()); + assertEquals( + Optional.empty(), + cache.findReadyBootstrapServer(time.milliseconds()) + ); + assertEquals(1, cache.backoffBeforeAvailableBootstrapServer(time.milliseconds())); + + // Timeout the first backoff and show that that node is ready + time.sleep(1); + Node bootstrapNode3 = cache.findReadyBootstrapServer(time.milliseconds()).get(); + assertEquals(bootstrapNode1, bootstrapNode3); + assertEquals(0, cache.backoffBeforeAvailableBootstrapServer(time.milliseconds())); + } + + @Test + public void testFindReadyWithInflightRequest() { + Node otherNode = new Node(1, "other-node", 1234); + List bootstrapList = makeBootstrapList(3); + RequestManager cache = new RequestManager( + bootstrapList, + retryBackoffMs, + requestTimeoutMs, + random + ); + + // Send request to a node that is not in the bootstrap list + cache.onRequestSent(otherNode, 1, time.milliseconds()); + assertEquals(Optional.empty(), cache.findReadyBootstrapServer(time.milliseconds())); + } + + @Test + public void testFindReadyWithRequestTimedout() { + Node otherNode = new Node(1, "other-node", 1234); + List bootstrapList = makeBootstrapList(3); + RequestManager cache = new RequestManager( + bootstrapList, + retryBackoffMs, + requestTimeoutMs, + random + ); + + // Send request to a node that is not in the bootstrap list + cache.onRequestSent(otherNode, 1, time.milliseconds()); + assertTrue(cache.isResponseExpected(otherNode, 1)); + assertEquals(Optional.empty(), cache.findReadyBootstrapServer(time.milliseconds())); + + // Timeout the request + time.sleep(requestTimeoutMs); + Node bootstrapNode = cache.findReadyBootstrapServer(time.milliseconds()).get(); + assertTrue(bootstrapList.contains(bootstrapNode)); + assertFalse(cache.isResponseExpected(otherNode, 1)); + } + + @Test + public void testAnyInflightRequestWithAnyRequest() { + Node otherNode = new Node(1, "other-node", 1234); + List bootstrapList = makeBootstrapList(3); + RequestManager cache = new RequestManager( + bootstrapList, + retryBackoffMs, + requestTimeoutMs, + random + ); + + assertFalse(cache.hasAnyInflightRequest(time.milliseconds())); + + // Send a request and check state + cache.onRequestSent(otherNode, 11, time.milliseconds()); + assertTrue(cache.hasAnyInflightRequest(time.milliseconds())); + + // Wait until the request times out + time.sleep(requestTimeoutMs); + assertFalse(cache.hasAnyInflightRequest(time.milliseconds())); + + // Send another request and fail it + cache.onRequestSent(otherNode, 12, time.milliseconds()); + cache.onResponseResult(otherNode, 12, false, time.milliseconds()); + assertFalse(cache.hasAnyInflightRequest(time.milliseconds())); + + // Send another request and mark it successful + cache.onRequestSent(otherNode, 12, time.milliseconds()); + cache.onResponseResult(otherNode, 12, true, time.milliseconds()); + assertFalse(cache.hasAnyInflightRequest(time.milliseconds())); + } + + private List makeBootstrapList(int numberOfNodes) { + return IntStream.iterate(-2, id -> id - 1) + .limit(numberOfNodes) + .mapToObj(id -> new Node(id, String.format("mock-boot-host%d", id), 1234)) + .collect(Collectors.toList()); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java index 80f7df026f..82d51625ea 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java @@ -16,8 +16,8 @@ */ package org.apache.kafka.raft.internals; -import java.util.Arrays; import java.util.Optional; +import java.util.stream.IntStream; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.KRaftVersionRecord; @@ -52,7 +52,7 @@ private static KRaftControlRecordStateMachine buildPartitionListener(MockLog log @Test void testEmptyPartition() { MockLog log = buildLog(); - VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(voterSet)); @@ -65,7 +65,7 @@ void testEmptyPartition() { @Test void testUpdateWithoutSnapshot() { MockLog log = buildLog(); - VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; @@ -85,7 +85,7 @@ void testUpdateWithoutSnapshot() { ); // Append the voter set control record - VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6), true)); + VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(4, 5, 6), true)); log.appendAsLeader( MemoryRecords.withVotersRecord( log.endOffset().offset, @@ -108,7 +108,7 @@ void testUpdateWithoutSnapshot() { @Test void testUpdateWithEmptySnapshot() { MockLog log = buildLog(); - VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; @@ -136,7 +136,7 @@ void testUpdateWithEmptySnapshot() { ); // Append the voter set control record - VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6), true)); + VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(4, 5, 6), true)); log.appendAsLeader( MemoryRecords.withVotersRecord( log.endOffset().offset, @@ -159,14 +159,14 @@ void testUpdateWithEmptySnapshot() { @Test void testUpdateWithSnapshot() { MockLog log = buildLog(); - VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); int epoch = 1; KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); // Create a snapshot that has kraft.version and voter set control records short kraftVersion = 1; - VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6), true)); + VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(4, 5, 6), true)); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setRawSnapshotWriter(log.createNewSnapshotUnchecked(new OffsetAndEpoch(10, epoch)).get()) @@ -188,7 +188,7 @@ void testUpdateWithSnapshot() { @Test void testUpdateWithSnapshotAndLogOverride() { MockLog log = buildLog(); - VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; @@ -196,7 +196,7 @@ void testUpdateWithSnapshotAndLogOverride() { // Create a snapshot that has kraft.version and voter set control records short kraftVersion = 1; - VoterSet snapshotVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6), true)); + VoterSet snapshotVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(4, 5, 6), true)); OffsetAndEpoch snapshotId = new OffsetAndEpoch(10, epoch); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() @@ -235,7 +235,7 @@ void testUpdateWithSnapshotAndLogOverride() { @Test void testTruncateTo() { MockLog log = buildLog(); - VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; @@ -256,7 +256,7 @@ void testTruncateTo() { // Append the voter set control record long firstVoterSetOffset = log.endOffset().offset; - VoterSet firstVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6), true)); + VoterSet firstVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(4, 5, 6), true)); log.appendAsLeader( MemoryRecords.withVotersRecord( firstVoterSetOffset, @@ -303,7 +303,7 @@ void testTruncateTo() { @Test void testTrimPrefixTo() { MockLog log = buildLog(); - VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; @@ -325,7 +325,7 @@ void testTrimPrefixTo() { // Append the voter set control record long firstVoterSetOffset = log.endOffset().offset; - VoterSet firstVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(4, 5, 6), true)); + VoterSet firstVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(4, 5, 6), true)); log.appendAsLeader( MemoryRecords.withVotersRecord( firstVoterSetOffset, diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index 1b729e36d3..240a55d440 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.LogOffsetMetadata; import org.apache.kafka.raft.MockQuorumStateStore; import org.apache.kafka.raft.OffsetAndEpoch; @@ -32,13 +31,13 @@ import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; -import java.util.Map; import java.util.Collections; +import java.util.Map; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Random; -import java.util.Set; +import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -64,19 +63,11 @@ public void tearDown() { metrics.close(); } - private QuorumState buildQuorumState(Set voters, short kraftVersion) { - boolean withDirectoryId = kraftVersion > 0; - - return buildQuorumState( - VoterSetTest.voterSet(VoterSetTest.voterMap(voters, withDirectoryId)), - kraftVersion - ); - } - private QuorumState buildQuorumState(VoterSet voterSet, short kraftVersion) { return new QuorumState( OptionalInt.of(localId), localDirectoryId, + VoterSetTest.DEFAULT_LISTENER_NAME, () -> voterSet, () -> kraftVersion, electionTimeoutMs, @@ -88,11 +79,26 @@ private QuorumState buildQuorumState(VoterSet voterSet, short kraftVersion) { ); } + private VoterSet localStandaloneVoterSet(short kraftVersion) { + boolean withDirectoryId = kraftVersion > 0; + return VoterSetTest.voterSet( + Collections.singletonMap( + localId, + VoterSetTest.voterNode( + ReplicaKey.of( + localId, + withDirectoryId ? Optional.of(localDirectoryId) : Optional.empty() + ) + ) + ) + ); + } + @ParameterizedTest @ValueSource(shorts = {0, 1}) public void shouldRecordVoterQuorumState(short kraftVersion) { boolean withDirectoryId = kraftVersion > 0; - Map voterMap = VoterSetTest.voterMap(Utils.mkSet(1, 2), withDirectoryId); + Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2), withDirectoryId); voterMap.put( localId, VoterSetTest.voterNode( @@ -102,7 +108,8 @@ public void shouldRecordVoterQuorumState(short kraftVersion) { ) ) ); - QuorumState state = buildQuorumState(VoterSetTest.voterSet(voterMap), kraftVersion); + VoterSet voters = VoterSetTest.voterSet(voterMap); + QuorumState state = buildQuorumState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); raftMetrics = new KafkaRaftMetrics(metrics, "raft", state); @@ -144,7 +151,7 @@ public void shouldRecordVoterQuorumState(short kraftVersion) { state.leaderStateOrThrow().updateReplicaState(1, 0, new LogOffsetMetadata(5L)); assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToFollower(2, 1); + state.transitionToFollower(2, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); assertEquals("follower", getMetric(metrics, "current-state").metricValue()); assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); @@ -184,7 +191,11 @@ public void shouldRecordVoterQuorumState(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void shouldRecordNonVoterQuorumState(short kraftVersion) { - QuorumState state = buildQuorumState(Utils.mkSet(1, 2, 3), kraftVersion); + boolean withDirectoryId = kraftVersion > 0; + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(1, 2, 3), withDirectoryId) + ); + QuorumState state = buildQuorumState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); raftMetrics = new KafkaRaftMetrics(metrics, "raft", state); @@ -198,7 +209,7 @@ public void shouldRecordNonVoterQuorumState(short kraftVersion) { assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToFollower(2, 1); + state.transitionToFollower(2, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); assertEquals("observer", getMetric(metrics, "current-state").metricValue()); assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); @@ -227,7 +238,7 @@ public void shouldRecordNonVoterQuorumState(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void shouldRecordLogEnd(short kraftVersion) { - QuorumState state = buildQuorumState(Collections.singleton(localId), kraftVersion); + QuorumState state = buildQuorumState(localStandaloneVoterSet(kraftVersion), kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); raftMetrics = new KafkaRaftMetrics(metrics, "raft", state); @@ -243,7 +254,7 @@ public void shouldRecordLogEnd(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void shouldRecordNumUnknownVoterConnections(short kraftVersion) { - QuorumState state = buildQuorumState(Collections.singleton(localId), kraftVersion); + QuorumState state = buildQuorumState(localStandaloneVoterSet(kraftVersion), kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); raftMetrics = new KafkaRaftMetrics(metrics, "raft", state); @@ -257,7 +268,7 @@ public void shouldRecordNumUnknownVoterConnections(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void shouldRecordPollIdleRatio(short kraftVersion) { - QuorumState state = buildQuorumState(Collections.singleton(localId), kraftVersion); + QuorumState state = buildQuorumState(localStandaloneVoterSet(kraftVersion), kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); raftMetrics = new KafkaRaftMetrics(metrics, "raft", state); @@ -330,7 +341,7 @@ public void shouldRecordPollIdleRatio(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void shouldRecordLatency(short kraftVersion) { - QuorumState state = buildQuorumState(Collections.singleton(localId), kraftVersion); + QuorumState state = buildQuorumState(localStandaloneVoterSet(kraftVersion), kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); raftMetrics = new KafkaRaftMetrics(metrics, "raft", state); @@ -362,7 +373,7 @@ public void shouldRecordLatency(short kraftVersion) { @ParameterizedTest @ValueSource(shorts = {0, 1}) public void shouldRecordRate(short kraftVersion) { - QuorumState state = buildQuorumState(Collections.singleton(localId), kraftVersion); + QuorumState state = buildQuorumState(localStandaloneVoterSet(kraftVersion), kraftVersion); state.initialize(new OffsetAndEpoch(0L, 0)); raftMetrics = new KafkaRaftMetrics(metrics, "raft", state); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index 3a64479fad..e8896f3b57 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.IdentityHashMap; import java.util.List; import java.util.NoSuchElementException; @@ -31,6 +30,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; import net.jqwik.api.ForAll; import net.jqwik.api.Property; @@ -204,7 +204,7 @@ public void testControlRecordIterationWithKraftVersion0() { public void testControlRecordIterationWithKraftVersion1() { AtomicReference buffer = new AtomicReference<>(null); VoterSet voterSet = new VoterSet( - new HashMap<>(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)) + VoterSetTest.voterMap(IntStream.of(1, 2, 3), true) ); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setTime(new MockTime()) diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 22dd52ec36..ac5c3b39c9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -16,10 +16,10 @@ */ package org.apache.kafka.raft.internals; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.stream.IntStream; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -27,7 +27,7 @@ final public class VoterSetHistoryTest { @Test void testStaticVoterSet() { - VoterSet staticVoterSet = new VoterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet staticVoterSet = new VoterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); @@ -58,13 +58,13 @@ void TestNoStaticVoterSet() { @Test void testAddAt() { - Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true); + Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); assertThrows( IllegalArgumentException.class, - () -> votersHistory.addAt(-1, new VoterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true))) + () -> votersHistory.addAt(-1, new VoterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true))) ); assertEquals(staticVoterSet, votersHistory.lastValue()); @@ -90,7 +90,7 @@ void testAddAt() { void testAddAtNonOverlapping() { VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); - Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true); + Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); VoterSet voterSet = new VoterSet(new HashMap<>(voterMap)); // Add a starting voter to the history @@ -122,7 +122,7 @@ void testAddAtNonOverlapping() { @Test void testNonoverlappingFromStaticVoterSet() { - Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true); + Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); @@ -137,7 +137,7 @@ void testNonoverlappingFromStaticVoterSet() { @Test void testTruncateTo() { - Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true); + Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); @@ -163,7 +163,7 @@ void testTruncateTo() { @Test void testTrimPrefixTo() { - Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true); + Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); @@ -196,7 +196,7 @@ void testTrimPrefixTo() { @Test void testClear() { - Map voterMap = VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true); + Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java index f0ed10a542..9f879db10e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java @@ -18,7 +18,6 @@ import java.net.InetSocketAddress; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -26,8 +25,13 @@ import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.utils.Utils; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -41,22 +45,45 @@ void testEmptyVoterSet() { } @Test - void testVoterAddress() { - VoterSet voterSet = new VoterSet(voterMap(Arrays.asList(1, 2, 3), true)); - assertEquals(Optional.of(new InetSocketAddress("replica-1", 1234)), voterSet.voterAddress(1, "LISTENER")); - assertEquals(Optional.empty(), voterSet.voterAddress(1, "MISSING")); - assertEquals(Optional.empty(), voterSet.voterAddress(4, "LISTENER")); + void testVoterNode() { + VoterSet voterSet = new VoterSet(voterMap(IntStream.of(1, 2, 3), true)); + assertEquals( + Optional.of(new Node(1, "replica-1", 1234)), + voterSet.voterNode(1, DEFAULT_LISTENER_NAME) + ); + assertEquals(Optional.empty(), voterSet.voterNode(1, ListenerName.normalised("MISSING"))); + assertEquals(Optional.empty(), voterSet.voterNode(4, DEFAULT_LISTENER_NAME)); + } + + @Test + void testVoterNodes() { + VoterSet voterSet = new VoterSet(voterMap(IntStream.of(1, 2, 3), true)); + + assertEquals( + Utils.mkSet(new Node(1, "replica-1", 1234), new Node(2, "replica-2", 1234)), + voterSet.voterNodes(IntStream.of(1, 2).boxed(), DEFAULT_LISTENER_NAME) + ); + + assertThrows( + IllegalArgumentException.class, + () -> voterSet.voterNodes(IntStream.of(1, 2).boxed(), ListenerName.normalised("MISSING")) + ); + + assertThrows( + IllegalArgumentException.class, + () -> voterSet.voterNodes(IntStream.of(1, 4).boxed(), DEFAULT_LISTENER_NAME) + ); } @Test void testVoterIds() { - VoterSet voterSet = new VoterSet(voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet voterSet = new VoterSet(voterMap(IntStream.of(1, 2, 3), true)); assertEquals(new HashSet<>(Arrays.asList(1, 2, 3)), voterSet.voterIds()); } @Test void testAddVoter() { - Map aVoterMap = voterMap(Arrays.asList(1, 2, 3), true); + Map aVoterMap = voterMap(IntStream.of(1, 2, 3), true); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertEquals(Optional.empty(), voterSet.addVoter(voterNode(1, true))); @@ -68,7 +95,7 @@ void testAddVoter() { @Test void testRemoveVoter() { - Map aVoterMap = voterMap(Arrays.asList(1, 2, 3), true); + Map aVoterMap = voterMap(IntStream.of(1, 2, 3), true); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Optional.empty()))); @@ -83,7 +110,7 @@ void testRemoveVoter() { @Test void testIsVoterWithDirectoryId() { - Map aVoterMap = voterMap(Arrays.asList(1, 2, 3), true); + Map aVoterMap = voterMap(IntStream.of(1, 2, 3), true); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertTrue(voterSet.isVoter(aVoterMap.get(1).voterKey())); @@ -100,7 +127,7 @@ void testIsVoterWithDirectoryId() { @Test void testIsVoterWithoutDirectoryId() { - Map aVoterMap = voterMap(Arrays.asList(1, 2, 3), false); + Map aVoterMap = voterMap(IntStream.of(1, 2, 3), false); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertTrue(voterSet.isVoter(ReplicaKey.of(1, Optional.empty()))); @@ -111,7 +138,7 @@ void testIsVoterWithoutDirectoryId() { @Test void testIsOnlyVoterInStandalone() { - Map aVoterMap = voterMap(Arrays.asList(1), true); + Map aVoterMap = voterMap(IntStream.of(1), true); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertTrue(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey())); @@ -125,7 +152,7 @@ void testIsOnlyVoterInStandalone() { @Test void testIsOnlyVoterInNotStandalone() { - Map aVoterMap = voterMap(Arrays.asList(1, 2), true); + Map aVoterMap = voterMap(IntStream.of(1, 2), true); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); assertFalse(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey())); @@ -142,14 +169,14 @@ void testIsOnlyVoterInNotStandalone() { @Test void testRecordRoundTrip() { - VoterSet voterSet = new VoterSet(voterMap(Arrays.asList(1, 2, 3), true)); + VoterSet voterSet = new VoterSet(voterMap(IntStream.of(1, 2, 3), true)); assertEquals(voterSet, VoterSet.fromVotersRecord(voterSet.toVotersRecord((short) 0))); } @Test void testOverlappingMajority() { - Map startingVoterMap = voterMap(Arrays.asList(1, 2, 3), true); + Map startingVoterMap = voterMap(IntStream.of(1, 2, 3), true); VoterSet startingVoterSet = voterSet(startingVoterMap); VoterSet biggerVoterSet = startingVoterSet @@ -172,7 +199,7 @@ void testOverlappingMajority() { @Test void testNonoverlappingMajority() { - Map startingVoterMap = voterMap(Arrays.asList(1, 2, 3, 4, 5), true); + Map startingVoterMap = voterMap(IntStream.of(1, 2, 3, 4, 5), true); VoterSet startingVoterSet = voterSet(startingVoterMap); // Two additions don't have an overlapping majority @@ -217,20 +244,27 @@ private void assertMajorities(boolean overlap, VoterSet a, VoterSet b) { ); } + public static final ListenerName DEFAULT_LISTENER_NAME = ListenerName.normalised("LISTENER"); + public static Map voterMap( - Collection replicas, + IntStream replicas, boolean withDirectoryId ) { return replicas - .stream() + .boxed() .collect( Collectors.toMap( Function.identity(), - id -> VoterSetTest.voterNode(id, withDirectoryId) + id -> voterNode(id, withDirectoryId) ) ); } + public static Map voterMap(Stream replicas) { + return replicas + .collect(Collectors.toMap(ReplicaKey::id, VoterSetTest::voterNode)); + } + public static VoterSet.VoterNode voterNode(int id, boolean withDirectoryId) { return voterNode( ReplicaKey.of( @@ -244,7 +278,7 @@ public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { return new VoterSet.VoterNode( replicaKey, Collections.singletonMap( - "LISTENER", + DEFAULT_LISTENER_NAME, InetSocketAddress.createUnresolved( String.format("replica-%d", replicaKey.id()), 1234 @@ -257,4 +291,8 @@ public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { public static VoterSet voterSet(Map voters) { return new VoterSet(voters); } + + public static VoterSet voterSet(Stream voterKeys) { + return voterSet(voterMap(voterKeys)); + } } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java index 17b7c5d9f3..32a980296b 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java @@ -17,12 +17,11 @@ package org.apache.kafka.snapshot; - import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.HashMap; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.IntStream; import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.SnapshotFooterRecord; import org.apache.kafka.common.message.SnapshotHeaderRecord; @@ -97,7 +96,7 @@ void testBuilderKRaftVersion0WithVoterSet() { OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); int maxBatchSize = 1024; VoterSet voterSet = VoterSetTest.voterSet( - new HashMap<>(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)) + new HashMap<>(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)) ); AtomicReference buffer = new AtomicReference<>(null); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() @@ -117,7 +116,7 @@ void testKBuilderRaftVersion1WithVoterSet() { OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); int maxBatchSize = 1024; VoterSet voterSet = VoterSetTest.voterSet( - new HashMap<>(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)) + new HashMap<>(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)) ); AtomicReference buffer = new AtomicReference<>(null); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() From ec278a6864926cdfd2f4117e53048734f3f392e7 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 4 Jun 2024 21:04:34 +0200 Subject: [PATCH 019/128] MINOR: Fix return tag on Javadocs for consumer group-related Admin methods (#16197) Reviewers: Greg Harris , Chia-Ping Tsai --- .../main/java/org/apache/kafka/clients/admin/Admin.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 d936ec80ff..291250aae9 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 @@ -901,7 +901,7 @@ default DescribeConsumerGroupsResult describeConsumerGroups(Collection g * List the consumer groups available in the cluster. * * @param options The options to use when listing the consumer groups. - * @return The ListGroupsResult. + * @return The ListConsumerGroupsResult. */ ListConsumerGroupsResult listConsumerGroups(ListConsumerGroupsOptions options); @@ -911,7 +911,7 @@ default DescribeConsumerGroupsResult describeConsumerGroups(Collection g * This is a convenience method for {@link #listConsumerGroups(ListConsumerGroupsOptions)} with default options. * See the overload for more details. * - * @return The ListGroupsResult. + * @return The ListConsumerGroupsResult. */ default ListConsumerGroupsResult listConsumerGroups() { return listConsumerGroups(new ListConsumerGroupsOptions()); @@ -921,7 +921,7 @@ default ListConsumerGroupsResult listConsumerGroups() { * List the consumer group offsets available in the cluster. * * @param options The options to use when listing the consumer group offsets. - * @return The ListGroupOffsetsResult + * @return The ListConsumerGroupOffsetsResult */ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) { @SuppressWarnings("deprecation") @@ -939,7 +939,7 @@ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, * This is a convenience method for {@link #listConsumerGroupOffsets(Map, ListConsumerGroupOffsetsOptions)} * to list offsets of all partitions of one group with default options. * - * @return The ListGroupOffsetsResult. + * @return The ListConsumerGroupOffsetsResult. */ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId) { return listConsumerGroupOffsets(groupId, new ListConsumerGroupOffsetsOptions()); From 0e945093a3fccaf3707eee404a221de83c636507 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?TingI=C4=81u=20=22Ting=22=20K=C3=AC?= <51072200+frankvicky@users.noreply.github.com> Date: Wed, 5 Jun 2024 04:21:20 +0800 Subject: [PATCH 020/128] KAFKA-15305 The background thread should try to process the remaining task until the shutdown timer is expired. (#16156) Reviewers: Lianet Magrans , Chia-Ping Tsai --- .../internals/ConsumerNetworkThread.java | 5 ++-- .../internals/NetworkClientDelegate.java | 7 +++++ .../internals/ConsumerNetworkThreadTest.java | 21 ++++++++++++++ .../internals/NetworkClientDelegateTest.java | 29 +++++++++++++++++++ 4 files changed, 60 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index adee659460..7616ac6912 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -293,12 +293,13 @@ private void closeInternal(final Duration timeout) { * Check the unsent queue one last time and poll until all requests are sent or the timer runs out. */ private void sendUnsentRequests(final Timer timer) { - if (networkClientDelegate.unsentRequests().isEmpty()) + if (!networkClientDelegate.hasAnyPendingRequests()) return; + do { networkClientDelegate.poll(timer.remainingMs(), timer.currentTimeMs()); timer.update(); - } while (timer.notExpired() && !networkClientDelegate.unsentRequests().isEmpty()); + } while (timer.notExpired() && networkClientDelegate.hasAnyPendingRequests()); } void cleanup() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 141f5f955c..2cd6f6d853 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -130,6 +130,13 @@ public void poll(final long timeoutMs, final long currentTimeMs) { checkDisconnects(currentTimeMs); } + /** + * Return true if there is at least one in-flight request or unsent request. + */ + public boolean hasAnyPendingRequests() { + return client.hasInFlightRequests() || !unsentRequests.isEmpty(); + } + /** * Tries to send the requests in the unsentRequest queue. If the request doesn't have an assigned node, it will * find the leastLoadedOne, and will be retried in the next {@code poll()}. If the request is expired, a diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java index 8c3f97dd64..b6780fd427 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -339,6 +339,27 @@ void testRunOnceInvokesReaper() { verify(applicationEventReaper).reap(any(Long.class)); } + @Test + void testSendUnsentRequest() { + String groupId = "group-id"; + NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest( + new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id()) + .setKey(groupId)), + Optional.empty()); + + networkClient.add(request); + assertTrue(networkClient.hasAnyPendingRequests()); + assertFalse(networkClient.unsentRequests().isEmpty()); + assertFalse(client.hasInFlightRequests()); + consumerNetworkThread.cleanup(); + + assertTrue(networkClient.unsentRequests().isEmpty()); + assertFalse(client.hasInFlightRequests()); + assertFalse(networkClient.hasAnyPendingRequests()); + } + private void prepareOffsetCommitRequest(final Map expectedOffsets, final Errors error, final boolean disconnected) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java index 70f33bfdf4..7d53ca385b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java @@ -43,6 +43,7 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -140,6 +141,34 @@ public void testEnsureTimerSetOnAdd() { assertEquals(REQUEST_TIMEOUT_MS, ncd.unsentRequests().poll().timer().timeoutMs()); } + @Test + public void testHasAnyPendingRequests() throws Exception { + try (NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate()) { + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); + networkClientDelegate.add(unsentRequest); + + // unsent + assertTrue(networkClientDelegate.hasAnyPendingRequests()); + assertFalse(networkClientDelegate.unsentRequests().isEmpty()); + assertFalse(client.hasInFlightRequests()); + + networkClientDelegate.poll(0, time.milliseconds()); + + // in-flight + assertTrue(networkClientDelegate.hasAnyPendingRequests()); + assertTrue(networkClientDelegate.unsentRequests().isEmpty()); + assertTrue(client.hasInFlightRequests()); + + client.respond(FindCoordinatorResponse.prepareResponse(Errors.NONE, GROUP_ID, mockNode())); + networkClientDelegate.poll(0, time.milliseconds()); + + // get response + assertFalse(networkClientDelegate.hasAnyPendingRequests()); + assertTrue(networkClientDelegate.unsentRequests().isEmpty()); + assertFalse(client.hasInFlightRequests()); + } + } + public NetworkClientDelegate newNetworkClientDelegate() { LogContext logContext = new LogContext(); Properties properties = new Properties(); From bf0ca8498a0c1de98503782b1ce03db53f137020 Mon Sep 17 00:00:00 2001 From: Dongnuo Lyu <139248811+dongnuo123@users.noreply.github.com> Date: Wed, 5 Jun 2024 02:08:38 -0400 Subject: [PATCH 021/128] MINOR: Adjust validateOffsetCommit/Fetch in ConsumerGroup to ensure compatibility with classic protocol members (#16145) During online migration, there could be ConsumerGroup that has members that uses the classic protocol. In the current implementation, `STALE_MEMBER_EPOCH` could be thrown in ConsumerGroup offset fetch/commit validation but it's not supported by the classic protocol. Thus this patch changed `ConsumerGroup#validateOffsetCommit` and `ConsumerGroup#validateOffsetFetch` to ensure compatibility. Reviewers: Jeff Kim , David Jacot --- .../server/OffsetCommitRequestTest.scala | 2 +- .../apache/kafka/coordinator/group/Group.java | 5 +- .../group/OffsetMetadataManager.java | 32 +++----- .../group/classic/ClassicGroup.java | 4 +- .../group/consumer/ConsumerGroup.java | 55 +++++++++++--- .../group/OffsetMetadataManagerTest.java | 75 ++++++++++++------- .../group/classic/ClassicGroupTest.java | 28 ++++--- .../group/consumer/ConsumerGroupTest.java | 69 +++++++++++++++-- 8 files changed, 189 insertions(+), 81 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala index 696e6534bf..afc934ac0b 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala @@ -96,7 +96,7 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator topic = "foo", partition = 0, offset = 100L, - expectedError = Errors.NONE, + expectedError = if (useNewProtocol && version < 9) Errors.UNSUPPORTED_VERSION else Errors.NONE, version = version.toShort ) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java index f0d930648e..acadf6d592 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java @@ -100,12 +100,15 @@ public static GroupType parse(String name) { * @param generationIdOrMemberEpoch The generation id for genetic groups or the member epoch * for consumer groups. * @param isTransactional Whether the offset commit is transactional or not. + * @param apiVersion The api version. */ void validateOffsetCommit( String memberId, String groupInstanceId, int generationIdOrMemberEpoch, - boolean isTransactional + boolean isTransactional, + short apiVersion + ) throws KafkaException; /** diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java index 01c861c038..9e2bc6c62a 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java @@ -325,24 +325,13 @@ private Group validateOffsetCommit( } } - try { - group.validateOffsetCommit( - request.memberId(), - request.groupInstanceId(), - request.generationIdOrMemberEpoch(), - false - ); - } catch (StaleMemberEpochException ex) { - // The STALE_MEMBER_EPOCH error is only returned for new consumer group (KIP-848). When - // it is, the member should be using the OffsetCommit API version >= 9. As we don't - // support upgrading from the old to the new protocol yet, we return UNSUPPORTED_VERSION - // error if an older version is used. We will revise this when the upgrade path is implemented. - if (context.header.apiVersion() >= 9) { - throw ex; - } else { - throw Errors.UNSUPPORTED_VERSION.exception(); - } - } + group.validateOffsetCommit( + request.memberId(), + request.groupInstanceId(), + request.generationIdOrMemberEpoch(), + false, + context.apiVersion() + ); return group; } @@ -350,9 +339,11 @@ private Group validateOffsetCommit( /** * Validates an TxnOffsetCommit request. * + * @param context The request context. * @param request The actual request. */ private Group validateTransactionalOffsetCommit( + RequestContext context, TxnOffsetCommitRequestData request ) throws ApiException { Group group; @@ -375,7 +366,8 @@ private Group validateTransactionalOffsetCommit( request.memberId(), request.groupInstanceId(), request.generationId(), - true + true, + context.apiVersion() ); } catch (StaleMemberEpochException ex) { throw Errors.ILLEGAL_GENERATION.exception(); @@ -530,7 +522,7 @@ public CoordinatorResult commitT RequestContext context, TxnOffsetCommitRequestData request ) throws ApiException { - validateTransactionalOffsetCommit(request); + validateTransactionalOffsetCommit(context, request); final TxnOffsetCommitResponseData response = new TxnOffsetCommitResponseData(); final List records = new ArrayList<>(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java index c9f5bc75ca..dd0d5c15fd 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java @@ -824,13 +824,15 @@ public void validateMember( * @param groupInstanceId The group instance id. * @param generationId The generation id. * @param isTransactional Whether the offset commit is transactional or not. + * @param apiVersion The api version. */ @Override public void validateOffsetCommit( String memberId, String groupInstanceId, int generationId, - boolean isTransactional + boolean isTransactional, + short apiVersion ) throws CoordinatorNotAvailableException, UnknownMemberIdException, IllegalGenerationException, FencedInstanceIdException { if (isInState(DEAD)) { throw Errors.COORDINATOR_NOT_AVAILABLE.exception(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index bbc544289b..98f37a7ed6 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -21,8 +21,10 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.IllegalGenerationException; import org.apache.kafka.common.errors.StaleMemberEpochException; import org.apache.kafka.common.errors.UnknownMemberIdException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.protocol.Errors; @@ -792,21 +794,36 @@ public DeadlineAndEpoch metadataRefreshDeadline() { * @param memberEpoch The member epoch. * @param isTransactional Whether the offset commit is transactional or not. It has no * impact when a consumer group is used. + * @param apiVersion The api version. + * @throws UnknownMemberIdException If the member is not found. + * @throws StaleMemberEpochException If the member uses the consumer protocol and the provided + * member epoch doesn't match the actual member epoch. + * @throws IllegalGenerationException If the member uses the classic protocol and the provided + * generation id is not equal to the member epoch. */ @Override public void validateOffsetCommit( String memberId, String groupInstanceId, int memberEpoch, - boolean isTransactional - ) throws UnknownMemberIdException, StaleMemberEpochException { + boolean isTransactional, + short apiVersion + ) throws UnknownMemberIdException, StaleMemberEpochException, IllegalGenerationException { // When the member epoch is -1, the request comes from either the admin client // or a consumer which does not use the group management facility. In this case, // the request can commit offsets if the group is empty. if (memberEpoch < 0 && members().isEmpty()) return; final ConsumerGroupMember member = getOrMaybeCreateMember(memberId, false); - validateMemberEpoch(memberEpoch, member.memberEpoch()); + + // If the commit is not transactional and the member uses the new consumer protocol (KIP-848), + // the member should be using the OffsetCommit API version >= 9. + if (!isTransactional && !member.useClassicProtocol() && apiVersion < 9) { + throw new UnsupportedVersionException("OffsetCommit version 9 or above must be used " + + "by members using the consumer group protocol"); + } + + validateMemberEpoch(memberEpoch, member.memberEpoch(), member.useClassicProtocol()); } /** @@ -815,13 +832,18 @@ public void validateOffsetCommit( * @param memberId The member id for consumer groups. * @param memberEpoch The member epoch for consumer groups. * @param lastCommittedOffset The last committed offsets in the timeline. + * @throws UnknownMemberIdException If the member is not found. + * @throws StaleMemberEpochException If the member uses the consumer protocol and the provided + * member epoch doesn't match the actual member epoch. + * @throws IllegalGenerationException If the member uses the classic protocol and the provided + * generation id is not equal to the member epoch. */ @Override public void validateOffsetFetch( String memberId, int memberEpoch, long lastCommittedOffset - ) throws UnknownMemberIdException, StaleMemberEpochException { + ) throws UnknownMemberIdException, StaleMemberEpochException, IllegalGenerationException { // When the member id is null and the member epoch is -1, the request either comes // from the admin client or from a client which does not provide them. In this case, // the fetch request is accepted. @@ -832,7 +854,7 @@ public void validateOffsetFetch( throw new UnknownMemberIdException(String.format("Member %s is not a member of group %s.", memberId, groupId)); } - validateMemberEpoch(memberEpoch, member.memberEpoch()); + validateMemberEpoch(memberEpoch, member.memberEpoch(), member.useClassicProtocol()); } /** @@ -896,16 +918,27 @@ public boolean isInStates(Set statesFilter, long committedOffset) { } /** - * Throws a StaleMemberEpochException if the received member epoch does not match - * the expected member epoch. + * Throws an exception if the received member epoch does not match the expected member epoch. + * + * @param receivedMemberEpoch The received member epoch or generation id. + * @param expectedMemberEpoch The expected member epoch. + * @param useClassicProtocol The boolean indicating whether the checked member uses the classic protocol. + * @throws StaleMemberEpochException if the member with unmatched member epoch uses the consumer protocol. + * @throws IllegalGenerationException if the member with unmatched generation id uses the classic protocol. */ private void validateMemberEpoch( int receivedMemberEpoch, - int expectedMemberEpoch - ) throws StaleMemberEpochException { + int expectedMemberEpoch, + boolean useClassicProtocol + ) throws StaleMemberEpochException, IllegalGenerationException { if (receivedMemberEpoch != expectedMemberEpoch) { - throw new StaleMemberEpochException(String.format("The received member epoch %d does not match " - + "the expected member epoch %d.", receivedMemberEpoch, expectedMemberEpoch)); + if (useClassicProtocol) { + throw new IllegalGenerationException(String.format("The received generation id %d does not match " + + "the expected member epoch %d.", receivedMemberEpoch, expectedMemberEpoch)); + } else { + throw new StaleMemberEpochException(String.format("The received member epoch %d does not match " + + "the expected member epoch %d.", receivedMemberEpoch, expectedMemberEpoch)); + } } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java index e2684a7cab..6b328d6680 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.common.errors.RebalanceInProgressException; import org.apache.kafka.common.errors.StaleMemberEpochException; import org.apache.kafka.common.errors.UnknownMemberIdException; -import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; @@ -51,6 +50,7 @@ import org.apache.kafka.coordinator.group.assignor.RangeAssignor; import org.apache.kafka.coordinator.group.consumer.ConsumerGroup; import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; import org.apache.kafka.coordinator.group.classic.ClassicGroup; @@ -1142,14 +1142,8 @@ public void testConsumerGroupOffsetCommitWithStaleMemberEpoch() { assertThrows(StaleMemberEpochException.class, () -> context.commitOffset(request)); } - @ParameterizedTest - @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT) - public void testConsumerGroupOffsetCommitWithVersionSmallerThanVersion9(short version) { - // All the newer versions are fine. - if (version >= 9) return; - // Version 0 does not support MemberId and GenerationIdOrMemberEpoch fields. - if (version == 0) return; - + @Test + public void testConsumerGroupOffsetCommitWithIllegalGenerationId() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. @@ -1162,27 +1156,30 @@ public void testConsumerGroupOffsetCommitWithVersionSmallerThanVersion9(short ve group.updateMember(new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) + .setClassicMemberMetadata(new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata()) .build() ); - // Verify that the request is rejected with the correct exception. - assertThrows(UnsupportedVersionException.class, () -> context.commitOffset( - version, - new OffsetCommitRequestData() - .setGroupId("foo") - .setMemberId("member") - .setGenerationIdOrMemberEpoch(9) - .setTopics(Collections.singletonList( - new OffsetCommitRequestData.OffsetCommitRequestTopic() - .setName("bar") - .setPartitions(Collections.singletonList( - new OffsetCommitRequestData.OffsetCommitRequestPartition() - .setPartitionIndex(0) - .setCommittedOffset(100L) - )) - )) - ) - ); + OffsetCommitRequestData request = new OffsetCommitRequestData() + .setGroupId("foo") + .setMemberId("member") + .setGenerationIdOrMemberEpoch(9) + .setTopics(Collections.singletonList( + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setName("bar") + .setPartitions(Collections.singletonList( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(100L) + )) + )); + + // Verify that a smaller epoch is rejected. + assertThrows(IllegalGenerationException.class, () -> context.commitOffset(request)); + + // Verify that a larger epoch is rejected. + request.setGenerationIdOrMemberEpoch(11); + assertThrows(IllegalGenerationException.class, () -> context.commitOffset(request)); } @Test @@ -2294,6 +2291,30 @@ public void testConsumerGroupOffsetFetchWithStaleMemberEpoch() { () -> context.fetchAllOffsets("group", "member", 10, Long.MAX_VALUE)); } + @Test + public void testConsumerGroupOffsetFetchWithIllegalGenerationId() { + OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); + group.updateMember(new ConsumerGroupMember.Builder("member") + .setClassicMemberMetadata(new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata()) + .build() + ); + + List topics = Collections.singletonList( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("foo") + .setPartitionIndexes(Collections.singletonList(0)) + ); + + // Fetch offsets case. + assertThrows(IllegalGenerationException.class, + () -> context.fetchOffsets("group", "member", 10, topics, Long.MAX_VALUE)); + + // Fetch all offsets case. + assertThrows(IllegalGenerationException.class, + () -> context.fetchAllOffsets("group", "member", 10, Long.MAX_VALUE)); + } + @Test public void testGenericGroupOffsetDelete() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java index 14327ae967..76ab846738 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java @@ -30,10 +30,12 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection; import org.apache.kafka.common.message.JoinGroupResponseData; import org.apache.kafka.common.message.SyncGroupResponseData; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; import org.apache.kafka.coordinator.group.OffsetAndMetadata; import org.apache.kafka.coordinator.group.OffsetExpirationCondition; import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl; @@ -41,6 +43,7 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; import java.util.ArrayList; import java.util.Arrays; @@ -987,10 +990,11 @@ public void testMaybeElectNewJoinedLeaderChooseExisting() { assertTrue(group.isLeader(memberId)); } - @Test - public void testValidateOffsetCommit() { + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT) + public void testValidateOffsetCommit(short version) { // A call from the admin client without any parameters should pass. - group.validateOffsetCommit("", "", -1, false); + group.validateOffsetCommit("", "", -1, false, version); // Add a member. group.add(new ClassicGroupMember( @@ -1012,40 +1016,40 @@ public void testValidateOffsetCommit() { // No parameters and the group is not empty. assertThrows(UnknownMemberIdException.class, - () -> group.validateOffsetCommit("", "", -1, false)); + () -> group.validateOffsetCommit("", "", -1, false, version)); // A transactional offset commit without any parameters // and a non-empty group is accepted. - group.validateOffsetCommit("", null, -1, true); + group.validateOffsetCommit("", null, -1, true, version); // The member id does not exist. assertThrows(UnknownMemberIdException.class, - () -> group.validateOffsetCommit("unknown", "unknown", -1, false)); + () -> group.validateOffsetCommit("unknown", "unknown", -1, false, version)); // The instance id does not exist. assertThrows(UnknownMemberIdException.class, - () -> group.validateOffsetCommit("member-id", "unknown", -1, false)); + () -> group.validateOffsetCommit("member-id", "unknown", -1, false, version)); // The generation id is invalid. assertThrows(IllegalGenerationException.class, - () -> group.validateOffsetCommit("member-id", "instance-id", 0, false)); + () -> group.validateOffsetCommit("member-id", "instance-id", 0, false, version)); // Group is in prepare rebalance state. assertThrows(RebalanceInProgressException.class, - () -> group.validateOffsetCommit("member-id", "instance-id", 1, false)); + () -> group.validateOffsetCommit("member-id", "instance-id", 1, false, version)); // Group transitions to stable. group.transitionTo(STABLE); // This should work. - group.validateOffsetCommit("member-id", "instance-id", 1, false); + group.validateOffsetCommit("member-id", "instance-id", 1, false, version); // Replace static member. group.replaceStaticMember("instance-id", "member-id", "new-member-id"); // The old instance id should be fenced. assertThrows(FencedInstanceIdException.class, - () -> group.validateOffsetCommit("member-id", "instance-id", 1, false)); + () -> group.validateOffsetCommit("member-id", "instance-id", 1, false, version)); // Remove member and transitions to dead. group.remove("new-instance-id"); @@ -1053,7 +1057,7 @@ public void testValidateOffsetCommit() { // This should fail with CoordinatorNotAvailableException. assertThrows(CoordinatorNotAvailableException.class, - () -> group.validateOffsetCommit("member-id", "new-instance-id", 1, false)); + () -> group.validateOffsetCommit("member-id", "new-instance-id", 1, false, version)); } @Test diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index a483fa3be0..a67a5b098b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -20,11 +20,15 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.GroupNotEmptyException; +import org.apache.kafka.common.errors.IllegalGenerationException; import org.apache.kafka.common.errors.StaleMemberEpochException; import org.apache.kafka.common.errors.UnknownMemberIdException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; import org.apache.kafka.coordinator.group.Group; import org.apache.kafka.coordinator.group.MetadataImageBuilder; import org.apache.kafka.coordinator.group.OffsetAndMetadata; @@ -36,7 +40,6 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; import java.util.ArrayList; import java.util.Arrays; @@ -1017,31 +1020,81 @@ public void testMetadataRefreshDeadline() { } @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testValidateOffsetCommit(boolean isTransactional) { + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT) + public void testValidateTransactionalOffsetCommit(short version) { + boolean isTransactional = true; ConsumerGroup group = createConsumerGroup("group-foo"); // Simulate a call from the admin client without member id and member epoch. // This should pass only if the group is empty. - group.validateOffsetCommit("", "", -1, isTransactional); + group.validateOffsetCommit("", "", -1, isTransactional, version); // The member does not exist. assertThrows(UnknownMemberIdException.class, () -> - group.validateOffsetCommit("member-id", null, 0, isTransactional)); + group.validateOffsetCommit("member-id", null, 0, isTransactional, version)); // Create a member. group.updateMember(new ConsumerGroupMember.Builder("member-id").build()); // A call from the admin client should fail as the group is not empty. assertThrows(UnknownMemberIdException.class, () -> - group.validateOffsetCommit("", "", -1, isTransactional)); + group.validateOffsetCommit("", "", -1, isTransactional, version)); // The member epoch is stale. assertThrows(StaleMemberEpochException.class, () -> - group.validateOffsetCommit("member-id", "", 10, isTransactional)); + group.validateOffsetCommit("member-id", "", 10, isTransactional, version)); // This should succeed. - group.validateOffsetCommit("member-id", "", 0, isTransactional); + group.validateOffsetCommit("member-id", "", 0, isTransactional, version); + } + + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT) + public void testValidateOffsetCommit(short version) { + boolean isTransactional = false; + ConsumerGroup group = createConsumerGroup("group-foo"); + + // Simulate a call from the admin client without member id and member epoch. + // This should pass only if the group is empty. + group.validateOffsetCommit("", "", -1, isTransactional, version); + + // The member does not exist. + assertThrows(UnknownMemberIdException.class, () -> + group.validateOffsetCommit("member-id", null, 0, isTransactional, version)); + + // Create members. + group.updateMember( + new ConsumerGroupMember + .Builder("new-protocol-member-id").build() + ); + group.updateMember( + new ConsumerGroupMember.Builder("old-protocol-member-id") + .setClassicMemberMetadata(new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata()) + .build() + ); + + // A call from the admin client should fail as the group is not empty. + assertThrows(UnknownMemberIdException.class, () -> + group.validateOffsetCommit("", "", -1, isTransactional, version)); + + // The member epoch is stale. + if (version >= 9) { + assertThrows(StaleMemberEpochException.class, () -> + group.validateOffsetCommit("new-protocol-member-id", "", 10, isTransactional, version)); + } else { + assertThrows(UnsupportedVersionException.class, () -> + group.validateOffsetCommit("new-protocol-member-id", "", 10, isTransactional, version)); + } + assertThrows(IllegalGenerationException.class, () -> + group.validateOffsetCommit("old-protocol-member-id", "", 10, isTransactional, version)); + + // This should succeed. + if (version >= 9) { + group.validateOffsetCommit("new-protocol-member-id", "", 0, isTransactional, version); + } else { + assertThrows(UnsupportedVersionException.class, () -> + group.validateOffsetCommit("new-protocol-member-id", "", 0, isTransactional, version)); + } } @Test From eabb07bebe0b5f9b534abe1c32a67938eb56ba46 Mon Sep 17 00:00:00 2001 From: "Kuan-Po (Cooper) Tseng" Date: Tue, 4 Jun 2024 21:03:25 +0800 Subject: [PATCH 022/128] KAFKA-16888 Fix failed StorageToolTest.testFormatSucceedsIfAllDirectoriesAreAvailable and StorageToolTest.testFormatEmptyDirectory (#16186) Reviewers: Luke Chen , Chia-Ping Tsai --- core/src/test/scala/unit/kafka/tools/StorageToolTest.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 54a436f231..60ddda5bc3 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -182,7 +182,7 @@ Found problem: val bootstrapMetadata = StorageTool.buildBootstrapMetadata(MetadataVersion.latestTesting(), None, "test format command") assertEquals(0, StorageTool. formatCommand(new PrintStream(stream), Seq(tempDir.toString), metaProperties, bootstrapMetadata, MetadataVersion.latestTesting(), ignoreFormatted = false)) - assertTrue(stream.toString().startsWith("Formatting %s".format(tempDir))) + assertTrue(stream.toString().split("\\r?\\n").exists(_.startsWith("Formatting %s".format(tempDir)))) try assertEquals(1, StorageTool. formatCommand(new PrintStream(new ByteArrayOutputStream()), Seq(tempDir.toString), metaProperties, bootstrapMetadata, MetadataVersion.latestTesting(), ignoreFormatted = false)) catch { @@ -194,7 +194,7 @@ Found problem: val stream2 = new ByteArrayOutputStream() assertEquals(0, StorageTool. formatCommand(new PrintStream(stream2), Seq(tempDir.toString), metaProperties, bootstrapMetadata, MetadataVersion.latestTesting(), ignoreFormatted = true)) - assertEquals("All of the log directories are already formatted.%n".format(), stream2.toString()) + assertEquals(1, stream2.toString().split("\\r?\\n").count(_.startsWith("All of the log directories are already formatted"))) } finally Utils.delete(tempDir) } @@ -215,7 +215,6 @@ Found problem: assertEquals(0, runFormatCommand(stream, availableDirs)) val actual = stream.toString().split("\\r?\\n") val expect = availableDirs.map("Formatting %s".format(_)) - assertEquals(availableDirs.size, actual.size) expect.foreach(dir => { assertEquals(1, actual.count(_.startsWith(dir))) }) From fe7ebf085de9a92ab1542057cd011adf2efc830d Mon Sep 17 00:00:00 2001 From: Abhijeet Kumar Date: Wed, 5 Jun 2024 19:12:25 +0530 Subject: [PATCH 023/128] KAFKA-15265: Integrate RLMQuotaManager for throttling fetches from remote storage (#16071) Reviewers: Kamal Chandraprakash, Luke Chen , Satish Duggana --- .../kafka/log/remote/RemoteLogManager.java | 6 +- .../kafka/log/remote/RemoteLogReader.java | 7 +- .../scala/kafka/server/ReplicaManager.scala | 25 +++++-- .../kafka/log/remote/RemoteLogReaderTest.java | 16 +++- .../kafka/server/ReplicaManagerTest.scala | 73 +++++++++++++++++++ 5 files changed, 117 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index e5bf9597ca..5b0d91ff43 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -249,6 +249,10 @@ RLMQuotaManager createRLMFetchQuotaManager() { "Tracking fetch byte-rate for Remote Log Manager", time); } + public boolean isRemoteLogFetchQuotaExceeded() { + return rlmFetchQuotaManager.isQuotaExceeded(); + } + static RLMQuotaManagerConfig copyQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) { return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerCopyMaxBytesPerSecond(), rlmConfig.remoteLogManagerCopyNumQuotaSamples(), @@ -1660,7 +1664,7 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) throw * @throws java.util.concurrent.RejectedExecutionException if the task cannot be accepted for execution (task queue is full) */ public Future asyncRead(RemoteStorageFetchInfo fetchInfo, Consumer callback) { - return remoteStorageReaderThreadPool.submit(new RemoteLogReader(fetchInfo, this, callback, brokerTopicStats)); + return remoteStorageReaderThreadPool.submit(new RemoteLogReader(fetchInfo, this, callback, brokerTopicStats, rlmFetchQuotaManager)); } void doHandleLeaderOrFollowerPartitions(TopicIdPartition topicPartition, diff --git a/core/src/main/java/kafka/log/remote/RemoteLogReader.java b/core/src/main/java/kafka/log/remote/RemoteLogReader.java index 5d24b2bbbd..9395cbd60e 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogReader.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogReader.java @@ -16,6 +16,7 @@ */ package kafka.log.remote; +import kafka.log.remote.quota.RLMQuotaManager; import kafka.server.BrokerTopicStats; import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.utils.LogContext; @@ -34,17 +35,20 @@ public class RemoteLogReader implements Callable { private final RemoteLogManager rlm; private final BrokerTopicStats brokerTopicStats; private final Consumer callback; + private final RLMQuotaManager quotaManager; public RemoteLogReader(RemoteStorageFetchInfo fetchInfo, RemoteLogManager rlm, Consumer callback, - BrokerTopicStats brokerTopicStats) { + BrokerTopicStats brokerTopicStats, + RLMQuotaManager quotaManager) { this.fetchInfo = fetchInfo; this.rlm = rlm; this.brokerTopicStats = brokerTopicStats; this.callback = callback; this.brokerTopicStats.topicStats(fetchInfo.topicPartition.topic()).remoteFetchRequestRate().mark(); this.brokerTopicStats.allTopicsStats().remoteFetchRequestRate().mark(); + this.quotaManager = quotaManager; logger = new LogContext() { @Override public String logPrefix() { @@ -73,6 +77,7 @@ public Void call() { } logger.debug("Finished reading records from remote storage for topic partition {}", fetchInfo.topicPartition); + quotaManager.record(result.fetchDataInfo.map(fetchDataInfo -> fetchDataInfo.records.sizeInBytes()).orElse(0)); callback.accept(result); return null; diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index a2a070bcd0..6908ec096d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1757,12 +1757,25 @@ class ReplicaManager(val config: KafkaConfig, createLogReadResult(highWatermark, leaderLogStartOffset, leaderLogEndOffset, new OffsetMovedToTieredStorageException("Given offset" + offset + " is moved to tiered storage")) } else { - // For consume fetch requests, create a dummy FetchDataInfo with the remote storage fetch information. - // For the first topic-partition that needs remote data, we will use this information to read the data in another thread. - val fetchDataInfo = - new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false, Optional.empty(), - Optional.of(new RemoteStorageFetchInfo(adjustedMaxBytes, minOneMessage, tp.topicPartition(), - fetchInfo, params.isolation, params.hardMaxBytesLimit()))) + val fetchDataInfo = if (remoteLogManager.get.isRemoteLogFetchQuotaExceeded) { + // We do not want to send an exception in a LogReadResult response (like we do in other cases when we send + // UnknownOffsetMetadata), because it is classified as an error in reading the data, and a response is + // immediately sent back to the client. Instead, we want to serve data for the other topic partitions of the + // fetch request via delayed fetch if required (when sending immediate response, we skip delayed fetch). + new FetchDataInfo( + LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, + MemoryRecords.EMPTY, + false, + Optional.empty(), + Optional.empty() + ) + } else { + // For consume fetch requests, create a dummy FetchDataInfo with the remote storage fetch information. + // For the first topic-partition that needs remote data, we will use this information to read the data in another thread. + new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false, Optional.empty(), + Optional.of(new RemoteStorageFetchInfo(adjustedMaxBytes, minOneMessage, tp.topicPartition(), + fetchInfo, params.isolation, params.hardMaxBytesLimit()))) + } LogReadResult(fetchDataInfo, divergingEpoch = None, diff --git a/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java b/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java index bff58364b9..8b1e1bd32a 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java @@ -16,6 +16,7 @@ */ package kafka.log.remote; +import kafka.log.remote.quota.RLMQuotaManager; import kafka.server.BrokerTopicStats; import kafka.utils.TestUtils; import org.apache.kafka.common.TopicPartition; @@ -45,6 +46,7 @@ public class RemoteLogReaderTest { public static final String TOPIC = "test"; RemoteLogManager mockRLM = mock(RemoteLogManager.class); BrokerTopicStats brokerTopicStats = null; + RLMQuotaManager mockQuotaManager = mock(RLMQuotaManager.class); LogOffsetMetadata logOffsetMetadata = new LogOffsetMetadata(100); Records records = mock(Records.class); @@ -62,7 +64,7 @@ public void testRemoteLogReaderWithoutError() throws RemoteStorageException, IOE Consumer callback = mock(Consumer.class); RemoteStorageFetchInfo remoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, new TopicPartition(TOPIC, 0), null, null, false); - RemoteLogReader remoteLogReader = new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback, brokerTopicStats); + RemoteLogReader remoteLogReader = new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback, brokerTopicStats, mockQuotaManager); remoteLogReader.call(); // verify the callback did get invoked with the expected remoteLogReadResult @@ -73,6 +75,11 @@ public void testRemoteLogReaderWithoutError() throws RemoteStorageException, IOE assertTrue(actualRemoteLogReadResult.fetchDataInfo.isPresent()); assertEquals(fetchDataInfo, actualRemoteLogReadResult.fetchDataInfo.get()); + // verify the record method on quota manager was called with the expected value + ArgumentCaptor recordedArg = ArgumentCaptor.forClass(Double.class); + verify(mockQuotaManager, times(1)).record(recordedArg.capture()); + assertEquals(100, recordedArg.getValue()); + // Verify metrics for remote reads are updated correctly assertEquals(1, brokerTopicStats.topicStats(TOPIC).remoteFetchRequestRate().count()); assertEquals(100, brokerTopicStats.topicStats(TOPIC).remoteFetchBytesRate().count()); @@ -89,7 +96,7 @@ public void testRemoteLogReaderWithError() throws RemoteStorageException, IOExce Consumer callback = mock(Consumer.class); RemoteStorageFetchInfo remoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, new TopicPartition(TOPIC, 0), null, null, false); - RemoteLogReader remoteLogReader = new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback, brokerTopicStats); + RemoteLogReader remoteLogReader = new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback, brokerTopicStats, mockQuotaManager); remoteLogReader.call(); // verify the callback did get invoked with the expected remoteLogReadResult @@ -99,6 +106,11 @@ public void testRemoteLogReaderWithError() throws RemoteStorageException, IOExce assertTrue(actualRemoteLogReadResult.error.isPresent()); assertFalse(actualRemoteLogReadResult.fetchDataInfo.isPresent()); + // verify the record method on quota manager was called with the expected value + ArgumentCaptor recordedArg = ArgumentCaptor.forClass(Double.class); + verify(mockQuotaManager, times(1)).record(recordedArg.capture()); + assertEquals(0, recordedArg.getValue()); + // Verify metrics for remote reads are updated correctly assertEquals(1, brokerTopicStats.topicStats(TOPIC).remoteFetchRequestRate().count()); assertEquals(0, brokerTopicStats.topicStats(TOPIC).remoteFetchBytesRate().count()); diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 0c055da2a9..9f820155c8 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -6703,6 +6703,79 @@ class ReplicaManagerTest { )) } } + + @Test + def testRemoteReadQuotaExceeded(): Unit = { + when(mockRemoteLogManager.isRemoteLogFetchQuotaExceeded).thenReturn(true) + + val tp0 = new TopicPartition(topic, 0) + val tpId0 = new TopicIdPartition(topicId, tp0) + val fetch: Seq[(TopicIdPartition, LogReadResult)] = readFromLogWithOffsetOutOfRange(tp0) + + assertEquals(1, fetch.size) + assertEquals(tpId0, fetch.head._1) + val fetchInfo = fetch.head._2.info + assertEquals(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, fetchInfo.fetchOffsetMetadata) + assertFalse(fetchInfo.records.records().iterator().hasNext) + assertFalse(fetchInfo.firstEntryIncomplete) + assertFalse(fetchInfo.abortedTransactions.isPresent) + assertFalse(fetchInfo.delayedRemoteStorageFetch.isPresent) + } + + @Test + def testRemoteReadQuotaNotExceeded(): Unit = { + when(mockRemoteLogManager.isRemoteLogFetchQuotaExceeded).thenReturn(false) + + val tp0 = new TopicPartition(topic, 0) + val tpId0 = new TopicIdPartition(topicId, tp0) + val fetch: Seq[(TopicIdPartition, LogReadResult)] = readFromLogWithOffsetOutOfRange(tp0) + + assertEquals(1, fetch.size) + assertEquals(tpId0, fetch.head._1) + val fetchInfo = fetch.head._2.info + assertEquals(1L, fetchInfo.fetchOffsetMetadata.messageOffset) + assertEquals(UnifiedLog.UnknownOffset, fetchInfo.fetchOffsetMetadata.segmentBaseOffset) + assertEquals(-1, fetchInfo.fetchOffsetMetadata.relativePositionInSegment) + assertEquals(MemoryRecords.EMPTY, fetchInfo.records) + assertTrue(fetchInfo.delayedRemoteStorageFetch.isPresent) + } + + private def readFromLogWithOffsetOutOfRange(tp: TopicPartition): Seq[(TopicIdPartition, LogReadResult)] = { + val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), aliveBrokerIds = Seq(0, 1, 2), enableRemoteStorage = true, shouldMockLog = true) + try { + val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints) + replicaManager.createPartition(tp).createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints = offsetCheckpoints, None) + val partition0Replicas = Seq[Integer](0, 1).asJava + val topicIds = Map(tp.topic -> topicId).asJava + val leaderEpoch = 0 + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + Seq( + new LeaderAndIsrPartitionState() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(partition0Replicas) + .setPartitionEpoch(0) + .setReplicas(partition0Replicas) + .setIsNew(true) + ).asJava, + topicIds, + Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() + replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) + + val params = new FetchParams(ApiKeys.FETCH.latestVersion, -1, 1, 1000, 0, 100, FetchIsolation.HIGH_WATERMARK, None.asJava) + replicaManager.readFromLog( + params, + Seq(new TopicIdPartition(topicId, 0, topic) -> new PartitionData(topicId, 1, 0, 100000, Optional.of[Integer](leaderEpoch), Optional.of(leaderEpoch))), + UnboundedQuota, + readFromPurgatory = false) + } finally { + replicaManager.shutdown(checkpointHW = false) + } + } + // Some threads are closed, but the state didn't reflect in the JVM immediately, so add some wait time for it private def assertNoNonDaemonThreadsWithWaiting(threadNamePrefix: String, waitTimeMs: Long = 500L): Unit = { var nonDemonThreads: mutable.Set[Thread] = mutable.Set.empty[Thread] From c1accdbab9e0e71959b5a0a5959c8341a512b426 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Wed, 5 Jun 2024 11:35:32 -0700 Subject: [PATCH 024/128] KAFKA-16858: Throw DataException from validateValue on array and map schemas without inner schemas (#16161) Signed-off-by: Greg Harris Reviewers: Chris Egerton --- .../kafka/connect/data/ConnectSchema.java | 36 +++-- .../kafka/connect/data/ConnectSchemaTest.java | 140 ++++++++++++++++++ .../apache/kafka/connect/data/StructTest.java | 33 ----- 3 files changed, 167 insertions(+), 42 deletions(-) diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java index 2a6e71c381..cf5f01502c 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java @@ -213,11 +213,15 @@ public static void validateValue(Schema schema, Object value) { validateValue(null, schema, value); } - public static void validateValue(String name, Schema schema, Object value) { + public static void validateValue(String field, Schema schema, Object value) { + validateValue(schema, value, field == null ? "value" : "field: \"" + field + "\""); + } + + private static void validateValue(Schema schema, Object value, String location) { if (value == null) { if (!schema.isOptional()) - throw new DataException("Invalid value: null used for required field: \"" + name - + "\", schema type: " + schema.type()); + throw new DataException("Invalid value: null used for required " + location + + ", schema type: " + schema.type()); return; } @@ -236,8 +240,8 @@ public static void validateValue(String name, Schema schema, Object value) { exceptionMessage.append(" \"").append(schema.name()).append("\""); } exceptionMessage.append(" with type ").append(schema.type()).append(": ").append(value.getClass()); - if (name != null) { - exceptionMessage.append(" for field: \"").append(name).append("\""); + if (location != null) { + exceptionMessage.append(" for ").append(location); } throw new DataException(exceptionMessage.toString()); } @@ -251,19 +255,33 @@ public static void validateValue(String name, Schema schema, Object value) { break; case ARRAY: List array = (List) value; - for (Object entry : array) - validateValue(schema.valueSchema(), entry); + String entryLocation = "element of array " + location; + Schema arrayValueSchema = assertSchemaNotNull(schema.valueSchema(), entryLocation); + for (Object entry : array) { + validateValue(arrayValueSchema, entry, entryLocation); + } break; case MAP: Map map = (Map) value; + String keyLocation = "key of map " + location; + String valueLocation = "value of map " + location; + Schema mapKeySchema = assertSchemaNotNull(schema.keySchema(), keyLocation); + Schema mapValueSchema = assertSchemaNotNull(schema.valueSchema(), valueLocation); for (Map.Entry entry : map.entrySet()) { - validateValue(schema.keySchema(), entry.getKey()); - validateValue(schema.valueSchema(), entry.getValue()); + validateValue(mapKeySchema, entry.getKey(), keyLocation); + validateValue(mapValueSchema, entry.getValue(), valueLocation); } break; } } + private static Schema assertSchemaNotNull(Schema schema, String location) { + if (schema == null) { + throw new DataException("No schema defined for " + location); + } + return schema; + } + private static List> expectedClassesFor(Schema schema) { List> expectedClasses = LOGICAL_TYPE_CLASSES.get(schema.name()); if (expectedClasses == null) diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java index 25e6db3469..43c2342fe3 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java @@ -330,4 +330,144 @@ public void testEmptyStruct() { new Struct(emptyStruct); } + private void assertInvalidValueForSchema(String fieldName, Schema schema, Object value, String message) { + Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, schema, value)); + assertEquals(message, e.getMessage()); + } + + @Test + public void testValidateFieldWithInvalidValueType() { + String fieldName = "field"; + assertInvalidValueForSchema(fieldName, new FakeSchema(), new Object(), + "Invalid Java object for schema \"fake\" with type null: class java.lang.Object for field: \"field\""); + assertInvalidValueForSchema(null, Schema.INT8_SCHEMA, new Object(), + "Invalid Java object for schema with type INT8: class java.lang.Object for value"); + assertInvalidValueForSchema(fieldName, Schema.INT8_SCHEMA, new Object(), + "Invalid Java object for schema with type INT8: class java.lang.Object for field: \"field\""); + } + + @Test + public void testValidateFieldWithInvalidValueMismatchTimestamp() { + long longValue = 1000L; + String fieldName = "field"; + + ConnectSchema.validateValue(fieldName, Schema.INT64_SCHEMA, longValue); + + assertInvalidValueForSchema(fieldName, Timestamp.SCHEMA, longValue, + "Invalid Java object for schema \"org.apache.kafka.connect.data.Timestamp\" " + + "with type INT64: class java.lang.Long for field: \"field\""); + } + + @Test + public void testValidateList() { + String fieldName = "field"; + + // Optional element schema + Schema optionalStrings = SchemaBuilder.array(Schema.OPTIONAL_STRING_SCHEMA); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.emptyList()); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonList("hello")); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonList(null)); + ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList("hello", "world")); + ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList("hello", null)); + ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList(null, "world")); + assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonList(true), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for element of array field: \"field\""); + + // Required element schema + Schema requiredStrings = SchemaBuilder.array(Schema.STRING_SCHEMA); + ConnectSchema.validateValue(fieldName, requiredStrings, Collections.emptyList()); + ConnectSchema.validateValue(fieldName, requiredStrings, Collections.singletonList("hello")); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonList(null), + "Invalid value: null used for required element of array field: \"field\", schema type: STRING"); + ConnectSchema.validateValue(fieldName, requiredStrings, Arrays.asList("hello", "world")); + assertInvalidValueForSchema(fieldName, requiredStrings, Arrays.asList("hello", null), + "Invalid value: null used for required element of array field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, requiredStrings, Arrays.asList(null, "world"), + "Invalid value: null used for required element of array field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonList(true), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for element of array field: \"field\""); + + // Null element schema + Schema nullElements = SchemaBuilder.type(Schema.Type.ARRAY); + assertInvalidValueForSchema(fieldName, nullElements, Collections.emptyList(), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList("hello"), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList(null), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList("hello", "world"), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList("hello", null), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList(null, "world"), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList(true), + "No schema defined for element of array field: \"field\""); + } + + @Test + public void testValidateMap() { + String fieldName = "field"; + + // Optional element schema + Schema optionalStrings = SchemaBuilder.map(Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.emptyMap()); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap("key", "value")); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap("key", null)); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap(null, "value")); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap(null, null)); + assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonMap("key", true), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonMap(true, "value"), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for key of map field: \"field\""); + + // Required element schema + Schema requiredStrings = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA); + ConnectSchema.validateValue(fieldName, requiredStrings, Collections.emptyMap()); + ConnectSchema.validateValue(fieldName, requiredStrings, Collections.singletonMap("key", "value")); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap("key", null), + "Invalid value: null used for required value of map field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(null, "value"), + "Invalid value: null used for required key of map field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(null, null), + "Invalid value: null used for required key of map field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap("key", true), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(true, "value"), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for key of map field: \"field\""); + + // Null key schema + Schema nullKeys = SchemaBuilder.type(Schema.Type.MAP); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.emptyMap(), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", "value"), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", null), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(null, "value"), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(null, null), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", true), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(true, "value"), + "No schema defined for key of map field: \"field\""); + + // Null value schema + Schema nullValues = SchemaBuilder.mapWithNullValues(Schema.OPTIONAL_STRING_SCHEMA); + assertInvalidValueForSchema(fieldName, nullValues, Collections.emptyMap(), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", "value"), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", null), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(null, "value"), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(null, null), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", true), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(true, "value"), + "No schema defined for value of map field: \"field\""); + } } diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java index 55ccc81bed..6dee26ca83 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java @@ -304,39 +304,6 @@ public void testValidateStructWithNullValue() { e.getMessage()); } - @Test - public void testValidateFieldWithInvalidValueType() { - String fieldName = "field"; - FakeSchema fakeSchema = new FakeSchema(); - - Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, - fakeSchema, new Object())); - assertEquals("Invalid Java object for schema \"fake\" with type null: class java.lang.Object for field: \"field\"", - e.getMessage()); - - e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, - Schema.INT8_SCHEMA, new Object())); - assertEquals("Invalid Java object for schema with type INT8: class java.lang.Object for field: \"field\"", - e.getMessage()); - - e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(Schema.INT8_SCHEMA, new Object())); - assertEquals("Invalid Java object for schema with type INT8: class java.lang.Object", e.getMessage()); - } - - @Test - public void testValidateFieldWithInvalidValueMismatchTimestamp() { - String fieldName = "field"; - long longValue = 1000L; - - // Does not throw - ConnectSchema.validateValue(fieldName, Schema.INT64_SCHEMA, longValue); - - Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, - Timestamp.SCHEMA, longValue)); - assertEquals("Invalid Java object for schema \"org.apache.kafka.connect.data.Timestamp\" " + - "with type INT64: class java.lang.Long for field: \"field\"", e.getMessage()); - } - @Test public void testPutNullField() { final String fieldName = "fieldName"; From dd6d6f4a5aa4f5109690d08d41509ded057b780d Mon Sep 17 00:00:00 2001 From: Ayoub Omari Date: Thu, 6 Jun 2024 00:05:04 +0200 Subject: [PATCH 025/128] KAFKA-16573: Specify node and store where serdes are needed (#15790) Reviewers: Matthias J. Sax , Bruno Cadonna , Anna Sophie Blee-Goldman --- .../apache/kafka/streams/StreamsConfig.java | 2 +- .../internals/WrappingNullableUtils.java | 13 +- .../streams/processor/internals/SinkNode.java | 15 +- .../processor/internals/SourceNode.java | 15 +- .../state/internals/MeteredKeyValueStore.java | 15 +- .../state/internals/MeteredSessionStore.java | 15 +- .../MeteredTimestampedKeyValueStore.java | 2 +- .../MeteredVersionedKeyValueStore.java | 18 +- .../state/internals/MeteredWindowStore.java | 13 +- .../internals/StoreSerdeInitializer.java | 76 +++++++++ .../internals/ProcessorNodeTest.java | 8 +- .../processor/internals/SinkNodeTest.java | 68 +++++++- .../processor/internals/SourceNodeTest.java | 83 ++++++++- .../internals/StoreSerdeInitializerTest.java | 157 ++++++++++++++++++ 14 files changed, 433 insertions(+), 67 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializer.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializerTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 25e86928b0..025a8a5b39 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1875,7 +1875,7 @@ public KafkaClientSupplier getKafkaClientSupplier() { @SuppressWarnings("WeakerAccess") public Serde defaultKeySerde() { final Object keySerdeConfigSetting = get(DEFAULT_KEY_SERDE_CLASS_CONFIG); - if (keySerdeConfigSetting == null) { + if (keySerdeConfigSetting == null) { throw new ConfigException("Please specify a key serde or set one through StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG"); } try { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WrappingNullableUtils.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WrappingNullableUtils.java index b904608c3d..5dff888e62 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WrappingNullableUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WrappingNullableUtils.java @@ -33,22 +33,19 @@ private static Deserializer prepareDeserializer(final Deserializer spe final Deserializer deserializerToUse; if (specificDeserializer == null) { - final Deserializer contextKeyDeserializer = context.keySerde().deserializer(); - final Deserializer contextValueDeserializer = context.valueSerde().deserializer(); - deserializerToUse = (Deserializer) (isKey ? contextKeyDeserializer : contextValueDeserializer); + deserializerToUse = (Deserializer) (isKey ? context.keySerde().deserializer() : context.valueSerde().deserializer()); } else { deserializerToUse = specificDeserializer; initNullableDeserializer(deserializerToUse, new SerdeGetter(context)); } return deserializerToUse; } + @SuppressWarnings("unchecked") private static Serializer prepareSerializer(final Serializer specificSerializer, final ProcessorContext context, final boolean isKey, final String name) { final Serializer serializerToUse; if (specificSerializer == null) { - final Serializer contextKeySerializer = context.keySerde().serializer(); - final Serializer contextValueSerializer = context.valueSerde().serializer(); - serializerToUse = (Serializer) (isKey ? contextKeySerializer : contextValueSerializer); + serializerToUse = (Serializer) (isKey ? context.keySerde().serializer() : context.valueSerde().serializer()); } else { serializerToUse = specificSerializer; initNullableSerializer(serializerToUse, new SerdeGetter(context)); @@ -60,7 +57,7 @@ private static Serializer prepareSerializer(final Serializer specificS private static Serde prepareSerde(final Serde specificSerde, final SerdeGetter getter, final boolean isKey) { final Serde serdeToUse; if (specificSerde == null) { - serdeToUse = (Serde) (isKey ? getter.keySerde() : getter.valueSerde()); + serdeToUse = (Serde) (isKey ? getter.keySerde() : getter.valueSerde()); } else { serdeToUse = specificSerde; } @@ -93,12 +90,14 @@ public static Serde prepareKeySerde(final Serde specificSerde, final S public static Serde prepareValueSerde(final Serde specificSerde, final SerdeGetter getter) { return prepareSerde(specificSerde, getter, false); } + @SuppressWarnings({"rawtypes", "unchecked"}) public static void initNullableSerializer(final Serializer specificSerializer, final SerdeGetter getter) { if (specificSerializer instanceof WrappingNullableSerializer) { ((WrappingNullableSerializer) specificSerializer).setIfUnset(getter); } } + @SuppressWarnings({"rawtypes", "unchecked"}) public static void initNullableDeserializer(final Deserializer specificDeserializer, final SerdeGetter getter) { if (specificDeserializer instanceof WrappingNullableDeserializer) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index 6f508eff27..6e79616d30 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.api.Record; @@ -58,8 +60,17 @@ public void addChild(final ProcessorNode child) { public void init(final InternalProcessorContext context) { super.init(context); this.context = context; - keySerializer = prepareKeySerializer(keySerializer, context, this.name()); - valSerializer = prepareValueSerializer(valSerializer, context, this.name()); + try { + keySerializer = prepareKeySerializer(keySerializer, context, this.name()); + } catch (ConfigException | StreamsException e) { + throw new StreamsException(String.format("Failed to initialize key serdes for sink node %s", name()), e, context.taskId()); + } + + try { + valSerializer = prepareValueSerializer(valSerializer, context, this.name()); + } catch (final ConfigException | StreamsException e) { + throw new StreamsException(String.format("Failed to initialize value serdes for sink node %s", name()), e, context.taskId()); + } } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index 5d0c04b96a..2f53840acc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics; @@ -74,8 +76,17 @@ public void init(final InternalProcessorContext context) { super.init(context); this.context = context; - keyDeserializer = prepareKeyDeserializer(keyDeserializer, context, name()); - valDeserializer = prepareValueDeserializer(valDeserializer, context, name()); + try { + keyDeserializer = prepareKeyDeserializer(keyDeserializer, context, name()); + } catch (final ConfigException | StreamsException e) { + throw new StreamsException(String.format("Failed to initialize key serdes for source node %s", name()), e, context.taskId()); + } + + try { + valDeserializer = prepareValueDeserializer(valDeserializer, context, name()); + } catch (final ConfigException | StreamsException e) { + throw new StreamsException(String.format("Failed to initialize value serdes for source node %s", name()), e, context.taskId()); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index fbe42b8706..f828c50287 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -60,7 +60,6 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerde; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; import static org.apache.kafka.streams.state.internals.StoreQueryUtils.getDeserializeValue; @@ -187,21 +186,15 @@ protected Serde prepareValueSerdeForStore(final Serde valueSerde, final Se protected void initStoreSerde(final ProcessorContext context) { final String storeName = name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); - serdes = new StateSerdes<>( - changelogTopic, - prepareKeySerde(keySerde, new SerdeGetter(context)), - prepareValueSerdeForStore(valueSerde, new SerdeGetter(context)) - ); + serdes = StoreSerdeInitializer.prepareStoreSerde( + context, storeName, changelogTopic, keySerde, valueSerde, this::prepareValueSerdeForStore); } protected void initStoreSerde(final StateStoreContext context) { final String storeName = name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); - serdes = new StateSerdes<>( - changelogTopic, - prepareKeySerde(keySerde, new SerdeGetter(context)), - prepareValueSerdeForStore(valueSerde, new SerdeGetter(context)) - ); + serdes = StoreSerdeInitializer.prepareStoreSerde( + context, storeName, changelogTopic, keySerde, valueSerde, this::prepareValueSerdeForStore); } @SuppressWarnings("unchecked") diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 731bc3145c..233c2c00b8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -30,7 +30,6 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorContextUtils; -import org.apache.kafka.streams.processor.internals.SerdeGetter; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.query.FailureReason; import org.apache.kafka.streams.query.PositionBound; @@ -151,21 +150,15 @@ private void registerMetrics() { private void initStoreSerde(final ProcessorContext context) { final String storeName = name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); - serdes = new StateSerdes<>( - changelogTopic, - WrappingNullableUtils.prepareKeySerde(keySerde, new SerdeGetter(context)), - WrappingNullableUtils.prepareValueSerde(valueSerde, new SerdeGetter(context)) - ); + serdes = StoreSerdeInitializer.prepareStoreSerde( + context, storeName, changelogTopic, keySerde, valueSerde, WrappingNullableUtils::prepareValueSerde); } private void initStoreSerde(final StateStoreContext context) { final String storeName = name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); - serdes = new StateSerdes<>( - changelogTopic, - WrappingNullableUtils.prepareKeySerde(keySerde, new SerdeGetter(context)), - WrappingNullableUtils.prepareValueSerde(valueSerde, new SerdeGetter(context)) - ); + serdes = StoreSerdeInitializer.prepareStoreSerde( + context, storeName, changelogTopic, keySerde, valueSerde, WrappingNullableUtils::prepareValueSerde); } @SuppressWarnings("unchecked") diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java index 0b4702b9db..61e6533fb8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java @@ -86,6 +86,7 @@ public class MeteredTimestampedKeyValueStore (query, positionBound, config, store) -> runTimestampedKeyQuery(query, positionBound, config) ) ); + @SuppressWarnings("unchecked") @Override protected Serde> prepareValueSerdeForStore(final Serde> valueSerde, final SerdeGetter getter) { @@ -96,7 +97,6 @@ protected Serde> prepareValueSerdeForStore(final Serde getWithBinary(final K key) { try { return maybeMeasureLatency(() -> { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java index af3b6b77dc..3a12ca59f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java @@ -18,18 +18,18 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerde; -import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareValueSerde; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; import java.time.Instant; import java.util.Map; import java.util.Objects; + import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; @@ -303,11 +303,8 @@ protected void initStoreSerde(final ProcessorContext context) { // additionally init raw value serde final String storeName = super.name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); - plainValueSerdes = new StateSerdes<>( - changelogTopic, - prepareKeySerde(keySerde, new SerdeGetter(context)), - prepareValueSerde(plainValueSerde, new SerdeGetter(context)) - ); + plainValueSerdes = StoreSerdeInitializer.prepareStoreSerde( + context, storeName, changelogTopic, keySerde, plainValueSerde, WrappingNullableUtils::prepareValueSerde); } @Override @@ -317,11 +314,8 @@ protected void initStoreSerde(final StateStoreContext context) { // additionally init raw value serde final String storeName = super.name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); - plainValueSerdes = new StateSerdes<>( - changelogTopic, - prepareKeySerde(keySerde, new SerdeGetter(context)), - prepareValueSerde(plainValueSerde, new SerdeGetter(context)) - ); + plainValueSerdes = StoreSerdeInitializer.prepareStoreSerde( + context, storeName, changelogTopic, keySerde, plainValueSerde, WrappingNullableUtils::prepareValueSerde); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index a62e8c4756..bf6f749977 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -56,7 +56,6 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerde; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; import static org.apache.kafka.streams.state.internals.StoreQueryUtils.getDeserializeValue; @@ -170,19 +169,15 @@ private void registerMetrics() { private void initStoreSerde(final ProcessorContext context) { final String storeName = name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); - serdes = new StateSerdes<>( - changelogTopic, - prepareKeySerde(keySerde, new SerdeGetter(context)), - prepareValueSerde(valueSerde, new SerdeGetter(context))); + serdes = StoreSerdeInitializer.prepareStoreSerde( + context, storeName, changelogTopic, keySerde, valueSerde, this::prepareValueSerde); } private void initStoreSerde(final StateStoreContext context) { final String storeName = name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); - serdes = new StateSerdes<>( - changelogTopic, - prepareKeySerde(keySerde, new SerdeGetter(context)), - prepareValueSerde(valueSerde, new SerdeGetter(context))); + serdes = StoreSerdeInitializer.prepareStoreSerde( + context, storeName, changelogTopic, keySerde, valueSerde, this::prepareValueSerde); } @SuppressWarnings("unchecked") diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializer.java new file mode 100644 index 0000000000..1a9aa02f3c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializer.java @@ -0,0 +1,76 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.state.StateSerdes; + + +public class StoreSerdeInitializer { + static StateSerdes prepareStoreSerde(final StateStoreContext context, + final String storeName, + final String changelogTopic, + final Serde keySerde, + final Serde valueSerde, + final PrepareFunc prepareValueSerdeFunc) { + return new StateSerdes<>( + changelogTopic, + prepareSerde(WrappingNullableUtils::prepareKeySerde, storeName, keySerde, new SerdeGetter(context), true, context.taskId()), + prepareSerde(prepareValueSerdeFunc, storeName, valueSerde, new SerdeGetter(context), false, context.taskId()) + ); + } + + static StateSerdes prepareStoreSerde(final ProcessorContext context, + final String storeName, + final String changelogTopic, + final Serde keySerde, + final Serde valueSerde, + final PrepareFunc prepareValueSerdeFunc) { + return new StateSerdes<>( + changelogTopic, + prepareSerde(WrappingNullableUtils::prepareKeySerde, storeName, keySerde, new SerdeGetter(context), true, context.taskId()), + prepareSerde(prepareValueSerdeFunc, storeName, valueSerde, new SerdeGetter(context), false, context.taskId()) + ); + } + + private static Serde prepareSerde(final PrepareFunc prepare, + final String storeName, + final Serde serde, + final SerdeGetter getter, + final Boolean isKey, + final TaskId taskId) { + + final String serdeType = isKey ? "key" : "value"; + try { + return prepare.prepareSerde(serde, getter); + } catch (final ConfigException | StreamsException e) { + throw new StreamsException(String.format("Failed to initialize %s serdes for store %s", serdeType, storeName), e, taskId); + } + } +} + +interface PrepareFunc { + Serde prepareSerde(Serde serde, SerdeGetter getter); +} + diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index dfa3f9e422..fdd5214385 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -154,10 +153,9 @@ public void testTopologyLevelConfigException() { .flatMapValues(value -> Collections.singletonList("")); final Topology topology = builder.build(); - final ConfigException se = assertThrows(ConfigException.class, () -> new TopologyTestDriver(topology)); - final String msg = se.getMessage(); - assertTrue("Error about class cast with serdes", msg.contains("StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG")); - assertTrue("Error about class cast with serdes", msg.contains("specify a key serde")); + final StreamsException se = assertThrows(StreamsException.class, () -> new TopologyTestDriver(topology)); + assertThat(se.getMessage(), containsString("Failed to initialize key serdes for source node")); + assertThat(se.getCause().getMessage(), containsString("Please specify a key serde or set one through StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG")); } private static class ClassCastProcessor extends ExceptionalProcessor { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java index 7e7f7b824b..805f2fd5db 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java @@ -16,18 +16,27 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.test.InternalMockProcessorContext; import org.apache.kafka.test.MockRecordCollector; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; public class SinkNodeTest { private final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class); @@ -40,14 +49,21 @@ public class SinkNodeTest { // Used to verify that the correct exceptions are thrown if the compiler checks are bypassed @SuppressWarnings({"unchecked", "rawtypes"}) private final SinkNode illTypedSink = (SinkNode) sink; + private MockedStatic utilsMock; - @Before - public void before() { - sink.init(context); + @BeforeEach + public void setup() { + utilsMock = Mockito.mockStatic(WrappingNullableUtils.class); + } + + @AfterEach + public void cleanup() { + utilsMock.close(); } @Test public void shouldThrowStreamsExceptionOnInputRecordWithInvalidTimestamp() { + sink.init(context); // When/Then context.setTime(-1); // ensures a negative timestamp is set for the record we send next try { @@ -58,4 +74,46 @@ public void shouldThrowStreamsExceptionOnInputRecordWithInvalidTimestamp() { } } + @Test + public void shouldThrowStreamsExceptionOnUndefinedKeySerde() { + utilsMock.when(() -> WrappingNullableUtils.prepareKeySerializer(any(), any(), any())) + .thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG")); + + final Throwable exception = assertThrows(StreamsException.class, () -> sink.init(context)); + + assertThat( + exception.getMessage(), + equalTo("Failed to initialize key serdes for sink node anyNodeName") + ); + assertThat( + exception.getCause().getMessage(), + equalTo("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG") + ); + } + + @Test + public void shouldThrowStreamsExceptionOnUndefinedValueSerde() { + utilsMock.when(() -> WrappingNullableUtils.prepareValueSerializer(any(), any(), any())) + .thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG")); + + final Throwable exception = assertThrows(StreamsException.class, () -> sink.init(context)); + + assertThat( + exception.getMessage(), + equalTo("Failed to initialize value serdes for sink node anyNodeName") + ); + assertThat( + exception.getCause().getMessage(), + equalTo("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG") + ); + } + + @Test + public void shouldThrowStreamsExceptionWithExplicitErrorMessage() { + utilsMock.when(() -> WrappingNullableUtils.prepareKeySerializer(any(), any(), any())).thenThrow(new StreamsException("")); + + final Throwable exception = assertThrows(StreamsException.class, () -> sink.init(context)); + + assertThat(exception.getMessage(), equalTo("Failed to initialize key serdes for sink node anyNodeName")); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java index 03f22a3a91..f048f9948d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Metrics; @@ -24,11 +25,17 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.test.InternalMockProcessorContext; import org.apache.kafka.test.MockSourceNode; import org.apache.kafka.test.StreamsTestUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import java.nio.charset.StandardCharsets; import java.util.Map; @@ -39,9 +46,25 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; public class SourceNodeTest { + private MockedStatic utilsMock; + + @BeforeEach + public void setup() { + utilsMock = Mockito.mockStatic(WrappingNullableUtils.class); + } + + @AfterEach + public void cleanup() { + utilsMock.close(); + } + + @Test public void shouldProvideTopicHeadersAndDataToKeyDeserializer() { final SourceNode sourceNode = new MockSourceNode<>(new TheDeserializer(), new TheDeserializer()); @@ -106,4 +129,62 @@ public void shouldExposeProcessMetrics() { contains(sensorNamePrefix + ".s.process") ); } + + @Test + public void shouldThrowStreamsExceptionOnUndefinedKeySerde() { + final InternalMockProcessorContext context = new InternalMockProcessorContext<>(); + + final SourceNode node = + new SourceNode<>(context.currentNode().name(), new TheDeserializer(), new TheDeserializer()); + + utilsMock.when(() -> WrappingNullableUtils.prepareKeyDeserializer(any(), any(), any())) + .thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG")); + + final Throwable exception = assertThrows(StreamsException.class, () -> node.init(context)); + + assertThat( + exception.getMessage(), + equalTo("Failed to initialize key serdes for source node TESTING_NODE") + ); + assertThat( + exception.getCause().getMessage(), + equalTo("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG") + ); + } + + @Test + public void shouldThrowStreamsExceptionOnUndefinedValueSerde() { + final InternalMockProcessorContext context = new InternalMockProcessorContext<>(); + + final SourceNode node = + new SourceNode<>(context.currentNode().name(), new TheDeserializer(), new TheDeserializer()); + + utilsMock.when(() -> WrappingNullableUtils.prepareValueDeserializer(any(), any(), any())) + .thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG")); + + final Throwable exception = assertThrows(StreamsException.class, () -> node.init(context)); + + assertThat( + exception.getMessage(), + equalTo("Failed to initialize value serdes for source node TESTING_NODE") + ); + assertThat( + exception.getCause().getMessage(), + equalTo("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG") + ); + } + + @Test + public void shouldThrowStreamsExceptionWithExplicitErrorMessage() { + final InternalMockProcessorContext context = new InternalMockProcessorContext<>(); + + final SourceNode node = + new SourceNode<>(context.currentNode().name(), new TheDeserializer(), new TheDeserializer()); + + utilsMock.when(() -> WrappingNullableUtils.prepareKeyDeserializer(any(), any(), any())).thenThrow(new StreamsException("")); + + final Throwable exception = assertThrows(StreamsException.class, () -> node.init(context)); + + assertThat(exception.getMessage(), equalTo("Failed to initialize key serdes for source node TESTING_NODE")); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializerTest.java new file mode 100644 index 0000000000..2a692f278e --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializerTest.java @@ -0,0 +1,157 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.test.MockInternalNewProcessorContext; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; + +public class StoreSerdeInitializerTest { + + private MockedStatic utilsMock; + + @BeforeEach + public void setup() { + utilsMock = Mockito.mockStatic(WrappingNullableUtils.class); + } + + @AfterEach + public void cleanup() { + utilsMock.close(); + } + + @Test + public void shouldPrepareStoreSerdeForProcessorContext() { + final Serde keySerde = new Serdes.StringSerde(); + final Serde valueSerde = new Serdes.StringSerde(); + + final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>(); + + utilsMock.when(() -> WrappingNullableUtils.prepareKeySerde(any(), any())).thenReturn(keySerde); + utilsMock.when(() -> WrappingNullableUtils.prepareValueSerde(any(), any())).thenReturn(valueSerde); + + final StateSerdes result = StoreSerdeInitializer.prepareStoreSerde( + (ProcessorContext) context, "myStore", "topic", keySerde, valueSerde, WrappingNullableUtils::prepareValueSerde); + + assertThat(result.keySerde(), equalTo(keySerde)); + assertThat(result.valueSerde(), equalTo(valueSerde)); + assertThat(result.topic(), equalTo("topic")); + } + + @Test + public void shouldThrowStreamsExceptionOnUndefinedKeySerdeForProcessorContext() { + final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>(); + + utilsMock.when(() -> WrappingNullableUtils.prepareKeySerde(any(), any())) + .thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG")); + + final Throwable exception = assertThrows(StreamsException.class, + () -> StoreSerdeInitializer.prepareStoreSerde((ProcessorContext) context, "myStore", "topic", + new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde)); + + assertThat(exception.getMessage(), equalTo("Failed to initialize key serdes for store myStore")); + assertThat(exception.getCause().getMessage(), equalTo("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG")); + } + + @Test + public void shouldThrowStreamsExceptionOnUndefinedValueSerdeForProcessorContext() { + final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>(); + + utilsMock.when(() -> WrappingNullableUtils.prepareValueSerde(any(), any())) + .thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG")); + + final Throwable exception = assertThrows(StreamsException.class, + () -> StoreSerdeInitializer.prepareStoreSerde((ProcessorContext) context, "myStore", "topic", + new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde)); + + assertThat(exception.getMessage(), equalTo("Failed to initialize value serdes for store myStore")); + assertThat(exception.getCause().getMessage(), equalTo("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG")); + } + + @Test + public void shouldThrowStreamsExceptionOnUndefinedKeySerdeForStateStoreContext() { + final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>(); + + utilsMock.when(() -> WrappingNullableUtils.prepareKeySerde(any(), any())) + .thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG")); + + final Throwable exception = assertThrows(StreamsException.class, + () -> StoreSerdeInitializer.prepareStoreSerde((StateStoreContext) context, "myStore", "topic", + new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde)); + + assertThat(exception.getMessage(), equalTo("Failed to initialize key serdes for store myStore")); + assertThat(exception.getCause().getMessage(), equalTo("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG")); + } + + @Test + public void shouldThrowStreamsExceptionOnUndefinedValueSerdeForStateStoreContext() { + final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>(); + + utilsMock.when(() -> WrappingNullableUtils.prepareValueSerde(any(), any())) + .thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG")); + + final Throwable exception = assertThrows(StreamsException.class, + () -> StoreSerdeInitializer.prepareStoreSerde((StateStoreContext) context, "myStore", "topic", + new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde)); + + assertThat(exception.getMessage(), equalTo("Failed to initialize value serdes for store myStore")); + assertThat(exception.getCause().getMessage(), equalTo("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG")); + } + + @Test + public void shouldThrowStreamsExceptionWithExplicitErrorMessageForProcessorContext() { + final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>(); + + utilsMock.when(() -> WrappingNullableUtils.prepareKeySerde(any(), any())).thenThrow(new StreamsException("")); + + final Throwable exception = assertThrows(StreamsException.class, + () -> StoreSerdeInitializer.prepareStoreSerde((ProcessorContext) context, "myStore", "topic", + new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde)); + + assertThat(exception.getMessage(), equalTo("Failed to initialize key serdes for store myStore")); + } + + @Test + public void shouldThrowStreamsExceptionWithExplicitErrorMessageForStateStoreContext() { + final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>(); + + utilsMock.when(() -> WrappingNullableUtils.prepareValueSerde(any(), any())).thenThrow(new StreamsException("")); + + final Throwable exception = assertThrows(StreamsException.class, + () -> StoreSerdeInitializer.prepareStoreSerde((StateStoreContext) context, "myStore", "topic", + new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde)); + + assertThat(exception.getMessage(), equalTo("Failed to initialize value serdes for store myStore")); + } +} From 1f6c5dc5df47605d164b7b27fa47c2b0c06f2770 Mon Sep 17 00:00:00 2001 From: Florin Akermann Date: Thu, 6 Jun 2024 00:22:31 +0200 Subject: [PATCH 026/128] KAFKA-12317: Update FK-left-join documentation (#15689) FK left-join was changed via KIP-962. This PR updates the docs accordingly. Reviewers: Ayoub Omari , Matthias J. Sax --- docs/streams/developer-guide/dsl-api.html | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index acd40ad835..fd5c22cae3 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -2542,6 +2542,10 @@

    KTable-KTable Foreign-Key