diff --git a/LICENSE-binary b/LICENSE-binary index 6e114e8e925..c8fa1e8207a 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -238,10 +238,10 @@ License Version 2.0: - jetty-util-12.0.22 - jose4j-0.9.6 - jspecify-1.0.0 -- log4j-api-2.24.3 -- log4j-core-2.24.3 -- log4j-slf4j-impl-2.24.3 -- log4j-1.2-api-2.24.3 +- log4j-api-2.25.1 +- log4j-core-2.25.1 +- log4j-slf4j-impl-2.25.1 +- log4j-1.2-api-2.25.1 - lz4-java-1.8.0 - maven-artifact-3.9.6 - metrics-core-2.2.0 diff --git a/README.md b/README.md index 8b5fe4c332e..06c0e3921eb 100644 --- a/README.md +++ b/README.md @@ -52,6 +52,7 @@ Follow instructions in https://kafka.apache.org/quickstart ### Running a particular unit/integration test ### ./gradlew clients:test --tests RequestResponseTest + ./gradlew streams:integration-tests:test --tests RestoreIntegrationTest ### Repeatedly running a particular unit/integration test with specific times by setting N ### N=500; I=0; while [ $I -lt $N ] && ./gradlew clients:test --tests RequestResponseTest --rerun --fail-fast; do (( I=$I+1 )); echo "Completed run: $I"; sleep 1; done @@ -59,6 +60,7 @@ Follow instructions in https://kafka.apache.org/quickstart ### Running a particular test method within a unit/integration test ### ./gradlew core:test --tests kafka.api.ProducerFailureHandlingTest.testCannotSendToInternalTopic ./gradlew clients:test --tests org.apache.kafka.clients.MetadataTest.testTimeToNextUpdate + ./gradlew streams:integration-tests:test --tests org.apache.kafka.streams.integration.RestoreIntegrationTest.shouldRestoreNullRecord ### Running a particular unit/integration test with log4j output ### By default, there will be only small number of logs output while testing. You can adjust it by changing the `log4j2.yaml` file in the module's `src/test/resources` directory. diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerSubscriptionTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerSubscriptionTest.java index 065e6600a54..ff840ebf2ea 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerSubscriptionTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerSubscriptionTest.java @@ -576,7 +576,7 @@ public class PlaintextConsumerSubscriptionTest { } @ClusterTest - public void testAsyncConsumerClassicConsumerSubscribeInvalidTopicCanUnsubscribe() throws InterruptedException { + public void testAsyncConsumerSubscribeInvalidTopicCanUnsubscribe() throws InterruptedException { testSubscribeInvalidTopicCanUnsubscribe(GroupProtocol.CONSUMER); } diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 1100479021f..692847a8b15 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1051,9 +1051,9 @@ public class NetworkClient implements KafkaClient { apiVersionsResponse.data().finalizedFeaturesEpoch()); apiVersions.update(node, nodeVersionInfo); this.connectionStates.ready(node); - log.debug("Node {} has finalized features epoch: {}, finalized features: {}, supported features: {}, ZK migration ready: {}, API versions: {}.", + log.debug("Node {} has finalized features epoch: {}, finalized features: {}, supported features: {}, API versions: {}.", node, apiVersionsResponse.data().finalizedFeaturesEpoch(), apiVersionsResponse.data().finalizedFeatures(), - apiVersionsResponse.data().supportedFeatures(), apiVersionsResponse.data().zkMigrationReady(), nodeVersionInfo); + apiVersionsResponse.data().supportedFeatures(), nodeVersionInfo); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AlterConfigOp.java b/clients/src/main/java/org/apache/kafka/clients/admin/AlterConfigOp.java index 48d5646764d..789c9f64a93 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/AlterConfigOp.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AlterConfigOp.java @@ -26,6 +26,20 @@ import java.util.stream.Collectors; /** * A class representing an alter configuration entry containing name, value and operation type. + *

+ * Note for Broker Logger Configuration:
+ * When altering broker logger levels (using {@link org.apache.kafka.common.config.ConfigResource.Type#BROKER_LOGGER}), + * it is strongly recommended to use log level constants from {@link org.apache.kafka.common.config.LogLevelConfig} instead of string literals. + * This ensures compatibility with Kafka's log level validation and avoids potential configuration errors. + *

+ * Example: + *

+ * Recommended approach:
+ * new AlterConfigOp(new ConfigEntry(loggerName, LogLevelConfig.DEBUG_LOG_LEVEL), OpType.SET)
+ *
+ * Avoid this:
+ * new AlterConfigOp(new ConfigEntry(loggerName, "DEBUG"), OpType.SET)
+ * 
*/ public class AlterConfigOp { 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 ac2d67d2022..90f83eac935 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 @@ -2334,7 +2334,7 @@ public class KafkaAdminClient extends AdminClient { } // First, we need to retrieve the node info. - DescribeClusterResult clusterResult = describeCluster(); + DescribeClusterResult clusterResult = describeCluster(new DescribeClusterOptions().timeoutMs(options.timeoutMs())); clusterResult.nodes().whenComplete( (nodes, exception) -> { if (exception != null) { @@ -5154,6 +5154,8 @@ public class KafkaAdminClient extends AdminClient { return ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP; } else if (offsetSpec instanceof OffsetSpec.LatestTieredSpec) { return ListOffsetsRequest.LATEST_TIERED_TIMESTAMP; + } else if (offsetSpec instanceof OffsetSpec.EarliestPendingUploadSpec) { + return ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP; } return ListOffsetsRequest.LATEST_TIMESTAMP; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java b/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java index 68f94cc493e..ad73c8d51f0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java @@ -28,6 +28,7 @@ public class OffsetSpec { public static class MaxTimestampSpec extends OffsetSpec { } public static class EarliestLocalSpec extends OffsetSpec { } public static class LatestTieredSpec extends OffsetSpec { } + public static class EarliestPendingUploadSpec extends OffsetSpec { } public static class TimestampSpec extends OffsetSpec { private final long timestamp; @@ -91,4 +92,13 @@ public class OffsetSpec { public static OffsetSpec latestTiered() { return new LatestTieredSpec(); } + + /** + * Used to retrieve the earliest offset of records that are pending upload to remote storage. + *
+ * Note: When tiered storage is not enabled, we will return unknown offset. + */ + public static OffsetSpec earliestPendingUpload() { + return new EarliestPendingUploadSpec(); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java index f7c495d7fd8..a46d6f24a7b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java @@ -103,12 +103,17 @@ public final class ListOffsetsHandler extends Batched offsetTimestampsByPartition.get(key) == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP); + boolean requireEarliestPendingUploadTimestamp = keys + .stream() + .anyMatch(key -> offsetTimestampsByPartition.get(key) == ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP); + int timeoutMs = options.timeoutMs() != null ? options.timeoutMs() : defaultApiTimeoutMs; return ListOffsetsRequest.Builder.forConsumer(true, options.isolationLevel(), supportsMaxTimestamp, requireEarliestLocalTimestamp, - requireTieredStorageTimestamp) + requireTieredStorageTimestamp, + requireEarliestPendingUploadTimestamp) .setTargetTimes(new ArrayList<>(topicsByName.values())) .setTimeoutMs(timeoutMs); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java index 7415412d050..5862ebdfafc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java @@ -47,6 +47,8 @@ public class ListOffsetsRequest extends AbstractRequest { public static final long LATEST_TIERED_TIMESTAMP = -5L; + public static final long EARLIEST_PENDING_UPLOAD_TIMESTAMP = -6L; + public static final int CONSUMER_REPLICA_ID = -1; public static final int DEBUGGING_REPLICA_ID = -2; @@ -58,16 +60,19 @@ public class ListOffsetsRequest extends AbstractRequest { public static Builder forConsumer(boolean requireTimestamp, IsolationLevel isolationLevel) { - return forConsumer(requireTimestamp, isolationLevel, false, false, false); + return forConsumer(requireTimestamp, isolationLevel, false, false, false, false); } public static Builder forConsumer(boolean requireTimestamp, IsolationLevel isolationLevel, boolean requireMaxTimestamp, boolean requireEarliestLocalTimestamp, - boolean requireTieredStorageTimestamp) { + boolean requireTieredStorageTimestamp, + boolean requireEarliestPendingUploadTimestamp) { short minVersion = ApiKeys.LIST_OFFSETS.oldestVersion(); - if (requireTieredStorageTimestamp) + if (requireEarliestPendingUploadTimestamp) + minVersion = 11; + else if (requireTieredStorageTimestamp) minVersion = 9; else if (requireEarliestLocalTimestamp) minVersion = 8; diff --git a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java index e0491943fef..bef65977be4 100644 --- a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java @@ -50,6 +50,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.StringJoiner; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; @@ -269,6 +270,7 @@ public class ClientTelemetryReporter implements MetricsReporter { private static final double INITIAL_PUSH_JITTER_LOWER = 0.5; private static final double INITIAL_PUSH_JITTER_UPPER = 1.5; + private final Set unsupportedCompressionTypes = ConcurrentHashMap.newKeySet(); private final ReadWriteLock lock = new ReentrantReadWriteLock(); private final Condition subscriptionLoaded = lock.writeLock().newCondition(); /* @@ -713,12 +715,26 @@ public class ClientTelemetryReporter implements MetricsReporter { return Optional.empty(); } - CompressionType compressionType = ClientTelemetryUtils.preferredCompressionType(localSubscription.acceptedCompressionTypes()); + CompressionType compressionType = ClientTelemetryUtils.preferredCompressionType(localSubscription.acceptedCompressionTypes(), unsupportedCompressionTypes); ByteBuffer compressedPayload; try { compressedPayload = ClientTelemetryUtils.compress(payload, compressionType); } catch (Throwable e) { - log.debug("Failed to compress telemetry payload for compression: {}, sending uncompressed data", compressionType); + // Distinguish between recoverable errors (NoClassDefFoundError for missing compression libs) + // and fatal errors (OutOfMemoryError, etc.) that should terminate telemetry. + if (e instanceof Error && !(e instanceof NoClassDefFoundError) && !(e.getCause() instanceof NoClassDefFoundError)) { + lock.writeLock().lock(); + try { + state = ClientTelemetryState.TERMINATED; + } finally { + lock.writeLock().unlock(); + } + log.error("Unexpected error occurred while compressing telemetry payload for compression: {}, stopping client telemetry", compressionType, e); + throw new KafkaException("Unexpected compression error", e); + } + + log.debug("Failed to compress telemetry payload for compression: {}, sending uncompressed data", compressionType, e); + unsupportedCompressionTypes.add(compressionType); compressedPayload = ByteBuffer.wrap(payload.toByteArray()); compressionType = CompressionType.NONE; } diff --git a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtils.java b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtils.java index 3c555afb3b0..111b041946c 100644 --- a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtils.java @@ -39,6 +39,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.function.Predicate; import io.opentelemetry.proto.metrics.v1.MetricsData; @@ -181,13 +182,23 @@ public class ClientTelemetryUtils { return validateResourceLabel(metadata, MetricsContext.NAMESPACE); } - public static CompressionType preferredCompressionType(List acceptedCompressionTypes) { - if (acceptedCompressionTypes != null && !acceptedCompressionTypes.isEmpty()) { - // Broker is providing the compression types in order of preference. Grab the - // first one. - return acceptedCompressionTypes.get(0); - } - return CompressionType.NONE; + /** + * Determines the preferred compression type from broker-accepted types, avoiding unsupported ones. + * + * @param acceptedCompressionTypes the list of compression types accepted by the broker in order + * of preference (must not be null, use empty list if no compression is accepted) + * @param unsupportedCompressionTypes the set of compression types that should be avoided due to + * missing libraries or previous failures (must not be null) + * @return the preferred compression type to use, or {@link CompressionType#NONE} if no acceptable + * compression type is available + */ + public static CompressionType preferredCompressionType(List acceptedCompressionTypes, Set unsupportedCompressionTypes) { + // Broker is providing the compression types in order of preference. Grab the + // first one that's supported. + return acceptedCompressionTypes.stream() + .filter(t -> !unsupportedCompressionTypes.contains(t)) + .findFirst() + .orElse(CompressionType.NONE); } public static ByteBuffer compress(MetricsData metrics, CompressionType compressionType) throws IOException { diff --git a/clients/src/main/resources/common/message/DescribeClusterResponse.json b/clients/src/main/resources/common/message/DescribeClusterResponse.json index a17e427c8c3..1911b1ec33d 100644 --- a/clients/src/main/resources/common/message/DescribeClusterResponse.json +++ b/clients/src/main/resources/common/message/DescribeClusterResponse.json @@ -36,7 +36,7 @@ { "name": "ClusterId", "type": "string", "versions": "0+", "about": "The cluster ID that responding broker belongs to." }, { "name": "ControllerId", "type": "int32", "versions": "0+", "default": "-1", "entityType": "brokerId", - "about": "The ID of the controller broker." }, + "about": "The ID of the controller. When handled by a controller, returns the current voter leader ID. When handled by a broker, returns a random alive broker ID as a fallback." }, { "name": "Brokers", "type": "[]DescribeClusterBroker", "versions": "0+", "about": "Each broker in the response.", "fields": [ { "name": "BrokerId", "type": "int32", "versions": "0+", "mapKey": true, "entityType": "brokerId", diff --git a/clients/src/main/resources/common/message/ListOffsetsRequest.json b/clients/src/main/resources/common/message/ListOffsetsRequest.json index 6f8ff7d6cf9..1a2de6ca30a 100644 --- a/clients/src/main/resources/common/message/ListOffsetsRequest.json +++ b/clients/src/main/resources/common/message/ListOffsetsRequest.json @@ -40,7 +40,9 @@ // Version 9 enables listing offsets by last tiered offset (KIP-1005). // // Version 10 enables async remote list offsets support (KIP-1075) - "validVersions": "1-10", + // + // Version 11 enables listing offsets by earliest pending upload offset (KIP-1023) + "validVersions": "1-11", "flexibleVersions": "6+", "latestVersionUnstable": false, "fields": [ diff --git a/clients/src/main/resources/common/message/ListOffsetsResponse.json b/clients/src/main/resources/common/message/ListOffsetsResponse.json index 7f9588847b9..1407273bf4d 100644 --- a/clients/src/main/resources/common/message/ListOffsetsResponse.json +++ b/clients/src/main/resources/common/message/ListOffsetsResponse.json @@ -40,7 +40,9 @@ // Version 9 enables listing offsets by last tiered offset (KIP-1005). // // Version 10 enables async remote list offsets support (KIP-1075) - "validVersions": "1-10", + // + // Version 11 enables listing offsets by earliest pending upload offset (KIP-1023) + "validVersions": "1-11", "flexibleVersions": "6+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 1098078b582..e7fa11177d3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -551,7 +551,8 @@ public class KafkaAdminClientTest { * Test if admin client can be closed in the callback invoked when * an api call completes. If calling {@link Admin#close()} in callback, AdminClient thread hangs */ - @Test @Timeout(10) + @Test + @Timeout(10) public void testCloseAdminClientInCallback() throws InterruptedException { MockTime time = new MockTime(); AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, mockCluster(3, 0)); @@ -8729,6 +8730,34 @@ public class KafkaAdminClientTest { } } + @Test + public void testListOffsetsEarliestPendingUploadSpecSpecMinVersion() throws Exception { + Node node = new Node(0, "localhost", 8120); + List nodes = Collections.singletonList(node); + List pInfos = new ArrayList<>(); + pInfos.add(new PartitionInfo("foo", 0, node, new Node[]{node}, new Node[]{node})); + final Cluster cluster = new Cluster( + "mockClusterId", + nodes, + pInfos, + Collections.emptySet(), + Collections.emptySet(), + node); + final TopicPartition tp0 = new TopicPartition("foo", 0); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster, + AdminClientConfig.RETRIES_CONFIG, "2")) { + + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.adminClient().listOffsets(Collections.singletonMap(tp0, OffsetSpec.earliestPendingUpload())); + + TestUtils.waitForCondition(() -> env.kafkaClient().requests().stream().anyMatch(request -> + request.requestBuilder().apiKey().messageType == ApiMessageType.LIST_OFFSETS && request.requestBuilder().oldestAllowedVersion() == 11 + ), "no listOffsets request has the expected oldestAllowedVersion"); + } + } + private Map makeTestFeatureUpdates() { return Utils.mkMap( Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, FeatureUpdate.UpgradeType.UPGRADE)), @@ -11668,4 +11697,27 @@ public class KafkaAdminClientTest { .setAssignmentEpoch(1)); return data; } + + @Test + @Timeout(30) + public void testDescribeTopicsTimeoutWhenNoBrokerResponds() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv( + mockCluster(1, 0), + AdminClientConfig.RETRIES_CONFIG, "0", + AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "30000")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + // Not using prepareResponse is equivalent to "no brokers respond". + long start = System.currentTimeMillis(); + DescribeTopicsResult result = env.adminClient().describeTopics(List.of("test-topic"), new DescribeTopicsOptions().timeoutMs(200)); + Map> topicDescriptionMap = result.topicNameValues(); + KafkaFuture topicDescription = topicDescriptionMap.get("test-topic"); + ExecutionException exception = assertThrows(ExecutionException.class, topicDescription::get); + // Duration should be greater than or equal to 200 ms but less than 30000 ms. + long duration = System.currentTimeMillis() - start; + + assertInstanceOf(TimeoutException.class, exception.getCause()); + assertTrue(duration >= 150L && duration < 30000); + } + } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java index 2cf4cbc00c9..48542c1a2fd 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java @@ -127,13 +127,16 @@ public class ListOffsetsRequestTest { .forConsumer(false, IsolationLevel.READ_COMMITTED); ListOffsetsRequest.Builder maxTimestampRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, true, false, false); + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, true, false, false, false); ListOffsetsRequest.Builder requireEarliestLocalTimestampRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, true, false); + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, true, false, false); ListOffsetsRequest.Builder requireTieredStorageTimestampRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false, true); + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false, true, false); + + ListOffsetsRequest.Builder requireEarliestPendingUploadTimestampRequestBuilder = ListOffsetsRequest.Builder + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false, false, true); assertEquals((short) 1, consumerRequestBuilder.oldestAllowedVersion()); assertEquals((short) 1, requireTimestampRequestBuilder.oldestAllowedVersion()); @@ -141,5 +144,6 @@ public class ListOffsetsRequestTest { assertEquals((short) 7, maxTimestampRequestBuilder.oldestAllowedVersion()); assertEquals((short) 8, requireEarliestLocalTimestampRequestBuilder.oldestAllowedVersion()); assertEquals((short) 9, requireTieredStorageTimestampRequestBuilder.oldestAllowedVersion()); + assertEquals((short) 11, requireEarliestPendingUploadTimestampRequestBuilder.oldestAllowedVersion()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java index b708b4eeb60..c06e853b073 100644 --- a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java @@ -19,6 +19,7 @@ package org.apache.kafka.common.telemetry.internals; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.GetTelemetrySubscriptionsRequestData; import org.apache.kafka.common.message.GetTelemetrySubscriptionsResponseData; @@ -63,8 +64,10 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; public class ClientTelemetryReporterTest { @@ -413,6 +416,134 @@ public class ClientTelemetryReporterTest { } } + @Test + public void testCreateRequestPushCompressionFallbackToNextType() { + clientTelemetryReporter.configure(configs); + clientTelemetryReporter.contextChange(metricsContext); + + ClientTelemetryReporter.DefaultClientTelemetrySender telemetrySender = (ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter.telemetrySender(); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.SUBSCRIPTION_IN_PROGRESS)); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.PUSH_NEEDED)); + + // Set up subscription with multiple compression types: GZIP -> LZ4 -> SNAPPY + ClientTelemetryReporter.ClientTelemetrySubscription subscription = new ClientTelemetryReporter.ClientTelemetrySubscription( + uuid, 1234, 20000, List.of(CompressionType.GZIP, CompressionType.LZ4, CompressionType.SNAPPY), true, null); + telemetrySender.updateSubscriptionResult(subscription, time.milliseconds()); + + try (MockedStatic mockedCompress = Mockito.mockStatic(ClientTelemetryUtils.class, new CallsRealMethods())) { + // First request: GZIP fails with NoClassDefFoundError, should use NONE for this request + mockedCompress.when(() -> ClientTelemetryUtils.compress(any(), eq(CompressionType.GZIP))).thenThrow(new NoClassDefFoundError("GZIP not available")); + + Optional> requestOptional = telemetrySender.createRequest(); + assertNotNull(requestOptional); + assertTrue(requestOptional.isPresent()); + assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); + PushTelemetryRequest request = (PushTelemetryRequest) requestOptional.get().build(); + + // Should fallback to NONE for this request (GZIP gets cached as unsupported) + assertEquals(CompressionType.NONE.id, request.data().compressionType()); + assertEquals(ClientTelemetryState.PUSH_IN_PROGRESS, telemetrySender.state()); + + // Reset state for next request + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.PUSH_NEEDED)); + + // Second request: LZ4 is selected (since GZIP is now cached as unsupported), LZ4 fails, should use NONE + // Note that some libraries eg. LZ4 return KafkaException with cause as NoClassDefFoundError + mockedCompress.when(() -> ClientTelemetryUtils.compress(any(), eq(CompressionType.LZ4))).thenThrow(new KafkaException(new NoClassDefFoundError("LZ4 not available"))); + + requestOptional = telemetrySender.createRequest(); + assertNotNull(requestOptional); + assertTrue(requestOptional.isPresent()); + assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); + request = (PushTelemetryRequest) requestOptional.get().build(); + + // Should fallback to NONE for this request (LZ4 gets cached as unsupported) + assertEquals(CompressionType.NONE.id, request.data().compressionType()); + assertEquals(ClientTelemetryState.PUSH_IN_PROGRESS, telemetrySender.state()); + + // Reset state for next request + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.PUSH_NEEDED)); + + // Third request: SNAPPY is selected (since GZIP and LZ4 are now cached as unsupported), SNAPPY fails, should use NONE + mockedCompress.when(() -> ClientTelemetryUtils.compress(any(), eq(CompressionType.SNAPPY))).thenThrow(new NoClassDefFoundError("SNAPPY not available")); + + requestOptional = telemetrySender.createRequest(); + assertNotNull(requestOptional); + assertTrue(requestOptional.isPresent()); + assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); + request = (PushTelemetryRequest) requestOptional.get().build(); + + // Should fallback to NONE for this request (SNAPPY gets cached as unsupported) + assertEquals(CompressionType.NONE.id, request.data().compressionType()); + assertEquals(ClientTelemetryState.PUSH_IN_PROGRESS, telemetrySender.state()); + + // Reset state for next request + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.PUSH_NEEDED)); + + // Fourth request: All compression types are now cached as unsupported, should use NONE directly + requestOptional = telemetrySender.createRequest(); + assertNotNull(requestOptional); + assertTrue(requestOptional.isPresent()); + assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); + request = (PushTelemetryRequest) requestOptional.get().build(); + + // Should use NONE directly (no compression types are supported) + assertEquals(CompressionType.NONE.id, request.data().compressionType()); + assertEquals(ClientTelemetryState.PUSH_IN_PROGRESS, telemetrySender.state()); + } + } + + @Test + public void testCreateRequestPushCompressionFallbackAndTermination() { + clientTelemetryReporter.configure(configs); + clientTelemetryReporter.contextChange(metricsContext); + + ClientTelemetryReporter.DefaultClientTelemetrySender telemetrySender = (ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter.telemetrySender(); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.SUBSCRIPTION_IN_PROGRESS)); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.PUSH_NEEDED)); + + // Set up subscription with ZSTD compression type + ClientTelemetryReporter.ClientTelemetrySubscription subscription = new ClientTelemetryReporter.ClientTelemetrySubscription( + uuid, 1234, 20000, List.of(CompressionType.ZSTD, CompressionType.LZ4), true, null); + telemetrySender.updateSubscriptionResult(subscription, time.milliseconds()); + + try (MockedStatic mockedCompress = Mockito.mockStatic(ClientTelemetryUtils.class, new CallsRealMethods())) { + + // === Test 1: NoClassDefFoundError fallback (recoverable) === + mockedCompress.when(() -> ClientTelemetryUtils.compress(any(), eq(CompressionType.ZSTD))) + .thenThrow(new NoClassDefFoundError("com/github/luben/zstd/BufferPool")); + + assertEquals(ClientTelemetryState.PUSH_NEEDED, telemetrySender.state()); + + Optional> request1 = telemetrySender.createRequest(); + assertNotNull(request1); + assertTrue(request1.isPresent()); + assertInstanceOf(PushTelemetryRequest.class, request1.get().build()); + PushTelemetryRequest pushRequest1 = (PushTelemetryRequest) request1.get().build(); + assertEquals(CompressionType.NONE.id, pushRequest1.data().compressionType()); // Fallback to NONE + assertEquals(ClientTelemetryState.PUSH_IN_PROGRESS, telemetrySender.state()); + + // Reset state (simulate successful response handling) + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.PUSH_NEEDED)); + + // === Test 2: OutOfMemoryError causes termination (non-recoverable Error) === + mockedCompress.reset(); + mockedCompress.when(() -> ClientTelemetryUtils.compress(any(), eq(CompressionType.LZ4))) + .thenThrow(new OutOfMemoryError("Out of memory during compression")); + + assertEquals(ClientTelemetryState.PUSH_NEEDED, telemetrySender.state()); + + assertThrows(KafkaException.class, () -> telemetrySender.createRequest()); + assertEquals(ClientTelemetryState.TERMINATED, telemetrySender.state()); + + // === Test 3: After termination, no more requests === + Optional> request3 = telemetrySender.createRequest(); + assertNotNull(request3); + assertFalse(request3.isPresent()); // No request created + assertEquals(ClientTelemetryState.TERMINATED, telemetrySender.state()); // State remains TERMINATED + } + } + @Test public void testHandleResponseGetSubscriptions() { ClientTelemetryReporter.DefaultClientTelemetrySender telemetrySender = (ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter.telemetrySender(); diff --git a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtilsTest.java index 41679bed3f7..47925ff8e0a 100644 --- a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtilsTest.java @@ -30,10 +30,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.time.Instant; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.function.Predicate; import io.opentelemetry.proto.metrics.v1.Metric; @@ -69,12 +68,12 @@ public class ClientTelemetryUtilsTest { @Test public void testGetSelectorFromRequestedMetrics() { // no metrics selector - assertEquals(ClientTelemetryUtils.SELECTOR_NO_METRICS, ClientTelemetryUtils.getSelectorFromRequestedMetrics(Collections.emptyList())); + assertEquals(ClientTelemetryUtils.SELECTOR_NO_METRICS, ClientTelemetryUtils.getSelectorFromRequestedMetrics(List.of())); assertEquals(ClientTelemetryUtils.SELECTOR_NO_METRICS, ClientTelemetryUtils.getSelectorFromRequestedMetrics(null)); // all metrics selector - assertEquals(ClientTelemetryUtils.SELECTOR_ALL_METRICS, ClientTelemetryUtils.getSelectorFromRequestedMetrics(Collections.singletonList("*"))); + assertEquals(ClientTelemetryUtils.SELECTOR_ALL_METRICS, ClientTelemetryUtils.getSelectorFromRequestedMetrics(List.of("*"))); // specific metrics selector - Predicate selector = ClientTelemetryUtils.getSelectorFromRequestedMetrics(Arrays.asList("metric1", "metric2")); + Predicate selector = ClientTelemetryUtils.getSelectorFromRequestedMetrics(List.of("metric1", "metric2")); assertNotEquals(ClientTelemetryUtils.SELECTOR_NO_METRICS, selector); assertNotEquals(ClientTelemetryUtils.SELECTOR_ALL_METRICS, selector); assertTrue(selector.test(new MetricKey("metric1.test"))); @@ -86,7 +85,7 @@ public class ClientTelemetryUtilsTest { @Test public void testGetCompressionTypesFromAcceptedList() { assertEquals(0, ClientTelemetryUtils.getCompressionTypesFromAcceptedList(null).size()); - assertEquals(0, ClientTelemetryUtils.getCompressionTypesFromAcceptedList(Collections.emptyList()).size()); + assertEquals(0, ClientTelemetryUtils.getCompressionTypesFromAcceptedList(List.of()).size()); List compressionTypes = new ArrayList<>(); compressionTypes.add(CompressionType.GZIP.id); @@ -123,10 +122,24 @@ public class ClientTelemetryUtilsTest { @Test public void testPreferredCompressionType() { - assertEquals(CompressionType.NONE, ClientTelemetryUtils.preferredCompressionType(Collections.emptyList())); - assertEquals(CompressionType.NONE, ClientTelemetryUtils.preferredCompressionType(null)); - assertEquals(CompressionType.NONE, ClientTelemetryUtils.preferredCompressionType(Arrays.asList(CompressionType.NONE, CompressionType.GZIP))); - assertEquals(CompressionType.GZIP, ClientTelemetryUtils.preferredCompressionType(Arrays.asList(CompressionType.GZIP, CompressionType.NONE))); + // Test with no unsupported types + assertEquals(CompressionType.NONE, ClientTelemetryUtils.preferredCompressionType(List.of(), Set.of())); + assertEquals(CompressionType.NONE, ClientTelemetryUtils.preferredCompressionType(List.of(CompressionType.NONE, CompressionType.GZIP), Set.of())); + assertEquals(CompressionType.GZIP, ClientTelemetryUtils.preferredCompressionType(List.of(CompressionType.GZIP, CompressionType.NONE), Set.of())); + + // Test unsupported type filtering (returns first available type, or NONE if all are unsupported) + assertEquals(CompressionType.LZ4, ClientTelemetryUtils.preferredCompressionType(List.of(CompressionType.GZIP, CompressionType.LZ4), Set.of(CompressionType.GZIP))); + assertEquals(CompressionType.SNAPPY, ClientTelemetryUtils.preferredCompressionType(List.of(CompressionType.GZIP, CompressionType.LZ4, CompressionType.SNAPPY), Set.of(CompressionType.GZIP, CompressionType.LZ4))); + assertEquals(CompressionType.NONE, ClientTelemetryUtils.preferredCompressionType(List.of(CompressionType.GZIP, CompressionType.LZ4), Set.of(CompressionType.GZIP, CompressionType.LZ4))); + + // Test edge case: no match between requested and supported types + assertEquals(CompressionType.GZIP, ClientTelemetryUtils.preferredCompressionType(List.of(CompressionType.GZIP, CompressionType.LZ4), Set.of(CompressionType.SNAPPY))); + + // Test NullPointerException for null parameters + assertThrows(NullPointerException.class, () -> + ClientTelemetryUtils.preferredCompressionType(null, Set.of())); + assertThrows(NullPointerException.class, () -> + ClientTelemetryUtils.preferredCompressionType(List.of(CompressionType.GZIP, CompressionType.NONE), null)); } @ParameterizedTest @@ -150,19 +163,19 @@ public class ClientTelemetryUtilsTest { private MetricsData getMetricsData() { List metricsList = new ArrayList<>(); metricsList.add(SinglePointMetric.sum( - new MetricKey("metricName"), 1.0, true, Instant.now(), null, Collections.emptySet()) + new MetricKey("metricName"), 1.0, true, Instant.now(), null, Set.of()) .builder().build()); metricsList.add(SinglePointMetric.sum( - new MetricKey("metricName1"), 100.0, false, Instant.now(), Instant.now(), Collections.emptySet()) + new MetricKey("metricName1"), 100.0, false, Instant.now(), Instant.now(), Set.of()) .builder().build()); metricsList.add(SinglePointMetric.deltaSum( - new MetricKey("metricName2"), 1.0, true, Instant.now(), Instant.now(), Collections.emptySet()) + new MetricKey("metricName2"), 1.0, true, Instant.now(), Instant.now(), Set.of()) .builder().build()); metricsList.add(SinglePointMetric.gauge( - new MetricKey("metricName3"), 1.0, Instant.now(), Collections.emptySet()) + new MetricKey("metricName3"), 1.0, Instant.now(), Set.of()) .builder().build()); metricsList.add(SinglePointMetric.gauge( - new MetricKey("metricName4"), Long.valueOf(100), Instant.now(), Collections.emptySet()) + new MetricKey("metricName4"), Long.valueOf(100), Instant.now(), Set.of()) .builder().build()); MetricsData.Builder builder = MetricsData.newBuilder(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java index 3767e31ac7c..1a79698ae9f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java @@ -255,7 +255,7 @@ public abstract class Loggers { LoggerContext context = (LoggerContext) LogManager.getContext(false); var results = new HashMap(); context.getConfiguration().getLoggers().forEach((name, logger) -> results.put(name, loggerContext.getLogger(name))); - context.getLoggerRegistry().getLoggers().forEach(logger -> results.put(logger.getName(), logger)); + context.getLoggers().forEach(logger -> results.put(logger.getName(), logger)); return results; } diff --git a/core/src/main/java/kafka/server/share/SharePartition.java b/core/src/main/java/kafka/server/share/SharePartition.java index 68ced23a5f7..8ed094b85f1 100644 --- a/core/src/main/java/kafka/server/share/SharePartition.java +++ b/core/src/main/java/kafka/server/share/SharePartition.java @@ -260,10 +260,10 @@ public class SharePartition { private long endOffset; /** - * The initial read gap offset tracks if there are any gaps in the in-flight batch during initial - * read of the share partition state from the persister. + * The persister read result gap window tracks if there are any gaps in the in-flight batch during + * initial read of the share partition state from the persister. */ - private InitialReadGapOffset initialReadGapOffset; + private GapWindow persisterReadResultGapWindow; /** * We maintain the latest fetch offset and its metadata to estimate the minBytes requirement more efficiently. @@ -475,9 +475,9 @@ public class SharePartition { // in the cached state are not missed updateFindNextFetchOffset(true); endOffset = cachedState.lastEntry().getValue().lastOffset(); - // initialReadGapOffset is not required, if there are no gaps in the read state response + // gapWindow is not required, if there are no gaps in the read state response if (gapStartOffset != -1) { - initialReadGapOffset = new InitialReadGapOffset(endOffset, gapStartOffset); + persisterReadResultGapWindow = new GapWindow(endOffset, gapStartOffset); } // In case the persister read state RPC result contains no AVAILABLE records, we can update cached state // and start/end offsets. @@ -561,18 +561,27 @@ public class SharePartition { } long nextFetchOffset = -1; - long gapStartOffset = isInitialReadGapOffsetWindowActive() ? initialReadGapOffset.gapStartOffset() : -1; + long gapStartOffset = isPersisterReadGapWindowActive() ? persisterReadResultGapWindow.gapStartOffset() : -1; for (Map.Entry entry : cachedState.entrySet()) { // Check if there exists any gap in the in-flight batch which needs to be fetched. If - // initialReadGapOffset's endOffset is equal to the share partition's endOffset, then + // gapWindow's endOffset is equal to the share partition's endOffset, then // only the initial gaps should be considered. Once share partition's endOffset is past // initial read end offset then all gaps are anyway fetched. - if (isInitialReadGapOffsetWindowActive()) { + if (isPersisterReadGapWindowActive()) { if (entry.getKey() > gapStartOffset) { nextFetchOffset = gapStartOffset; break; } - gapStartOffset = entry.getValue().lastOffset() + 1; + // If the gapStartOffset is already past the last offset of the in-flight batch, + // then do not consider this batch for finding the next fetch offset. For example, + // consider during initialization, the gapWindow is set to 5 and the + // first cached batch is 15-18. First read will happen at offset 5 and say the data + // fetched is [5-6], now next fetch offset should be 7. This works fine but say + // subsequent read returns batch 8-11, and the gapStartOffset will be 12. Without + // the max check, the next fetch offset returned will be 7 which is incorrect. + // The natural gaps for which no data is available shall be considered hence + // take the max of the gapStartOffset and the last offset of the in-flight batch. + gapStartOffset = Math.max(entry.getValue().lastOffset() + 1, gapStartOffset); } // Check if the state is maintained per offset or batch. If the offsetState @@ -699,16 +708,33 @@ public class SharePartition { // Find the floor batch record for the request batch. The request batch could be // for a subset of the in-flight batch i.e. cached batch of offset 10-14 and request batch - // of 12-13. Hence, floor entry is fetched to find the sub-map. + // of 12-13. Hence, floor entry is fetched to find the sub-map. Secondly, when the share + // partition is initialized with persisted state, the start offset might be moved to a later + // offset. In such case, the first batch base offset might be less than the start offset. Map.Entry floorEntry = cachedState.floorEntry(baseOffset); - // We might find a batch with floor entry but not necessarily that batch has an overlap, - // if the request batch base offset is ahead of last offset from floor entry i.e. cached - // batch of 10-14 and request batch of 15-18, though floor entry is found but no overlap. - // Such scenario will be handled in the next step when considering the subMap. However, - // if the floor entry is found and the request batch base offset is within the floor entry - // then adjust the base offset to the floor entry so that acquire method can still work on - // previously cached batch boundaries. - if (floorEntry != null && floorEntry.getValue().lastOffset() >= baseOffset) { + if (floorEntry == null) { + // The initialize method check that there couldn't be any batches prior to the start offset. + // And once share partition starts fetching records, it will always fetch records, at least, + // from the start offset, but there could be cases where the batch base offset is prior + // to the start offset. This can happen when the share partition is initialized with + // partial persisted state and moved start offset i.e. start offset is not the batch's + // first offset. In such case, we need to adjust the base offset to the start offset. + // It's safe to adjust the base offset to the start offset when there isn't any floor + // i.e. no cached batches available prior to the request batch base offset. Hence, + // check for the floor entry and adjust the base offset accordingly. + if (baseOffset < startOffset) { + log.info("Adjusting base offset for the fetch as it's prior to start offset: {}-{}" + + "from {} to {}", groupId, topicIdPartition, baseOffset, startOffset); + baseOffset = startOffset; + } + } else if (floorEntry.getValue().lastOffset() >= baseOffset) { + // We might find a batch with floor entry but not necessarily that batch has an overlap, + // if the request batch base offset is ahead of last offset from floor entry i.e. cached + // batch of 10-14 and request batch of 15-18, though floor entry is found but no overlap. + // Such scenario will be handled in the next step when considering the subMap. However, + // if the floor entry is found and the request batch base offset is within the floor entry + // then adjust the base offset to the floor entry so that acquire method can still work on + // previously cached batch boundaries. baseOffset = floorEntry.getKey(); } // Validate if the fetch records are already part of existing batches and if available. @@ -743,10 +769,10 @@ public class SharePartition { } InFlightBatch inFlightBatch = entry.getValue(); - // If the initialReadGapOffset window is active, we need to treat the gaps in between the window as + // If the gapWindow window is active, we need to treat the gaps in between the window as // acquirable. Once the window is inactive (when we have acquired all the gaps inside the window), // the remaining gaps are natural (data does not exist at those offsets) and we need not acquire them. - if (isInitialReadGapOffsetWindowActive()) { + if (isPersisterReadGapWindowActive()) { // If nextBatchStartOffset is less than the key of the entry, this means the fetch happened for a gap in the cachedState. // Thus, a new batch needs to be acquired for the gap. if (maybeGapStartOffset < entry.getKey()) { @@ -755,7 +781,8 @@ public class SharePartition { result.addAll(shareAcquiredRecords.acquiredRecords()); acquiredCount += shareAcquiredRecords.count(); } - // Set nextBatchStartOffset as the last offset of the current in-flight batch + 1 + // Set nextBatchStartOffset as the last offset of the current in-flight batch + 1. + // Hence, after the loop iteration the next gap can be considered. maybeGapStartOffset = inFlightBatch.lastOffset() + 1; // If the acquired count is equal to the max fetch records then break the loop. if (acquiredCount >= maxRecordsToAcquire) { @@ -831,7 +858,7 @@ public class SharePartition { acquiredCount += shareAcquiredRecords.count(); } if (!result.isEmpty()) { - maybeUpdateReadGapFetchOffset(result.get(result.size() - 1).lastOffset() + 1); + maybeUpdatePersisterGapWindowStartOffset(result.get(result.size() - 1).lastOffset() + 1); return maybeFilterAbortedTransactionalAcquiredRecords(fetchPartitionData, isolationLevel, new ShareAcquiredRecords(result, acquiredCount)); } return new ShareAcquiredRecords(result, acquiredCount); @@ -1057,10 +1084,24 @@ public class SharePartition { /** * Updates the cached state, start and end offsets of the share partition as per the new log * start offset. The method is called when the log start offset is moved for the share partition. + *

+ * This method only archives the available records in the cached state that are before the new log + * start offset. It does not persist the archived state batches to the persister, rather it + * updates the cached state and offsets to reflect the new log start offset. The state in persister + * will be updated lazily during the acknowledge/release records API calls or acquisition lock timeout. + *

+ * The AVAILABLE state records can either have ongoing state transition or not. Hence, the archive + * records method will update the state of the records to ARCHIVED and set the terminal state flag + * hence if the transition is rolled back then the state will not be AVAILABLE again. However, + * the ACQUIRED state records will not be archived as they are still in-flight and acknowledge + * method also do not allow the state update for any offsets post the log start offset, hence those + * records will only be archived once acquisition lock timeout occurs. * * @param logStartOffset The new log start offset. */ void updateCacheAndOffsets(long logStartOffset) { + log.debug("Updating cached states for share partition: {}-{} with new log start offset: {}", + groupId, topicIdPartition, logStartOffset); lock.writeLock().lock(); try { if (logStartOffset <= startOffset) { @@ -1202,11 +1243,11 @@ public class SharePartition { } inFlightBatch.maybeInitializeOffsetStateUpdate(); } - isAnyOffsetArchived = isAnyOffsetArchived || archivePerOffsetBatchRecords(inFlightBatch, startOffset, endOffset - 1, initialState); + isAnyOffsetArchived = archivePerOffsetBatchRecords(inFlightBatch, startOffset, endOffset - 1, initialState) || isAnyOffsetArchived; continue; } // The in-flight batch is a full match hence change the state of the complete batch. - isAnyBatchArchived = isAnyBatchArchived || archiveCompleteBatch(inFlightBatch, initialState); + isAnyBatchArchived = archiveCompleteBatch(inFlightBatch, initialState) || isAnyBatchArchived; } return isAnyOffsetArchived || isAnyBatchArchived; } finally { @@ -1428,16 +1469,20 @@ public class SharePartition { } // Method to reduce the window that tracks gaps in the cachedState - private void maybeUpdateReadGapFetchOffset(long offset) { + private void maybeUpdatePersisterGapWindowStartOffset(long offset) { lock.writeLock().lock(); try { - if (initialReadGapOffset != null) { - if (initialReadGapOffset.endOffset() == endOffset) { - initialReadGapOffset.gapStartOffset(offset); + if (persisterReadResultGapWindow != null) { + // When last cached batch for persister's read gap window is acquired, then endOffset is + // same as the gapWindow's endOffset, but the gap offset to update in the method call + // is endOffset + 1. Hence, do not update the gap start offset if the request offset + // is ahead of the endOffset. + if (persisterReadResultGapWindow.endOffset() == endOffset && offset <= persisterReadResultGapWindow.endOffset()) { + persisterReadResultGapWindow.gapStartOffset(offset); } else { - // The initial read gap offset is not valid anymore as the end offset has moved - // beyond the initial read gap offset. Hence, reset the initial read gap offset. - initialReadGapOffset = null; + // The persister's read gap window is not valid anymore as the end offset has moved + // beyond the read gap window's endOffset. Hence, set the gap window to null. + persisterReadResultGapWindow = null; } } } finally { @@ -1445,6 +1490,15 @@ public class SharePartition { } } + /** + * The method calculates the last offset and maximum records to acquire. The adjustment is needed + * to ensure that the records acquired do not exceed the maximum in-flight messages limit. + * + * @param fetchOffset The offset from which the records are fetched. + * @param maxFetchRecords The maximum number of records to acquire. + * @param lastOffset The last offset to acquire records to, which is the last offset of the fetched batch. + * @return LastOffsetAndMaxRecords object, containing the last offset to acquire and the maximum records to acquire. + */ private LastOffsetAndMaxRecords lastOffsetAndMaxRecordsToAcquire(long fetchOffset, int maxFetchRecords, long lastOffset) { // There can always be records fetched exceeding the max in-flight messages limit. Hence, // we need to check if the share partition has reached the max in-flight messages limit @@ -1512,6 +1566,20 @@ public class SharePartition { // which falls under the max messages limit. As the max fetch records is the soft // limit, the last offset can be higher than the max messages. lastAcquiredOffset = lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + maxFetchRecords - 1); + // If the initial read gap offset window is active then it's not guaranteed that the + // batches align on batch boundaries. Hence, reset to last offset itself if the batch's + // last offset is greater than the last offset for acquisition, else there could be + // a situation where the batch overlaps with the initial read gap offset window batch. + // For example, if the initial read gap offset window is 10-30 i.e. gapWindow's + // startOffset is 10 and endOffset is 30, and the first persister's read batch is 15-30. + // Say first fetched batch from log is 10-30 and maxFetchRecords is 1, then the lastOffset + // in this method call would be 14. As the maxFetchRecords is lesser than the batch, + // hence last batch offset for request offset is fetched. In this example it will + // be 30, hence check if the initial read gap offset window is active and the last acquired + // offset should be adjusted to 14 instead of 30. + if (isPersisterReadGapWindowActive() && lastAcquiredOffset > lastOffset) { + lastAcquiredOffset = lastOffset; + } } // Create batches of acquired records. @@ -1528,7 +1596,7 @@ public class SharePartition { if (lastAcquiredOffset > endOffset) { endOffset = lastAcquiredOffset; } - maybeUpdateReadGapFetchOffset(lastAcquiredOffset + 1); + maybeUpdatePersisterGapWindowStartOffset(lastAcquiredOffset + 1); return new ShareAcquiredRecords(acquiredRecords, (int) (lastAcquiredOffset - firstAcquiredOffset + 1)); } finally { lock.writeLock().unlock(); @@ -2135,15 +2203,15 @@ public class SharePartition { // If the lastOffsetAcknowledged is equal to the last offset of entry, then the entire batch can potentially be removed. if (lastOffsetAcknowledged == entry.getValue().lastOffset()) { startOffset = cachedState.higherKey(lastOffsetAcknowledged); - if (isInitialReadGapOffsetWindowActive()) { + if (isPersisterReadGapWindowActive()) { // This case will arise if we have a situation where there is an acquirable gap after the lastOffsetAcknowledged. // Ex, the cachedState has following state batches -> {(0, 10), (11, 20), (31,40)} and all these batches are acked. - // There is a gap from 21 to 30. Let the initialReadGapOffset.gapStartOffset be 21. In this case, + // There is a gap from 21 to 30. Let the gapWindow's gapStartOffset be 21. In this case, // lastOffsetAcknowledged will be 20, but we cannot simply move the start offset to the first offset // of next cachedState batch (next cachedState batch is 31 to 40). There is an acquirable gap in between (21 to 30) - // and The startOffset should be at 21. Hence, we set startOffset to the minimum of initialReadGapOffset.gapStartOffset + // and The startOffset should be at 21. Hence, we set startOffset to the minimum of gapWindow.gapStartOffset // and higher key of lastOffsetAcknowledged - startOffset = Math.min(initialReadGapOffset.gapStartOffset(), startOffset); + startOffset = Math.min(persisterReadResultGapWindow.gapStartOffset(), startOffset); } lastKeyToRemove = entry.getKey(); } else { @@ -2208,8 +2276,8 @@ public class SharePartition { return isRecordStateAcknowledged(startOffsetState); } - private boolean isInitialReadGapOffsetWindowActive() { - return initialReadGapOffset != null && initialReadGapOffset.endOffset() == endOffset; + private boolean isPersisterReadGapWindowActive() { + return persisterReadResultGapWindow != null && persisterReadResultGapWindow.endOffset() == endOffset; } /** @@ -2232,7 +2300,7 @@ public class SharePartition { for (NavigableMap.Entry entry : cachedState.entrySet()) { InFlightBatch inFlightBatch = entry.getValue(); - if (isInitialReadGapOffsetWindowActive() && inFlightBatch.lastOffset() >= initialReadGapOffset.gapStartOffset()) { + if (isPersisterReadGapWindowActive() && inFlightBatch.lastOffset() >= persisterReadResultGapWindow.gapStartOffset()) { return lastOffsetAcknowledged; } @@ -2797,8 +2865,8 @@ public class SharePartition { } // Visible for testing - InitialReadGapOffset initialReadGapOffset() { - return initialReadGapOffset; + GapWindow persisterReadResultGapWindow() { + return persisterReadResultGapWindow; } // Visible for testing. @@ -2807,17 +2875,17 @@ public class SharePartition { } /** - * The InitialReadGapOffset class is used to record the gap start and end offset of the probable gaps + * The GapWindow class is used to record the gap start and end offset of the probable gaps * of available records which are neither known to Persister nor to SharePartition. Share Partition * will use this information to determine the next fetch offset and should try to fetch the records * in the gap. */ // Visible for Testing - static class InitialReadGapOffset { + static class GapWindow { private final long endOffset; private long gapStartOffset; - InitialReadGapOffset(long endOffset, long gapStartOffset) { + GapWindow(long endOffset, long gapStartOffset) { this.endOffset = endOffset; this.gapStartOffset = gapStartOffset; } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala index 75baa98da15..a3e9eacb66f 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala @@ -115,7 +115,13 @@ object TransactionLog { val version = buffer.getShort if (version >= TransactionLogValue.LOWEST_SUPPORTED_VERSION && version <= TransactionLogValue.HIGHEST_SUPPORTED_VERSION) { val value = new TransactionLogValue(new ByteBufferAccessor(buffer), version) - val transactionMetadata = new TransactionMetadata( + val state = TransactionState.fromId(value.transactionStatus) + val tps: util.Set[TopicPartition] = new util.HashSet[TopicPartition]() + if (!state.equals(TransactionState.EMPTY)) + value.transactionPartitions.forEach(partitionsSchema => { + partitionsSchema.partitionIds.forEach(partitionId => tps.add(new TopicPartition(partitionsSchema.topic, partitionId.intValue()))) + }) + Some(new TransactionMetadata( transactionalId, value.producerId, value.previousProducerId, @@ -123,20 +129,11 @@ object TransactionLog { value.producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, value.transactionTimeoutMs, - TransactionState.fromId(value.transactionStatus), - util.Set.of(), + state, + tps, value.transactionStartTimestampMs, value.transactionLastUpdateTimestampMs, - TransactionVersion.fromFeatureLevel(value.clientTransactionVersion)) - - if (!transactionMetadata.state.equals(TransactionState.EMPTY)) - value.transactionPartitions.forEach(partitionsSchema => { - transactionMetadata.addPartitions(partitionsSchema.partitionIds - .stream - .map(partitionId => new TopicPartition(partitionsSchema.topic, partitionId.intValue())) - .toList) - }) - Some(transactionMetadata) + TransactionVersion.fromFeatureLevel(value.clientTransactionVersion))) } else throw new IllegalStateException(s"Unknown version $version from the transaction log message value") } } diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index c64218246b3..124a4c7b78f 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -239,7 +239,7 @@ object DynamicBrokerConfig { } } val configHandler = new BrokerConfigHandler(config, quotaManagers) - configHandler.processConfigChanges("", dynamicPerBrokerConfigs) + configHandler.processConfigChanges("", dynamicDefaultConfigs) configHandler.processConfigChanges(config.brokerId.toString, dynamicPerBrokerConfigs) } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 496e50208db..070b3e544a6 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -172,7 +172,8 @@ object ReplicaManager { ListOffsetsRequest.LATEST_TIMESTAMP -> 1.toShort, ListOffsetsRequest.MAX_TIMESTAMP -> 7.toShort, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP -> 8.toShort, - ListOffsetsRequest.LATEST_TIERED_TIMESTAMP -> 9.toShort + ListOffsetsRequest.LATEST_TIERED_TIMESTAMP -> 9.toShort, + ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP -> 11.toShort ) def createLogReadResult(highWatermark: Long, @@ -788,7 +789,11 @@ class ReplicaManager(val config: KafkaConfig, hasCustomErrorMessage = customException.isDefined ) } - val entriesWithoutErrorsPerPartition = entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) } + // In non-transaction paths, errorResults is typically empty, so we can + // directly use entriesPerPartition instead of creating a new filtered collection + val entriesWithoutErrorsPerPartition = + if (errorResults.nonEmpty) entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) } + else entriesPerPartition val preAppendPartitionResponses = buildProducePartitionStatus(errorResults).map { case (k, status) => k -> status.responseStatus } diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index a47b9fd4d47..2b3183be80b 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -25,7 +25,6 @@ import kafka.network.SocketServer import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager, RaftManager} import kafka.server.{KafkaConfig, KafkaRequestHandlerPool} import kafka.utils.{CoreUtils, Logging} -import org.apache.kafka.common.errors.InvalidConfigurationException import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing @@ -115,8 +114,8 @@ class TestRaftServer( workloadGenerator = new RaftWorkloadGenerator( raftManager, time, - recordsPerSec = 20000, - recordSize = 256 + recordsPerSec = throughput, + recordSize = recordSize ) val requestHandler = new TestRaftRequestHandler( @@ -428,7 +427,7 @@ object TestRaftServer extends Logging { } private class TestRaftServerOptions(args: Array[String]) extends CommandDefaultOptions(args) { - val configOpt: OptionSpec[String] = parser.accepts("config", "Required configured file") + val configOpt: OptionSpec[String] = parser.accepts("config", "REQUIRED: The configured file") .withRequiredArg .describedAs("filename") .ofType(classOf[String]) @@ -446,12 +445,14 @@ object TestRaftServer extends Logging { .ofType(classOf[Int]) .defaultsTo(256) - val directoryId: OptionSpec[String] = parser.accepts("replica-directory-id", "The directory id of the replica") + val directoryId: OptionSpec[String] = parser.accepts("replica-directory-id", "REQUIRED: The directory id of the replica") .withRequiredArg .describedAs("directory id") .ofType(classOf[String]) options = parser.parse(args : _*) + + def checkArgs(): Unit = CommandLineUtils.checkRequiredArgs(parser, options, configOpt, directoryId) } def main(args: Array[String]): Unit = { @@ -459,16 +460,11 @@ object TestRaftServer extends Logging { try { CommandLineUtils.maybePrintHelpOrVersion(opts, "Standalone raft server for performance testing") + opts.checkArgs() val configFile = opts.options.valueOf(opts.configOpt) - if (configFile == null) { - throw new InvalidConfigurationException("Missing configuration file. Should specify with '--config'") - } - val directoryIdAsString = opts.options.valueOf(opts.directoryId) - if (directoryIdAsString == null) { - throw new InvalidConfigurationException("Missing replica directory id. Should specify with --replica-directory-id") - } + val serverProps = Utils.loadProps(configFile) // KafkaConfig requires either `process.roles` or `zookeeper.connect`. Neither are diff --git a/core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java b/core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java deleted file mode 100644 index 28c12cf6bce..00000000000 --- a/core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java +++ /dev/null @@ -1,458 +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 kafka.server.integration; -import kafka.integration.KafkaServerTestHarness; -import kafka.server.KafkaBroker; -import kafka.server.KafkaConfig; -import kafka.utils.Logging; -import kafka.utils.TestUtils; - -import org.apache.kafka.clients.CommonClientConfigs; -import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.admin.AlterConfigOp; -import org.apache.kafka.clients.admin.ConfigEntry; -import org.apache.kafka.clients.admin.FeatureUpdate; -import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.clients.admin.UpdateFeaturesOptions; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.TopicPartitionInfo; -import org.apache.kafka.common.config.ConfigResource; -import org.apache.kafka.common.config.TopicConfig; -import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.server.common.EligibleLeaderReplicasVersion; -import org.apache.kafka.server.common.MetadataVersion; -import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInfo; - -import java.io.File; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.function.BiFunction; -import java.util.stream.Collectors; - -import scala.collection.JavaConverters; -import scala.collection.Seq; -import scala.collection.mutable.HashMap; - -import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; - -public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarness implements Logging { - private String bootstrapServer; - private String testTopicName; - private Admin adminClient; - - @Override - public MetadataVersion metadataVersion() { - return MetadataVersion.IBP_4_0_IV1; - } - - @Override - public Seq generateConfigs() { - List brokerConfigs = new ArrayList<>(); - brokerConfigs.addAll(scala.collection.JavaConverters.seqAsJavaList(TestUtils.createBrokerConfigs( - 5, // The tests require 4 brokers to host the partition. However, we need the 5th broker to handle the admin client requests. - true, - true, - scala.Option.empty(), - scala.Option.empty(), - scala.Option.empty(), - true, - false, - false, - false, - new HashMap<>(), - 1, - false, - 1, - (short) 4, - 0, - false - ))); - List configs = new ArrayList<>(); - for (Properties props : brokerConfigs) { - configs.add(KafkaConfig.fromProps(props)); - } - return JavaConverters.asScalaBuffer(configs).toSeq(); - } - - @BeforeEach - @Override - public void setUp(TestInfo info) { - super.setUp(info); - // create adminClient - Properties props = new Properties(); - bootstrapServer = bootstrapServers(listenerName()); - props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer); - adminClient = Admin.create(props); - adminClient.updateFeatures( - Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, - new FeatureUpdate(EligibleLeaderReplicasVersion.ELRV_1.featureLevel(), FeatureUpdate.UpgradeType.UPGRADE)), - new UpdateFeaturesOptions() - ); - testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(), "ELR-test"); - } - - @AfterEach - public void close() throws Exception { - if (adminClient != null) adminClient.close(); - } - - @Test - public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr() throws ExecutionException, InterruptedException { - adminClient.createTopics( - List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); - TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000); - - ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); - Collection ops = new ArrayList<>(); - ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET)); - Map> configOps = Map.of(configResource, ops); - // alter configs on target cluster - adminClient.incrementalAlterConfigs(configOps).all().get(); - Producer producer = null; - Consumer consumer = null; - try { - TopicDescription testTopicDescription = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName); - TopicPartitionInfo topicPartitionInfo = testTopicDescription.partitions().get(0); - List initialReplicas = topicPartitionInfo.replicas(); - assertEquals(4, topicPartitionInfo.isr().size()); - assertEquals(0, topicPartitionInfo.elr().size()); - assertEquals(0, topicPartitionInfo.lastKnownElr().size()); - - Properties producerProps = new Properties(); - producerProps.putIfAbsent(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - producerProps.putIfAbsent(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer); - // Use Ack=1 for the producer. - producerProps.put(ProducerConfig.ACKS_CONFIG, "1"); - producer = new KafkaProducer(producerProps); - - Properties consumerProps = new Properties(); - consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer); - consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test"); - consumerProps.put(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, "10"); - consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - consumerProps.putIfAbsent(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - consumerProps.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - consumer = new KafkaConsumer<>(consumerProps); - consumer.subscribe(Set.of(testTopicName)); - - producer.send(new ProducerRecord<>(testTopicName, "0", "0")).get(); - waitUntilOneMessageIsConsumed(consumer); - - killBroker(initialReplicas.get(0).id()); - killBroker(initialReplicas.get(1).id()); - - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 2 && elrSize == 1; - }); - - // Now the partition is under min ISR. HWM should not advance. - producer.send(new ProducerRecord<>(testTopicName, "1", "1")).get(); - Thread.sleep(100); - assertEquals(0, consumer.poll(Duration.ofSeconds(1L)).count()); - - // Restore the min ISR and the previous log should be visible. - startBroker(initialReplicas.get(1).id()); - startBroker(initialReplicas.get(0).id()); - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 4 && elrSize == 0; - }); - - waitUntilOneMessageIsConsumed(consumer); - } finally { - restartDeadBrokers(false); - if (consumer != null) consumer.close(); - if (producer != null) producer.close(); - } - } - - void waitUntilOneMessageIsConsumed(Consumer consumer) { - TestUtils.waitUntilTrue( - () -> { - try { - ConsumerRecords record = consumer.poll(Duration.ofMillis(100L)); - return record.count() >= 1; - } catch (Exception e) { - return false; - } - }, - () -> "fail to consume messages", - DEFAULT_MAX_WAIT_MS, 100L - ); - } - - @Test - public void testElrMemberCanBeElected() throws ExecutionException, InterruptedException { - adminClient.createTopics( - List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); - TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000); - - ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); - Collection ops = new ArrayList<>(); - ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET)); - Map> configOps = Map.of(configResource, ops); - // alter configs on target cluster - adminClient.incrementalAlterConfigs(configOps).all().get(); - - try { - TopicDescription testTopicDescription = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName); - TopicPartitionInfo topicPartitionInfo = testTopicDescription.partitions().get(0); - List initialReplicas = topicPartitionInfo.replicas(); - assertEquals(4, topicPartitionInfo.isr().size()); - assertEquals(0, topicPartitionInfo.elr().size()); - assertEquals(0, topicPartitionInfo.lastKnownElr().size()); - - killBroker(initialReplicas.get(0).id()); - killBroker(initialReplicas.get(1).id()); - killBroker(initialReplicas.get(2).id()); - - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 1 && elrSize == 2; - }); - - killBroker(initialReplicas.get(3).id()); - - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 0 && elrSize == 3; - }); - - topicPartitionInfo = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions().get(0); - assertEquals(1, topicPartitionInfo.lastKnownElr().size(), topicPartitionInfo.toString()); - int expectLastKnownLeader = initialReplicas.get(3).id(); - assertEquals(expectLastKnownLeader, topicPartitionInfo.lastKnownElr().get(0).id(), topicPartitionInfo.toString()); - - // At this point, all the replicas are failed and the last know leader is No.3 and 3 members in the ELR. - // Restart one broker of the ELR and it should be the leader. - - int expectLeader = topicPartitionInfo.elr().stream() - .filter(node -> node.id() != expectLastKnownLeader).toList().get(0).id(); - - startBroker(expectLeader); - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 1 && elrSize == 2; - }); - - topicPartitionInfo = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions().get(0); - assertEquals(0, topicPartitionInfo.lastKnownElr().size(), topicPartitionInfo.toString()); - assertEquals(expectLeader, topicPartitionInfo.leader().id(), topicPartitionInfo.toString()); - - // Start another 2 brokers and the ELR fields should be cleaned. - topicPartitionInfo.replicas().stream().filter(node -> node.id() != expectLeader).limit(2) - .forEach(node -> startBroker(node.id())); - - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 3 && elrSize == 0; - }); - - topicPartitionInfo = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions().get(0); - assertEquals(0, topicPartitionInfo.lastKnownElr().size(), topicPartitionInfo.toString()); - assertEquals(expectLeader, topicPartitionInfo.leader().id(), topicPartitionInfo.toString()); - } finally { - restartDeadBrokers(false); - } - } - - @Test - public void testElrMemberShouldBeKickOutWhenUncleanShutdown() throws ExecutionException, InterruptedException { - adminClient.createTopics( - List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); - TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000); - - ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); - Collection ops = new ArrayList<>(); - ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET)); - Map> configOps = Map.of(configResource, ops); - // alter configs on target cluster - adminClient.incrementalAlterConfigs(configOps).all().get(); - - try { - TopicDescription testTopicDescription = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName); - TopicPartitionInfo topicPartitionInfo = testTopicDescription.partitions().get(0); - List initialReplicas = topicPartitionInfo.replicas(); - assertEquals(4, topicPartitionInfo.isr().size()); - assertEquals(0, topicPartitionInfo.elr().size()); - assertEquals(0, topicPartitionInfo.lastKnownElr().size()); - - killBroker(initialReplicas.get(0).id()); - killBroker(initialReplicas.get(1).id()); - killBroker(initialReplicas.get(2).id()); - killBroker(initialReplicas.get(3).id()); - - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 0 && elrSize == 3; - }); - topicPartitionInfo = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions().get(0); - - int brokerToBeUncleanShutdown = topicPartitionInfo.elr().get(0).id(); - KafkaBroker broker = brokers().find(b -> { - return b.config().brokerId() == brokerToBeUncleanShutdown; - }).get(); - Seq dirs = broker.logManager().liveLogDirs(); - assertEquals(1, dirs.size()); - CleanShutdownFileHandler handler = new CleanShutdownFileHandler(dirs.apply(0).toString()); - assertTrue(handler.exists()); - assertDoesNotThrow(() -> handler.delete()); - - // After remove the clean shutdown file, the broker should report unclean shutdown during restart. - startBroker(brokerToBeUncleanShutdown); - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 0 && elrSize == 2; - }); - topicPartitionInfo = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions().get(0); - assertNull(topicPartitionInfo.leader()); - assertEquals(1, topicPartitionInfo.lastKnownElr().size()); - } finally { - restartDeadBrokers(false); - } - } - - /* - This test is only valid for KIP-966 part 1. When the unclean recovery is implemented, it should be removed. - */ - @Test - public void testLastKnownLeaderShouldBeElectedIfEmptyElr() throws ExecutionException, InterruptedException { - adminClient.createTopics( - List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); - TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000); - - ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); - Collection ops = new ArrayList<>(); - ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET)); - Map> configOps = Map.of(configResource, ops); - // alter configs on target cluster - adminClient.incrementalAlterConfigs(configOps).all().get(); - - try { - TopicDescription testTopicDescription = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName); - TopicPartitionInfo topicPartitionInfo = testTopicDescription.partitions().get(0); - List initialReplicas = topicPartitionInfo.replicas(); - assertEquals(4, topicPartitionInfo.isr().size()); - assertEquals(0, topicPartitionInfo.elr().size()); - assertEquals(0, topicPartitionInfo.lastKnownElr().size()); - - killBroker(initialReplicas.get(0).id()); - killBroker(initialReplicas.get(1).id()); - killBroker(initialReplicas.get(2).id()); - killBroker(initialReplicas.get(3).id()); - - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 0 && elrSize == 3; - }); - topicPartitionInfo = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions().get(0); - int lastKnownLeader = topicPartitionInfo.lastKnownElr().get(0).id(); - - Set initialReplicaSet = initialReplicas.stream().map(node -> node.id()).collect(Collectors.toSet()); - brokers().foreach(broker -> { - if (initialReplicaSet.contains(broker.config().brokerId())) { - Seq dirs = broker.logManager().liveLogDirs(); - assertEquals(1, dirs.size()); - CleanShutdownFileHandler handler = new CleanShutdownFileHandler(dirs.apply(0).toString()); - assertDoesNotThrow(() -> handler.delete()); - } - return true; - }); - - - // After remove the clean shutdown file, the broker should report unclean shutdown during restart. - topicPartitionInfo.replicas().forEach(replica -> { - if (replica.id() != lastKnownLeader) startBroker(replica.id()); - }); - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize == 0 && elrSize == 1; - }); - topicPartitionInfo = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions().get(0); - assertNull(topicPartitionInfo.leader()); - assertEquals(1, topicPartitionInfo.lastKnownElr().size()); - - // Now if the last known leader goes through unclean shutdown, it will still be elected. - startBroker(lastKnownLeader); - waitForIsrAndElr((isrSize, elrSize) -> { - return isrSize > 0 && elrSize == 0; - }); - - TestUtils.waitUntilTrue( - () -> { - try { - TopicPartitionInfo partition = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions().get(0); - if (partition.leader() == null) return false; - return partition.lastKnownElr().isEmpty() && partition.elr().isEmpty() && partition.leader().id() == lastKnownLeader; - } catch (Exception e) { - return false; - } - }, - () -> String.format("Partition metadata for %s is not correct", testTopicName), - DEFAULT_MAX_WAIT_MS, 100L - ); - } finally { - restartDeadBrokers(false); - } - } - - void waitForIsrAndElr(BiFunction isIsrAndElrSizeSatisfied) { - TestUtils.waitUntilTrue( - () -> { - try { - TopicDescription topicDescription = adminClient.describeTopics(List.of(testTopicName)) - .allTopicNames().get().get(testTopicName); - TopicPartitionInfo partition = topicDescription.partitions().get(0); - return isIsrAndElrSizeSatisfied.apply(partition.isr().size(), partition.elr().size()); - } catch (Exception e) { - return false; - } - }, - () -> String.format("Partition metadata for %s is not propagated", testTopicName), - DEFAULT_MAX_WAIT_MS, 100L); - } -} diff --git a/core/src/test/java/kafka/server/share/SharePartitionTest.java b/core/src/test/java/kafka/server/share/SharePartitionTest.java index 30b49e17b16..47e214a716f 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionTest.java @@ -17,6 +17,7 @@ package kafka.server.share; import kafka.server.ReplicaManager; +import kafka.server.share.SharePartition.GapWindow; import kafka.server.share.SharePartition.SharePartitionState; import kafka.server.share.SharePartitionManager.SharePartitionListener; @@ -965,11 +966,11 @@ public class SharePartitionTest { assertEquals(3, sharePartition.cachedState().get(21L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(21L).offsetState()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - assertEquals(10, initialReadGapOffset.gapStartOffset()); - assertEquals(30, initialReadGapOffset.endOffset()); + assertEquals(10, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(30, persisterReadResultGapWindow.endOffset()); } @Test @@ -1010,11 +1011,11 @@ public class SharePartitionTest { assertEquals(3, sharePartition.cachedState().get(30L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(30L).offsetState()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - assertEquals(10, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + assertEquals(10, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -1051,11 +1052,11 @@ public class SharePartitionTest { assertEquals(3, sharePartition.cachedState().get(30L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(30L).offsetState()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - assertEquals(21, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + assertEquals(21, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -1082,10 +1083,676 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(31, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); - // Since there are no gaps present in the readState response, initialReadGapOffset should be null - assertNull(initialReadGapOffset); + // Since there are no gaps present in the readState response, persisterReadResultGapWindow should be null + assertNull(persisterReadResultGapWindow); + } + + @Test + public void testMaybeInitializeAndAcquire() { + Persister persister = Mockito.mock(Persister.class); + ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); + Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( + new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of( + PartitionFactory.newPartitionAllData(0, 3, 10L, Errors.NONE.code(), Errors.NONE.message(), + List.of( + new PersisterStateBatch(15L, 18L, RecordState.AVAILABLE.id, (short) 2), + new PersisterStateBatch(20L, 22L, RecordState.ARCHIVED.id, (short) 2), + new PersisterStateBatch(26L, 30L, RecordState.AVAILABLE.id, (short) 3))))))); + Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult)); + SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); + + CompletableFuture result = sharePartition.maybeInitialize(); + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + + assertEquals(SharePartitionState.ACTIVE, sharePartition.partitionState()); + assertEquals(3, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.startOffset()); + assertEquals(30, sharePartition.endOffset()); + assertEquals(10, sharePartition.nextFetchOffset()); + + assertEquals(18, sharePartition.cachedState().get(15L).lastOffset()); + assertEquals(22, sharePartition.cachedState().get(20L).lastOffset()); + assertEquals(30, sharePartition.cachedState().get(26L).lastOffset()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); + + // Create a single batch record that covers the entire range from 10 to 30 of initial read gap. + // The records in the batch are from 10 to 49. + MemoryRecords records = memoryRecords(40, 10); + // Set max fetch records to 1, records will be acquired till the first gap is encountered. + List acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 1, + 10, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 5); + + assertArrayEquals(expectedAcquiredRecord(10, 14, 1).toArray(), acquiredRecordsList.toArray()); + assertEquals(15, sharePartition.nextFetchOffset()); + assertEquals(4, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.cachedState().get(10L).firstOffset()); + assertEquals(14, sharePartition.cachedState().get(10L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(10L).batchState()); + assertEquals(1, sharePartition.cachedState().get(10L).batchDeliveryCount()); + assertNull(sharePartition.cachedState().get(10L).offsetState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(15L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + + // Send the same batch again to acquire the next set of records. + acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 10, + 15, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 13); + + List expectedAcquiredRecords = new ArrayList<>(expectedAcquiredRecord(15, 18, 3)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(19, 19, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(23, 25, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(26, 30, 4)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(31, sharePartition.nextFetchOffset()); + assertEquals(6, sharePartition.cachedState().size()); + assertEquals(19, sharePartition.cachedState().get(19L).firstOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(19L).batchState()); + assertEquals(1, sharePartition.cachedState().get(19L).batchDeliveryCount()); + assertNull(sharePartition.cachedState().get(19L).offsetState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(23, sharePartition.cachedState().get(23L).firstOffset()); + assertEquals(25, sharePartition.cachedState().get(23L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(23L).batchState()); + assertEquals(1, sharePartition.cachedState().get(23L).batchDeliveryCount()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(26L).batchState()); + assertEquals(30L, sharePartition.endOffset()); + // As all the gaps are now filled, the persisterReadResultGapWindow should be null. + assertNull(sharePartition.persisterReadResultGapWindow()); + + // Now initial read gap is filled, so the complete batch can be acquired despite max fetch records being 1. + acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 1, + 31, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 19); + + assertArrayEquals(expectedAcquiredRecord(31, 49, 1).toArray(), acquiredRecordsList.toArray()); + assertEquals(50, sharePartition.nextFetchOffset()); + assertEquals(7, sharePartition.cachedState().size()); + assertEquals(31, sharePartition.cachedState().get(31L).firstOffset()); + assertEquals(49, sharePartition.cachedState().get(31L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(31L).batchState()); + assertEquals(1, sharePartition.cachedState().get(31L).batchDeliveryCount()); + assertNull(sharePartition.cachedState().get(31L).offsetState()); + assertEquals(49L, sharePartition.endOffset()); + } + + @Test + public void testMaybeInitializeAndAcquireWithHigherMaxFetchRecords() { + Persister persister = Mockito.mock(Persister.class); + ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); + Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( + new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of( + PartitionFactory.newPartitionAllData(0, 3, 10L, Errors.NONE.code(), Errors.NONE.message(), + List.of( + new PersisterStateBatch(15L, 18L, RecordState.AVAILABLE.id, (short) 2), + new PersisterStateBatch(20L, 22L, RecordState.ARCHIVED.id, (short) 2), + new PersisterStateBatch(26L, 30L, RecordState.AVAILABLE.id, (short) 3))))))); + Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult)); + SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); + + CompletableFuture result = sharePartition.maybeInitialize(); + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + + assertEquals(SharePartitionState.ACTIVE, sharePartition.partitionState()); + assertEquals(3, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.startOffset()); + assertEquals(30, sharePartition.endOffset()); + assertEquals(10, sharePartition.nextFetchOffset()); + + assertEquals(18, sharePartition.cachedState().get(15L).lastOffset()); + assertEquals(22, sharePartition.cachedState().get(20L).lastOffset()); + assertEquals(30, sharePartition.cachedState().get(26L).lastOffset()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); + + // Create a single batch record that covers the entire range from 10 to 30 of initial read gap. + // The records in the batch are from 10 to 49. + MemoryRecords records = memoryRecords(40, 10); + // Set max fetch records to 500, all records should be acquired. + List acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 500, + 10, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 37); + + List expectedAcquiredRecords = new ArrayList<>(expectedAcquiredRecord(10, 14, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(15, 18, 3)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(19, 19, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(23, 25, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(26, 30, 4)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(31, 49, 1)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(50, sharePartition.nextFetchOffset()); + assertEquals(7, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.cachedState().get(10L).firstOffset()); + assertEquals(14, sharePartition.cachedState().get(10L).lastOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).firstOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).lastOffset()); + assertEquals(23, sharePartition.cachedState().get(23L).firstOffset()); + assertEquals(25, sharePartition.cachedState().get(23L).lastOffset()); + assertEquals(31, sharePartition.cachedState().get(31L).firstOffset()); + assertEquals(49, sharePartition.cachedState().get(31L).lastOffset()); + + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(10L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(19L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(23L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(26L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(31L).batchState()); + assertEquals(49L, sharePartition.endOffset()); + // As all the gaps are now filled, the persisterReadResultGapWindow should be null. + assertNull(sharePartition.persisterReadResultGapWindow()); + } + + @Test + public void testMaybeInitializeAndAcquireWithFetchBatchLastOffsetWithinCachedBatch() { + Persister persister = Mockito.mock(Persister.class); + ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); + Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( + new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of( + PartitionFactory.newPartitionAllData(0, 3, 10L, Errors.NONE.code(), Errors.NONE.message(), + List.of( + new PersisterStateBatch(15L, 18L, RecordState.AVAILABLE.id, (short) 2), + new PersisterStateBatch(20L, 22L, RecordState.ARCHIVED.id, (short) 2), + new PersisterStateBatch(26L, 30L, RecordState.AVAILABLE.id, (short) 3))))))); + Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult)); + SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); + + CompletableFuture result = sharePartition.maybeInitialize(); + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + + assertEquals(SharePartitionState.ACTIVE, sharePartition.partitionState()); + assertEquals(3, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.startOffset()); + assertEquals(30, sharePartition.endOffset()); + assertEquals(10, sharePartition.nextFetchOffset()); + + assertEquals(18, sharePartition.cachedState().get(15L).lastOffset()); + assertEquals(22, sharePartition.cachedState().get(20L).lastOffset()); + assertEquals(30, sharePartition.cachedState().get(26L).lastOffset()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); + + // Create a single batch record that ends in between the cached batch and the fetch offset is + // post startOffset. + MemoryRecords records = memoryRecords(16, 12); + // Set max fetch records to 500, records should be acquired till the last offset of the fetched batch. + List acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 500, + 10, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 13); + + List expectedAcquiredRecords = new ArrayList<>(expectedAcquiredRecord(12, 14, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(15, 18, 3)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(19, 19, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(23, 25, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecords(26, 27, 4)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(28, sharePartition.nextFetchOffset()); + assertEquals(6, sharePartition.cachedState().size()); + assertEquals(12, sharePartition.cachedState().get(12L).firstOffset()); + assertEquals(14, sharePartition.cachedState().get(12L).lastOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).firstOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).lastOffset()); + assertEquals(23, sharePartition.cachedState().get(23L).firstOffset()); + assertEquals(25, sharePartition.cachedState().get(23L).lastOffset()); + + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(12L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(19L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(23L).batchState()); + assertThrows(IllegalStateException.class, () -> sharePartition.cachedState().get(26L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(26L).offsetState().get(26L).state()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(26L).offsetState().get(27L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).offsetState().get(28L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).offsetState().get(29L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).offsetState().get(30L).state()); + assertEquals(30L, sharePartition.endOffset()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(28L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + } + + @Test + public void testMaybeInitializeAndAcquireWithFetchBatchPriorStartOffset() { + Persister persister = Mockito.mock(Persister.class); + ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); + Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( + new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of( + PartitionFactory.newPartitionAllData(0, 3, 10L, Errors.NONE.code(), Errors.NONE.message(), + List.of( + new PersisterStateBatch(15L, 18L, RecordState.AVAILABLE.id, (short) 2), + new PersisterStateBatch(20L, 22L, RecordState.ARCHIVED.id, (short) 2), + new PersisterStateBatch(26L, 30L, RecordState.AVAILABLE.id, (short) 3))))))); + Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult)); + SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); + + CompletableFuture result = sharePartition.maybeInitialize(); + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + + assertEquals(SharePartitionState.ACTIVE, sharePartition.partitionState()); + assertEquals(3, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.startOffset()); + assertEquals(30, sharePartition.endOffset()); + assertEquals(10, sharePartition.nextFetchOffset()); + + assertEquals(18, sharePartition.cachedState().get(15L).lastOffset()); + assertEquals(22, sharePartition.cachedState().get(20L).lastOffset()); + assertEquals(30, sharePartition.cachedState().get(26L).lastOffset()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); + + // Create a single batch record where first offset is prior startOffset. + MemoryRecords records = memoryRecords(16, 6); + // Set max fetch records to 500, records should be acquired till the last offset of the fetched batch. + List acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 500, + 10, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 10); + + List expectedAcquiredRecords = new ArrayList<>(expectedAcquiredRecord(10, 14, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(15, 18, 3)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(19, 19, 1)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(23, sharePartition.nextFetchOffset()); + assertEquals(5, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.cachedState().get(10L).firstOffset()); + assertEquals(14, sharePartition.cachedState().get(10L).lastOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).firstOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).lastOffset()); + + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(10L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(19L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertEquals(30L, sharePartition.endOffset()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(20L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + } + + @Test + public void testMaybeInitializeAndAcquireWithMultipleBatches() { + Persister persister = Mockito.mock(Persister.class); + ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); + Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( + new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of( + PartitionFactory.newPartitionAllData(0, 3, 5L, Errors.NONE.code(), Errors.NONE.message(), + List.of( + new PersisterStateBatch(15L, 18L, RecordState.AVAILABLE.id, (short) 2), + new PersisterStateBatch(20L, 22L, RecordState.ARCHIVED.id, (short) 2), + new PersisterStateBatch(26L, 30L, RecordState.AVAILABLE.id, (short) 3))))))); + Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult)); + SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); + + CompletableFuture result = sharePartition.maybeInitialize(); + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + + assertEquals(SharePartitionState.ACTIVE, sharePartition.partitionState()); + assertEquals(3, sharePartition.cachedState().size()); + assertEquals(5, sharePartition.startOffset()); + assertEquals(30, sharePartition.endOffset()); + assertEquals(5, sharePartition.nextFetchOffset()); + + assertEquals(18, sharePartition.cachedState().get(15L).lastOffset()); + assertEquals(22, sharePartition.cachedState().get(20L).lastOffset()); + assertEquals(30, sharePartition.cachedState().get(26L).lastOffset()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(5L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); + + // Create multiple batch records that covers the entire range from 5 to 30 of initial read gap. + // The records in the batch are from 5 to 49. + ByteBuffer buffer = ByteBuffer.allocate(4096); + memoryRecordsBuilder(buffer, 2, 5).close(); + memoryRecordsBuilder(buffer, 1, 8).close(); + memoryRecordsBuilder(buffer, 2, 10).close(); + memoryRecordsBuilder(buffer, 6, 13).close(); + memoryRecordsBuilder(buffer, 3, 19).close(); + memoryRecordsBuilder(buffer, 9, 22).close(); + memoryRecordsBuilder(buffer, 19, 31).close(); + buffer.flip(); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + // Set max fetch records to 1, records will be acquired till the first gap is encountered. + List acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 1, + 5L, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 2); + + assertArrayEquals(expectedAcquiredRecord(5, 6, 1).toArray(), acquiredRecordsList.toArray()); + assertEquals(7, sharePartition.nextFetchOffset()); + assertEquals(4, sharePartition.cachedState().size()); + assertEquals(5, sharePartition.cachedState().get(5L).firstOffset()); + assertEquals(6, sharePartition.cachedState().get(5L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(5L).batchState()); + assertEquals(1, sharePartition.cachedState().get(5L).batchDeliveryCount()); + assertNull(sharePartition.cachedState().get(5L).offsetState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(7L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + + // Remove first batch from the records as the fetch offset has moved forward to 7 offset. + List batch = TestUtils.toList(records.batches()); + records = records.slice(batch.get(0).sizeInBytes(), records.sizeInBytes() - batch.get(0).sizeInBytes()); + // Send the batch again to acquire the next set of records. + acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 3, + 7L, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 4); + + assertArrayEquals(expectedAcquiredRecord(8, 11, 1).toArray(), acquiredRecordsList.toArray()); + assertEquals(12, sharePartition.nextFetchOffset()); + assertEquals(5, sharePartition.cachedState().size()); + assertEquals(8, sharePartition.cachedState().get(8L).firstOffset()); + assertEquals(11, sharePartition.cachedState().get(8L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(8L).batchState()); + assertEquals(1, sharePartition.cachedState().get(8L).batchDeliveryCount()); + assertNull(sharePartition.cachedState().get(8L).offsetState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertEquals(30L, sharePartition.endOffset()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(12L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + + // Remove the next 2 batches from the records as the fetch offset has moved forward to 12 offset. + int size = batch.get(1).sizeInBytes() + batch.get(2).sizeInBytes(); + records = records.slice(size, records.sizeInBytes() - size); + // Send the records with 8 as max fetch records to acquire new and existing cached batches. + acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 8, + 12, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 10); + + List expectedAcquiredRecords = new ArrayList<>(expectedAcquiredRecord(13, 14, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(15, 18, 3)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(19, 19, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(23, 25, 1)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(26, sharePartition.nextFetchOffset()); + assertEquals(8, sharePartition.cachedState().size()); + assertEquals(13, sharePartition.cachedState().get(13L).firstOffset()); + assertEquals(14, sharePartition.cachedState().get(13L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(13L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); + assertEquals(19, sharePartition.cachedState().get(19L).firstOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(19L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(23, sharePartition.cachedState().get(23L).firstOffset()); + assertEquals(25, sharePartition.cachedState().get(23L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(23L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertEquals(30L, sharePartition.endOffset()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(26L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + + // Remove the next 2 batches from the records as the fetch offset has moved forward to 26 offset. + // Do not remove the 5th batch as it's only partially acquired. + size = batch.get(3).sizeInBytes() + batch.get(4).sizeInBytes(); + records = records.slice(size, records.sizeInBytes() - size); + // Send the records with 10 as max fetch records to acquire the existing and till end of the + // fetched data. + acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 10, + 26, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 24); + + expectedAcquiredRecords = new ArrayList<>(expectedAcquiredRecord(26, 30, 4)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(31, 49, 1)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(50, sharePartition.nextFetchOffset()); + assertEquals(9, sharePartition.cachedState().size()); + assertEquals(31, sharePartition.cachedState().get(31L).firstOffset()); + assertEquals(49, sharePartition.cachedState().get(31L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(31L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(26L).batchState()); + assertEquals(49L, sharePartition.endOffset()); + // As all the gaps are now filled, the persisterReadResultGapWindow should be null. + assertNull(sharePartition.persisterReadResultGapWindow()); + } + + @Test + public void testMaybeInitializeAndAcquireWithMultipleBatchesAndLastOffsetWithinCachedBatch() { + Persister persister = Mockito.mock(Persister.class); + ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); + Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( + new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of( + PartitionFactory.newPartitionAllData(0, 3, 5L, Errors.NONE.code(), Errors.NONE.message(), + List.of( + new PersisterStateBatch(15L, 18L, RecordState.AVAILABLE.id, (short) 2), + new PersisterStateBatch(20L, 22L, RecordState.ARCHIVED.id, (short) 2), + new PersisterStateBatch(26L, 30L, RecordState.AVAILABLE.id, (short) 3))))))); + Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult)); + SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); + + CompletableFuture result = sharePartition.maybeInitialize(); + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + + assertEquals(SharePartitionState.ACTIVE, sharePartition.partitionState()); + assertEquals(3, sharePartition.cachedState().size()); + assertEquals(5, sharePartition.startOffset()); + assertEquals(30, sharePartition.endOffset()); + assertEquals(5, sharePartition.nextFetchOffset()); + + assertEquals(18, sharePartition.cachedState().get(15L).lastOffset()); + assertEquals(22, sharePartition.cachedState().get(20L).lastOffset()); + assertEquals(30, sharePartition.cachedState().get(26L).lastOffset()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(5L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); + + // Create multiple batch records that ends in between the cached batch and the fetch offset is + // post startOffset. + ByteBuffer buffer = ByteBuffer.allocate(4096); + memoryRecordsBuilder(buffer, 2, 7).close(); + memoryRecordsBuilder(buffer, 2, 10).close(); + memoryRecordsBuilder(buffer, 6, 13).close(); + // Though 19 offset is a gap but still be acquired. + memoryRecordsBuilder(buffer, 8, 20).close(); + buffer.flip(); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + // Set max fetch records to 500, records should be acquired till the last offset of the fetched batch. + List acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 500, + 5, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 18); + + List expectedAcquiredRecords = new ArrayList<>(expectedAcquiredRecord(7, 14, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(15, 18, 3)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(19, 19, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(23, 25, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecords(26, 27, 4)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(28, sharePartition.nextFetchOffset()); + assertEquals(6, sharePartition.cachedState().size()); + assertEquals(7, sharePartition.cachedState().get(7L).firstOffset()); + assertEquals(14, sharePartition.cachedState().get(7L).lastOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).firstOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).lastOffset()); + assertEquals(23, sharePartition.cachedState().get(23L).firstOffset()); + assertEquals(25, sharePartition.cachedState().get(23L).lastOffset()); + + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(7L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(19L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(23L).batchState()); + assertThrows(IllegalStateException.class, () -> sharePartition.cachedState().get(26L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(26L).offsetState().get(26L).state()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(26L).offsetState().get(27L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).offsetState().get(28L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).offsetState().get(29L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).offsetState().get(30L).state()); + assertEquals(30L, sharePartition.endOffset()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(28L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + } + + @Test + public void testMaybeInitializeAndAcquireWithMultipleBatchesPriorStartOffset() { + Persister persister = Mockito.mock(Persister.class); + ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); + Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( + new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of( + PartitionFactory.newPartitionAllData(0, 3, 10L, Errors.NONE.code(), Errors.NONE.message(), + List.of( + new PersisterStateBatch(15L, 18L, RecordState.AVAILABLE.id, (short) 2), + new PersisterStateBatch(20L, 22L, RecordState.ARCHIVED.id, (short) 2), + new PersisterStateBatch(26L, 30L, RecordState.AVAILABLE.id, (short) 3))))))); + Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult)); + SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); + + CompletableFuture result = sharePartition.maybeInitialize(); + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + + assertEquals(SharePartitionState.ACTIVE, sharePartition.partitionState()); + assertEquals(3, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.startOffset()); + assertEquals(30, sharePartition.endOffset()); + assertEquals(10, sharePartition.nextFetchOffset()); + + assertEquals(18, sharePartition.cachedState().get(15L).lastOffset()); + assertEquals(22, sharePartition.cachedState().get(20L).lastOffset()); + assertEquals(30, sharePartition.cachedState().get(26L).lastOffset()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); + assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); + + // Create multiple batch records where multiple batches base offsets are prior startOffset. + ByteBuffer buffer = ByteBuffer.allocate(4096); + memoryRecordsBuilder(buffer, 2, 3).close(); + memoryRecordsBuilder(buffer, 1, 6).close(); + memoryRecordsBuilder(buffer, 4, 8).close(); + memoryRecordsBuilder(buffer, 10, 13).close(); + buffer.flip(); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + // Set max fetch records to 500, records should be acquired till the last offset of the fetched batch. + List acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + BATCH_SIZE, + 500, + 10, + fetchPartitionData(records), + FETCH_ISOLATION_HWM), + 10); + + List expectedAcquiredRecords = new ArrayList<>(expectedAcquiredRecord(10, 14, 1)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(15, 18, 3)); + expectedAcquiredRecords.addAll(expectedAcquiredRecord(19, 19, 1)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(23, sharePartition.nextFetchOffset()); + assertEquals(5, sharePartition.cachedState().size()); + assertEquals(10, sharePartition.cachedState().get(10L).firstOffset()); + assertEquals(14, sharePartition.cachedState().get(10L).lastOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).firstOffset()); + assertEquals(19, sharePartition.cachedState().get(19L).lastOffset()); + + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(10L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(19L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); + assertEquals(30L, sharePartition.endOffset()); + assertNotNull(sharePartition.persisterReadResultGapWindow()); + assertEquals(20L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); } @Test @@ -2368,12 +3035,12 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(16, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - // After records are acquired, the initialReadGapOffset should be updated - assertEquals(16, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + // After records are acquired, the persisterReadResultGapWindow should be updated + assertEquals(16, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -2407,12 +3074,12 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(41, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - // After records are acquired, the initialReadGapOffset should be updated - assertEquals(21, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + // After records are acquired, the persisterReadResultGapWindow should be updated + assertEquals(21, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -2460,12 +3127,12 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(26, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - // After records are acquired, the initialReadGapOffset should be updated - assertEquals(26, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + // After records are acquired, the persisterReadResultGapWindow should be updated + assertEquals(26, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -2504,12 +3171,12 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(26, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - // After records are acquired, the initialReadGapOffset should be updated - assertEquals(26, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + // After records are acquired, the persisterReadResultGapWindow should be updated + assertEquals(26, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -2561,12 +3228,12 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(86, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - // After records are acquired, the initialReadGapOffset should be updated - assertEquals(86, initialReadGapOffset.gapStartOffset()); - assertEquals(90, initialReadGapOffset.endOffset()); + // After records are acquired, the persisterReadResultGapWindow should be updated + assertEquals(86, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(90, persisterReadResultGapWindow.endOffset()); } @Test @@ -2605,12 +3272,12 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(31, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - // After records are acquired, the initialReadGapOffset should be updated - assertEquals(31, initialReadGapOffset.gapStartOffset()); - assertEquals(70, initialReadGapOffset.endOffset()); + // After records are acquired, the persisterReadResultGapWindow should be updated + assertEquals(31, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(70, persisterReadResultGapWindow.endOffset()); } @Test @@ -2656,12 +3323,12 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(76, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - // After records are acquired, the initialReadGapOffset should be updated - assertEquals(76, initialReadGapOffset.gapStartOffset()); - assertEquals(90, initialReadGapOffset.endOffset()); + // After records are acquired, the persisterReadResultGapWindow should be updated + assertEquals(76, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(90, persisterReadResultGapWindow.endOffset()); } @@ -2709,11 +3376,11 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(27, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - assertEquals(27, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + assertEquals(27, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -2758,11 +3425,11 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(21, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - assertEquals(21, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + assertEquals(21, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -2807,11 +3474,11 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(21, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - assertEquals(21, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + assertEquals(21, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -2859,8 +3526,8 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(51, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNull(persisterReadResultGapWindow); } @Test @@ -2903,8 +3570,8 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(61, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNull(persisterReadResultGapWindow); } @Test @@ -2949,8 +3616,8 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(61, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNull(persisterReadResultGapWindow); } @Test @@ -2998,8 +3665,8 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(61, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNull(persisterReadResultGapWindow); } @Test @@ -3039,11 +3706,11 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(41, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - assertEquals(31, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + assertEquals(31, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); // Fetching from the nextFetchOffset so that endOffset moves ahead records = memoryRecords(15, 41); @@ -3059,9 +3726,9 @@ public class SharePartitionTest { assertEquals(3, sharePartition.stateEpoch()); assertEquals(56, sharePartition.nextFetchOffset()); - // Since the endOffset is now moved ahead, the initialReadGapOffset should be empty - initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNull(initialReadGapOffset); + // Since the endOffset is now moved ahead, the persisterReadResultGapWindow should be empty + persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNull(persisterReadResultGapWindow); } @Test @@ -4493,6 +5160,122 @@ public class SharePartitionTest { assertNotNull(sharePartition.cachedState().get(32L).batchAcquisitionLockTimeoutTask()); } + @Test + public void testLsoMovementForArchivingAllAvailableBatches() { + SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); + + // A client acquires 4 batches, 11 -> 20, 21 -> 30, 31 -> 40, 41 -> 50. + fetchAcquiredRecords(sharePartition, memoryRecords(10, 11), 10); + fetchAcquiredRecords(sharePartition, memoryRecords(10, 21), 10); + fetchAcquiredRecords(sharePartition, memoryRecords(10, 31), 10); + fetchAcquiredRecords(sharePartition, memoryRecords(10, 41), 10); + + // After the acknowledgements, the state of share partition will be: + // 1. 11 -> 20: AVAILABLE + // 2. 21 -> 30: ACQUIRED + // 3. 31 -> 40: AVAILABLE + // 4. 41 -> 50: ACQUIRED + sharePartition.acknowledge(MEMBER_ID, List.of( + new ShareAcknowledgementBatch(11, 20, List.of((byte) 2)), + new ShareAcknowledgementBatch(31, 40, List.of((byte) 2)) + )); + + // Move the LSO to 41. When the LSO moves ahead, all batches that are AVAILABLE before the new LSO will be ARCHIVED. + // Thus, the state of the share partition will be: + // 1. 11 -> 20: ARCHIVED + // 2. 21 -> 30: ACQUIRED + // 3. 31 -> 40: ARCHIVED + // 4. 41 -> 50: ACQUIRED + // Note, the records that are in ACQUIRED state will remain in ACQUIRED state and will be transitioned to a Terminal + // state when the corresponding acquisition lock timer task expires. + sharePartition.updateCacheAndOffsets(41); + + assertEquals(51, sharePartition.nextFetchOffset()); + assertEquals(41, sharePartition.startOffset()); + assertEquals(50, sharePartition.endOffset()); + + assertEquals(4, sharePartition.cachedState().size()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(31L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(41L).batchState()); + + // The client acknowledges the batch 21 -> 30. Since this batch is before the LSO, nothing will be done and these + // records will remain in the ACQUIRED state. + sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(21L, 30L, List.of((byte) 2)))); + + // The batch is still in ACQUIRED state. + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); + + // Once the acquisition lock timer task for the batch 21 -> 30 is expired, these records will directly be + // ARCHIVED. + sharePartition.cachedState().get(21L).batchAcquisitionLockTimeoutTask().run(); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(21L).batchState()); + } + + @Test + public void testLsoMovementForArchivingAllAvailableOffsets() { + SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); + + // A client acquires 4 batches, 11 -> 20, 21 -> 30, 31 -> 40, 41 -> 50. + fetchAcquiredRecords(sharePartition, memoryRecords(10, 11), 10); + fetchAcquiredRecords(sharePartition, memoryRecords(10, 21), 10); + fetchAcquiredRecords(sharePartition, memoryRecords(10, 31), 10); + fetchAcquiredRecords(sharePartition, memoryRecords(10, 41), 10); + + // After the acknowledgements, the share partition state will be: + // 1. 11 -> 20: AVAILABLE + // 2. 21 -> 30: ACQUIRED + // 3. 31 -> 40: AVAILABLE + // 4. 41 -> 50: ACQUIRED + sharePartition.acknowledge(MEMBER_ID, List.of( + new ShareAcknowledgementBatch(11, 20, List.of((byte) 2)), + new ShareAcknowledgementBatch(31, 40, List.of((byte) 2)) + )); + + // Move the LSO to 36. When the LSO moves ahead, all records that are AVAILABLE before the new LSO will be ARCHIVED. + // Thus, the state of the share partition will be: + // 1. 11 -> 20: ARCHIVED + // 2. 21 -> 30: ACQUIRED + // 3. 31 -> 35: ARCHIVED + // 3. 36 -> 40: AVAILABLE + // 4. 41 -> 50: ACQUIRED + // Note, the records that are in ACQUIRED state will remain in ACQUIRED state and will be transitioned to a Terminal + // state when the corresponding acquisition lock timer task expires. + sharePartition.updateCacheAndOffsets(36); + + assertEquals(36, sharePartition.nextFetchOffset()); + assertEquals(36, sharePartition.startOffset()); + assertEquals(50, sharePartition.endOffset()); + + assertEquals(4, sharePartition.cachedState().size()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(31L).offsetState().get(31L).state()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(31L).offsetState().get(32L).state()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(31L).offsetState().get(33L).state()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(31L).offsetState().get(34L).state()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(31L).offsetState().get(35L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(31L).offsetState().get(36L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(31L).offsetState().get(37L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(31L).offsetState().get(38L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(31L).offsetState().get(39L).state()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(31L).offsetState().get(40L).state()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(41L).batchState()); + + // The client acknowledges the batch 21 -> 30. Since this batch is before the LSO, nothing will be done and these + // records will remain in the ACQUIRED state. + sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(21L, 30L, List.of((byte) 2)))); + + // The batch is still in ACQUIRED state. + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); + + // Once the acquisition lock timer task for the batch 21 -> 30 is expired, these records will directly be + // ARCHIVED. + sharePartition.cachedState().get(21L).batchAcquisitionLockTimeoutTask().run(); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(21L).batchState()); + } + @Test public void testLsoMovementForArchivingOffsets() { SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); @@ -6000,11 +6783,11 @@ public class SharePartitionTest { assertEquals(40, sharePartition.endOffset()); assertEquals(21, sharePartition.nextFetchOffset()); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - assertEquals(21, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + assertEquals(21, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); } @Test @@ -6574,16 +7357,16 @@ public class SharePartitionTest { sharePartition.maybeInitialize(); - SharePartition.InitialReadGapOffset initialReadGapOffset = sharePartition.initialReadGapOffset(); - assertNotNull(initialReadGapOffset); + GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); + assertNotNull(persisterReadResultGapWindow); - // Since there is a gap in the beginning, the initialReadGapOffset window is same as the cachedState - assertEquals(11, initialReadGapOffset.gapStartOffset()); - assertEquals(40, initialReadGapOffset.endOffset()); + // Since there is a gap in the beginning, the persisterReadResultGapWindow window is same as the cachedState + assertEquals(11, persisterReadResultGapWindow.gapStartOffset()); + assertEquals(40, persisterReadResultGapWindow.endOffset()); long lastOffsetAcknowledged = sharePartition.findLastOffsetAcknowledged(); - // Since the initialReadGapOffset window begins at startOffset, we cannot count any of the offsets as acknowledged. + // Since the persisterReadResultGapWindow window begins at startOffset, we cannot count any of the offsets as acknowledged. // Thus, lastOffsetAcknowledged should be -1 assertEquals(-1, lastOffsetAcknowledged); } diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 75447e04df3..170ee3679f4 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -1100,9 +1100,13 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) @MethodSource(Array("getTestGroupProtocolParametersAll")) def testServersCanStartWithInvalidStaticConfigsAndValidDynamicConfigs(groupProtocol: String): Unit = { + TestNumReplicaFetcherMetricsReporter.testReporters.clear() + // modify snapshot interval config to explicitly take snapshot on a broker with valid dynamic configs val props = defaultStaticConfig(numServers) props.put(MetadataLogConfig.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, "10000") + props.put(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, classOf[TestNumReplicaFetcherMetricsReporter].getName) + props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") val kafkaConfig = KafkaConfig.fromProps(props) val newBroker = createBroker(kafkaConfig).asInstanceOf[BrokerServer] @@ -1110,6 +1114,15 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup alterSslKeystoreUsingConfigCommand(sslProperties1, listenerPrefix(SecureExternal)) + // Add num.replica.fetchers to the cluster-level config. + val clusterLevelProps = new Properties + clusterLevelProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") + reconfigureServers(clusterLevelProps, perBrokerConfig = false, (ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2")) + + // Wait for the metrics reporter to be configured + val initialReporter = TestNumReplicaFetcherMetricsReporter.waitForReporters(1).head + initialReporter.verifyState(reconfigureCount = 1, numFetcher = 2) + TestUtils.ensureConsistentKRaftMetadata(servers, controllerServer) TestUtils.waitUntilTrue( @@ -1122,11 +1135,19 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup newBroker.shutdown() newBroker.awaitShutdown() + // Clean up the test reporter + TestNumReplicaFetcherMetricsReporter.testReporters.clear() + val invalidStaticConfigs = defaultStaticConfig(newBroker.config.brokerId) invalidStaticConfigs.putAll(securityProps(invalidSslConfigs, KEYSTORE_PROPS, listenerPrefix(SecureExternal))) newBroker.config.updateCurrentConfig(KafkaConfig.fromProps(invalidStaticConfigs)) newBroker.startup() + + // Verify that the custom MetricsReporter is not reconfigured after restart. + // If readDynamicBrokerConfigsFromSnapshot works correctly, the reporter should maintain its state. + val reporterAfterRestart = TestNumReplicaFetcherMetricsReporter.waitForReporters(1).head + reporterAfterRestart.verifyState(reconfigureCount = 0, numFetcher = 2) } private def awaitInitialPositions(consumer: Consumer[_, _]): Unit = { @@ -1635,6 +1656,64 @@ class TestMetricsReporter extends MetricsReporter with Reconfigurable with Close } } +object TestNumReplicaFetcherMetricsReporter { + val testReporters = new ConcurrentLinkedQueue[TestNumReplicaFetcherMetricsReporter]() + + def waitForReporters(count: Int): List[TestNumReplicaFetcherMetricsReporter] = { + TestUtils.waitUntilTrue(() => testReporters.size == count, msg = "Metrics reporters size not matched. Expected: " + count + ", actual: " + testReporters.size()) + + val reporters = testReporters.asScala.toList + TestUtils.waitUntilTrue(() => reporters.forall(_.configureCount == 1), msg = "Metrics reporters not configured") + reporters + } +} + + +class TestNumReplicaFetcherMetricsReporter extends MetricsReporter { + import TestNumReplicaFetcherMetricsReporter._ + @volatile var configureCount = 0 + @volatile var reconfigureCount = 0 + @volatile var numFetchers: Int = 1 + testReporters.add(this) + + override def init(metrics: util.List[KafkaMetric]): Unit = { + } + + override def configure(configs: util.Map[String, _]): Unit = { + configureCount += 1 + numFetchers = configs.get(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG).toString.toInt + } + + override def metricChange(metric: KafkaMetric): Unit = { + } + + override def metricRemoval(metric: KafkaMetric): Unit = { + } + + override def reconfigurableConfigs(): util.Set[String] = { + util.Set.of(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG) + } + + override def validateReconfiguration(configs: util.Map[String, _]): Unit = { + val numFetchers = configs.get(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG).toString.toInt + if (numFetchers <= 0) + throw new ConfigException(s"Invalid num.replica.fetchers $numFetchers") + } + + override def reconfigure(configs: util.Map[String, _]): Unit = { + reconfigureCount += 1 + numFetchers = configs.get(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG).toString.toInt + } + + override def close(): Unit = { + } + + def verifyState(reconfigureCount: Int, numFetcher: Int = 1): Unit = { + assertEquals(reconfigureCount, this.reconfigureCount) + assertEquals(numFetcher, this.numFetchers) + } +} + class MockFileConfigProvider extends FileConfigProvider { @throws(classOf[IOException]) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index d30d5a1040e..da54113ae5c 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -41,7 +41,7 @@ import org.apache.kafka.server.storage.log.{FetchIsolation, UnexpectedAppendOffs import org.apache.kafka.server.util.{KafkaScheduler, MockTime, Scheduler} import org.apache.kafka.storage.internals.checkpoint.{LeaderEpochCheckpointFile, PartitionMetadataFile} import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache -import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, Cleaner, EpochEntry, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetResultHolder, OffsetsOutOfOrderException, ProducerStateManager, ProducerStateManagerConfig, RecordValidationException, UnifiedLog, VerificationGuard} +import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, AsyncOffsetReader, Cleaner, EpochEntry, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetResultHolder, OffsetsOutOfOrderException, ProducerStateManager, ProducerStateManagerConfig, RecordValidationException, UnifiedLog, VerificationGuard} import org.apache.kafka.storage.internals.utils.Throttler import org.apache.kafka.storage.log.metrics.{BrokerTopicMetrics, BrokerTopicStats} import org.junit.jupiter.api.Assertions._ @@ -2416,6 +2416,193 @@ class UnifiedLogTest { KafkaConfig.fromProps(props) } + @Test + def testFetchEarliestPendingUploadTimestampNoRemoteStorage(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + val log = createLog(logDir, logConfig) + + // Test initial state before any records + assertFetchOffsetBySpecialTimestamp(log, None, new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, -1, Optional.of(-1)), + ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) + + // Append records + val _ = prepareLogWithSequentialRecords(log, recordCount = 2) + + // Test state after records are appended + assertFetchOffsetBySpecialTimestamp(log, None, new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, -1, Optional.of(-1)), + ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) + } + + @Test + def testFetchEarliestPendingUploadTimestampWithRemoteStorage(): Unit = { + val logStartOffset = 0 + val (remoteLogManager: RemoteLogManager, log: UnifiedLog, timestampAndEpochs: Seq[TimestampAndEpoch]) = prepare(logStartOffset) + + val (firstTimestamp, firstLeaderEpoch) = (timestampAndEpochs.head.timestamp, timestampAndEpochs.head.leaderEpoch) + val (secondTimestamp, secondLeaderEpoch) = (timestampAndEpochs(1).timestamp, timestampAndEpochs(1).leaderEpoch) + val (_, thirdLeaderEpoch) = (timestampAndEpochs(2).timestamp, timestampAndEpochs(2).leaderEpoch) + + doAnswer(ans => { + val timestamp = ans.getArgument(1).asInstanceOf[Long] + Optional.of(timestamp) + .filter(_ == timestampAndEpochs.head.timestamp) + .map[TimestampAndOffset](x => new TimestampAndOffset(x, 0L, Optional.of(timestampAndEpochs.head.leaderEpoch))) + }).when(remoteLogManager).findOffsetByTimestamp(ArgumentMatchers.eq(log.topicPartition), + anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache)) + + // Offset 0 (first timestamp) is in remote storage and deleted locally. Offset 1 (second timestamp) is in local storage. + log.updateLocalLogStartOffset(1) + log.updateHighestOffsetInRemoteStorage(0) + + // In the assertions below we test that offset 0 (first timestamp) is only in remote and offset 1 (second timestamp) is in local storage. + assertFetchOffsetByTimestamp(log, Some(remoteLogManager), Some(new TimestampAndOffset(firstTimestamp, 0L, Optional.of(firstLeaderEpoch))), firstTimestamp) + assertFetchOffsetByTimestamp(log, Some(remoteLogManager), Some(new TimestampAndOffset(secondTimestamp, 1L, Optional.of(secondLeaderEpoch))), secondTimestamp) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager), new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager), new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager), new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 1L, Optional.of(secondLeaderEpoch)), + ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager), new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 3L, Optional.of(thirdLeaderEpoch)), + ListOffsetsRequest.LATEST_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager), new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 1L, Optional.of(secondLeaderEpoch)), + ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) + } + + @Test + def testFetchEarliestPendingUploadTimestampWithRemoteStorageNoLocalDeletion(): Unit = { + val logStartOffset = 0 + val (remoteLogManager: RemoteLogManager, log: UnifiedLog, timestampAndEpochs: Seq[TimestampAndEpoch]) = prepare(logStartOffset) + + val (firstTimestamp, firstLeaderEpoch) = (timestampAndEpochs.head.timestamp, timestampAndEpochs.head.leaderEpoch) + val (secondTimestamp, secondLeaderEpoch) = (timestampAndEpochs(1).timestamp, timestampAndEpochs(1).leaderEpoch) + val (_, thirdLeaderEpoch) = (timestampAndEpochs(2).timestamp, timestampAndEpochs(2).leaderEpoch) + + // Offsets upto 1 are in remote storage + doAnswer(ans => { + val timestamp = ans.getArgument(1).asInstanceOf[Long] + Optional.of( + timestamp match { + case x if x == firstTimestamp => new TimestampAndOffset(x, 0L, Optional.of(firstLeaderEpoch)) + case x if x == secondTimestamp => new TimestampAndOffset(x, 1L, Optional.of(secondLeaderEpoch)) + case _ => null + } + ) + }).when(remoteLogManager).findOffsetByTimestamp(ArgumentMatchers.eq(log.topicPartition), + anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache)) + + // Offsets 0, 1 (first and second timestamps) are in remote storage and not deleted locally. + log.updateLocalLogStartOffset(0) + log.updateHighestOffsetInRemoteStorage(1) + + // In the assertions below we test that offset 0 (first timestamp) and offset 1 (second timestamp) are on both remote and local storage + assertFetchOffsetByTimestamp(log, Some(remoteLogManager), Some(new TimestampAndOffset(firstTimestamp, 0L, Optional.of(firstLeaderEpoch))), firstTimestamp) + assertFetchOffsetByTimestamp(log, Some(remoteLogManager), Some(new TimestampAndOffset(secondTimestamp, 1L, Optional.of(secondLeaderEpoch))), secondTimestamp) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager), new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 1L, Optional.of(secondLeaderEpoch)), + ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 3L, Optional.of(thirdLeaderEpoch)), + ListOffsetsRequest.LATEST_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(thirdLeaderEpoch)), + ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) + } + + @Test + def testFetchEarliestPendingUploadTimestampNoSegmentsUploaded(): Unit = { + val logStartOffset = 0 + val (remoteLogManager: RemoteLogManager, log: UnifiedLog, timestampAndEpochs: Seq[TimestampAndEpoch]) = prepare(logStartOffset) + + val (firstTimestamp, firstLeaderEpoch) = (timestampAndEpochs.head.timestamp, timestampAndEpochs.head.leaderEpoch) + val (secondTimestamp, secondLeaderEpoch) = (timestampAndEpochs(1).timestamp, timestampAndEpochs(1).leaderEpoch) + val (_, thirdLeaderEpoch) = (timestampAndEpochs(2).timestamp, timestampAndEpochs(2).leaderEpoch) + + // No offsets are in remote storage + doAnswer(_ => Optional.empty[TimestampAndOffset]()) + .when(remoteLogManager).findOffsetByTimestamp(ArgumentMatchers.eq(log.topicPartition), + anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache)) + + // Offsets 0, 1, 2 (first, second and third timestamps) are in local storage only and not uploaded to remote storage. + log.updateLocalLogStartOffset(0) + log.updateHighestOffsetInRemoteStorage(-1) + + // In the assertions below we test that offset 0 (first timestamp), offset 1 (second timestamp) and offset 2 (third timestamp) are only on the local storage. + assertFetchOffsetByTimestamp(log, Some(remoteLogManager), Some(new TimestampAndOffset(firstTimestamp, 0L, Optional.of(firstLeaderEpoch))), firstTimestamp) + assertFetchOffsetByTimestamp(log, Some(remoteLogManager), Some(new TimestampAndOffset(secondTimestamp, 1L, Optional.of(secondLeaderEpoch))), secondTimestamp) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, -1L, Optional.of(-1)), + ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 3L, Optional.of(thirdLeaderEpoch)), + ListOffsetsRequest.LATEST_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) + } + + @Test + def testFetchEarliestPendingUploadTimestampStaleHighestOffsetInRemote(): Unit = { + val logStartOffset = 100 + val (remoteLogManager: RemoteLogManager, log: UnifiedLog, timestampAndEpochs: Seq[TimestampAndEpoch]) = prepare(logStartOffset) + + val (firstTimestamp, firstLeaderEpoch) = (timestampAndEpochs.head.timestamp, timestampAndEpochs.head.leaderEpoch) + val (secondTimestamp, secondLeaderEpoch) = (timestampAndEpochs(1).timestamp, timestampAndEpochs(1).leaderEpoch) + val (_, thirdLeaderEpoch) = (timestampAndEpochs(2).timestamp, timestampAndEpochs(2).leaderEpoch) + + // Offsets 100, 101, 102 (first, second and third timestamps) are in local storage and not uploaded to remote storage. + // Tiered storage copy was disabled and then enabled again, because of which the remote log segments are deleted but + // the highest offset in remote storage has become stale + doAnswer(_ => Optional.empty[TimestampAndOffset]()) + .when(remoteLogManager).findOffsetByTimestamp(ArgumentMatchers.eq(log.topicPartition), + anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache)) + + log.updateLocalLogStartOffset(100) + log.updateHighestOffsetInRemoteStorage(50) + + // In the assertions below we test that offset 100 (first timestamp), offset 101 (second timestamp) and offset 102 (third timestamp) are only on the local storage. + assertFetchOffsetByTimestamp(log, Some(remoteLogManager), Some(new TimestampAndOffset(firstTimestamp, 100L, Optional.of(firstLeaderEpoch))), firstTimestamp) + assertFetchOffsetByTimestamp(log, Some(remoteLogManager), Some(new TimestampAndOffset(secondTimestamp, 101L, Optional.of(secondLeaderEpoch))), secondTimestamp) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 100L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 50L, Optional.empty()), + ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 100L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 103L, Optional.of(thirdLeaderEpoch)), + ListOffsetsRequest.LATEST_TIMESTAMP) + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager),new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 100L, Optional.of(firstLeaderEpoch)), + ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) + } + + private def prepare(logStartOffset: Int): (RemoteLogManager, UnifiedLog, Seq[TimestampAndEpoch]) = { + val config: KafkaConfig = createKafkaConfigWithRLM + val purgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]("RemoteListOffsets", config.brokerId) + val remoteLogManager = spy(new RemoteLogManager(config.remoteLogManagerConfig, + 0, + logDir.getAbsolutePath, + "clusterId", + mockTime, + _ => Optional.empty[UnifiedLog](), + (_, _) => {}, + brokerTopicStats, + new Metrics(), + Optional.empty)) + remoteLogManager.setDelayedOperationPurgatory(purgatory) + + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1, remoteLogStorageEnable = true) + val log = createLog(logDir, logConfig, logStartOffset = logStartOffset, remoteStorageSystemEnable = true, remoteLogManager = Some(remoteLogManager)) + + // Verify earliest pending upload offset for empty log + assertFetchOffsetBySpecialTimestamp(log, Some(remoteLogManager), new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, logStartOffset, Optional.empty()), + ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) + + val timestampAndEpochs = prepareLogWithSequentialRecords(log, recordCount = 3) + (remoteLogManager, log, timestampAndEpochs) + } + /** * Test the Log truncate operations */ @@ -4786,6 +4973,44 @@ class UnifiedLogTest { (log, segmentWithOverflow) } + + private def assertFetchOffsetByTimestamp(log: UnifiedLog, remoteLogManagerOpt: Option[RemoteLogManager], expected: Option[TimestampAndOffset], timestamp: Long): Unit = { + val remoteOffsetReader = getRemoteOffsetReader(remoteLogManagerOpt) + val offsetResultHolder = log.fetchOffsetByTimestamp(timestamp, remoteOffsetReader) + assertTrue(offsetResultHolder.futureHolderOpt.isPresent) + offsetResultHolder.futureHolderOpt.get.taskFuture.get(1, TimeUnit.SECONDS) + assertTrue(offsetResultHolder.futureHolderOpt.get.taskFuture.isDone) + assertTrue(offsetResultHolder.futureHolderOpt.get.taskFuture.get().hasTimestampAndOffset) + assertEquals(expected.get, offsetResultHolder.futureHolderOpt.get.taskFuture.get().timestampAndOffset().orElse(null)) + } + + private def assertFetchOffsetBySpecialTimestamp(log: UnifiedLog, remoteLogManagerOpt: Option[RemoteLogManager], expected: TimestampAndOffset, timestamp: Long): Unit = { + val remoteOffsetReader = getRemoteOffsetReader(remoteLogManagerOpt) + val offsetResultHolder = log.fetchOffsetByTimestamp(timestamp, remoteOffsetReader) + assertEquals(new OffsetResultHolder(expected), offsetResultHolder) + } + + private def getRemoteOffsetReader(remoteLogManagerOpt: Option[Any]): Optional[AsyncOffsetReader] = { + remoteLogManagerOpt match { + case Some(remoteLogManager) => Optional.of(remoteLogManager.asInstanceOf[AsyncOffsetReader]) + case None => Optional.empty[AsyncOffsetReader]() + } + } + + private def prepareLogWithSequentialRecords(log: UnifiedLog, recordCount: Int): Seq[TimestampAndEpoch] = { + val firstTimestamp = mockTime.milliseconds() + + (0 until recordCount).map { i => + val timestampAndEpoch = TimestampAndEpoch(firstTimestamp + i, i) + log.appendAsLeader( + TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = timestampAndEpoch.timestamp), + timestampAndEpoch.leaderEpoch + ) + timestampAndEpoch + } + } + + case class TimestampAndEpoch(timestamp: Long, leaderEpoch: Int) } object UnifiedLogTest { diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index 1d0dfe019d8..046ef52a7de 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -1180,4 +1180,564 @@ class AbstractFetcherThreadTest { fetcher.processFetchRequest(partitionData, fetchRequestOpt) assertEquals(0, replicaState.logEndOffset, "FetchResponse should be ignored when leader epoch does not match") } + + private def emptyReplicaState(rlmEnabled: Boolean, partition: TopicPartition, fetcher: MockFetcherThread): PartitionState = { + // Follower begins with an empty log + val replicaState = PartitionState(Seq(), leaderEpoch = 0, highWatermark = 0L, rlmEnabled = rlmEnabled) + fetcher.setReplicaState(partition, replicaState) + fetcher.addPartitions(Map(partition -> initialFetchState(topicIds.get(partition.topic), fetchOffset = 0, leaderEpoch = 0))) + replicaState + } + + /** + * Test: Empty Follower Fetch with TieredStorage Disabled and Leader LogStartOffset = 0 + * + * Purpose: + * - Simulate a leader with logs starting at offset 0 and validate how the follower + * behaves when TieredStorage is disabled. + * + * Conditions: + * - TieredStorage: **Disabled** + * - Leader LogStartOffset: **0** + * + * Scenario: + * - The leader starts with a log at offset 0, containing three record batches offset at 0, 150, and 199. + * - The follower begins fetching, and we validate the correctness of its replica state as it fetches. + * + * Expected Outcomes: + * 1. The follower fetch state should transition to `FETCHING` initially. + * 2. After the first poll, one record batch is fetched. + * 3. After subsequent polls, the entire leader log is fetched: + * - Replica log size: 3 + * - Replica LogStartOffset: 0 + * - Replica LogEndOffset: 200 + * - Replica HighWatermark: 199 + */ + @Test + def testEmptyFollowerFetchTieredStorageDisabledLeaderLogStartOffsetZero(): Unit = { + val rlmEnabled = false + val partition = new TopicPartition("topic1", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + + val replicaState = emptyReplicaState(rlmEnabled, partition, fetcher) + + val leaderLog = Seq( + // LogStartOffset = LocalLogStartOffset = 0 + mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 150, leaderEpoch = 0, new SimpleRecord("d".getBytes)), + mkBatch(baseOffset = 199, leaderEpoch = 0, new SimpleRecord("e".getBytes)) + ) + + val leaderState = PartitionState( + leaderLog, + leaderEpoch = 0, + highWatermark = 199L, + rlmEnabled = rlmEnabled + ) + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + fetcher.doWork() + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(1, replicaState.log.size) + assertEquals(0, replicaState.logStartOffset) + assertEquals(1, replicaState.logEndOffset) + assertEquals(Some(1), fetcher.fetchState(partition).map(_.fetchOffset())) + + // Only 1 record batch is returned after a poll so calling 'n' number of times to get the desired result. + for (_ <- 1 to 2) fetcher.doWork() + assertEquals(3, replicaState.log.size) + assertEquals(0, replicaState.logStartOffset) + assertEquals(200, replicaState.logEndOffset) + assertEquals(199, replicaState.highWatermark) + } + + /** + * Test: Empty Follower Fetch with TieredStorage Disabled and Leader LogStartOffset != 0 + * + * Purpose: + * - Validate follower behavior when the leader's log starts at a non-zero offset (10). + * + * Conditions: + * - TieredStorage: **Disabled** + * - Leader LogStartOffset: **10** + * + * Scenario: + * - The leader log starts at offset 10 with batches at 10, 150, and 199. + * - The follower starts fetching from offset 10. + * + * Expected Outcomes: + * 1. The follower's initial log is empty. + * 2. Replica offsets after polls: + * - LogStartOffset = 10 + * - LogEndOffset = 200 + * - HighWatermark = 199 + */ + @Test + def testEmptyFollowerFetchTieredStorageDisabledLeaderLogStartOffsetNonZero(): Unit = { + val rlmEnabled = false + val partition = new TopicPartition("topic1", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + + val replicaState = emptyReplicaState(rlmEnabled, partition, fetcher) + + val leaderLog = Seq( + // LogStartOffset = LocalLogStartOffset = 10 + mkBatch(baseOffset = 10, leaderEpoch = 0, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 150, leaderEpoch = 0, new SimpleRecord("d".getBytes)), + mkBatch(baseOffset = 199, leaderEpoch = 0, new SimpleRecord("e".getBytes)) + ) + + val leaderState = PartitionState( + leaderLog, + leaderEpoch = 0, + highWatermark = 199L, + rlmEnabled = rlmEnabled + ) + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + fetcher.doWork() + // Follower gets out-of-range error (no messages received), fetch offset is updated from 0 to 10 + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.log.size) + assertEquals(10, replicaState.logStartOffset) + assertEquals(10, replicaState.logEndOffset) + assertEquals(Some(10), fetcher.fetchState(partition).map(_.fetchOffset())) + + // Only 1 record batch is returned after a poll so calling 'n' number of times to get the desired result. + for (_ <- 1 to 3) fetcher.doWork() + assertEquals(3, replicaState.log.size) + assertEquals(10, replicaState.logStartOffset) + assertEquals(200, replicaState.logEndOffset) + assertEquals(199, replicaState.highWatermark) + } + + /** + * Test: Empty Follower Fetch with TieredStorage Enabled, Leader LogStartOffset = 0, and No Local Deletions + * + * Purpose: + * - Simulate TieredStorage enabled and validate follower fetching behavior when the leader + * log starts at 0 and no segments have been uploaded or deleted locally. + * + * Conditions: + * - TieredStorage: **Enabled** + * - Leader LogStartOffset: **0** + * - Leader LocalLogStartOffset: **0** (No local segments deleted). + * + * Scenario: + * - The leader log contains three record batches at offsets 0, 150, and 199. + * - The follower starts fetching from offset 0. + * + * Expected Outcomes: + * 1. The replica log accurately reflects the leader's log: + * - LogStartOffset = 0 + * - LocalLogStartOffset = 0 + * - LogEndOffset = 200 + * - HighWatermark = 199 + */ + @Test + def testEmptyFollowerFetchTieredStorageEnabledLeaderLogStartOffsetZeroNoLocalDeletions(): Unit = { + val rlmEnabled = true + val partition = new TopicPartition("topic1", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + + val replicaState = emptyReplicaState(rlmEnabled, partition, fetcher) + + val leaderLog = Seq( + // LogStartOffset = LocalLogStartOffset = 0 + mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 150, leaderEpoch = 0, new SimpleRecord("d".getBytes)), + mkBatch(baseOffset = 199, leaderEpoch = 0, new SimpleRecord("e".getBytes)) + ) + + val leaderState = PartitionState( + leaderLog, + leaderEpoch = 0, + highWatermark = 199L, + rlmEnabled = rlmEnabled + ) + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + fetcher.doWork() + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(1, replicaState.log.size) + assertEquals(0, replicaState.logStartOffset) + assertEquals(0, replicaState.localLogStartOffset) + assertEquals(1, replicaState.logEndOffset) + assertEquals(199, replicaState.highWatermark) + assertEquals(Some(1), fetcher.fetchState(partition).map(_.fetchOffset())) + + // Only 1 record batch is returned after a poll so calling 'n' number of times to get the desired result. + for (_ <- 1 to 2) fetcher.doWork() + assertEquals(3, replicaState.log.size) + assertEquals(0, replicaState.logStartOffset) + assertEquals(0, replicaState.localLogStartOffset) + assertEquals(200, replicaState.logEndOffset) + assertEquals(199, replicaState.highWatermark) + } + + /** + * Test: Empty Follower Fetch with TieredStorage Enabled, Leader LogStartOffset = 0, and Local Deletions + * + * Purpose: + * - Simulate TieredStorage enabled with some segments uploaded and deleted locally, causing + * a difference between the leader's LogStartOffset (0) and LocalLogStartOffset (> 0). + * + * Conditions: + * - TieredStorage: **Enabled** + * - Leader LogStartOffset: **0** + * - Leader LocalLogStartOffset: **100** (Some segments deleted locally). + * + * Scenario: + * - The leader log starts at offset 0 but the local leader log starts at offset 100. + * - The follower fetch operation begins from offset 0. + * + * Expected Outcomes: + * 1. After offset adjustments for local deletions: + * - LogStartOffset = 0 + * - LocalLogStartOffset = 100 + * - LogEndOffset = 200 + * - HighWatermark = 199 + */ + @Test + def testEmptyFollowerFetchTieredStorageEnabledLeaderLogStartOffsetZeroWithLocalDeletions(): Unit = { + val rlmEnabled = true + val partition = new TopicPartition("topic1", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + + val replicaState = emptyReplicaState(rlmEnabled, partition, fetcher) + + val leaderLog = Seq( + // LocalLogStartOffset = 100 + mkBatch(baseOffset = 100, leaderEpoch = 0, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 150, leaderEpoch = 0, new SimpleRecord("d".getBytes)), + mkBatch(baseOffset = 199, leaderEpoch = 0, new SimpleRecord("e".getBytes)) + ) + + val leaderState = PartitionState( + leaderLog, + leaderEpoch = 0, + highWatermark = 199L, + rlmEnabled = rlmEnabled + ) + leaderState.logStartOffset = 0 + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + fetcher.doWork() + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.log.size) + assertEquals(100, replicaState.localLogStartOffset) + assertEquals(100, replicaState.logEndOffset) + assertEquals(Some(100), fetcher.fetchState(partition).map(_.fetchOffset())) + + // Only 1 record batch is returned after a poll so calling 'n' number of times to get the desired result. + for (_ <- 1 to 3) fetcher.doWork() + assertEquals(3, replicaState.log.size) + assertEquals(0, replicaState.logStartOffset) + assertEquals(100, replicaState.localLogStartOffset) + assertEquals(200, replicaState.logEndOffset) + assertEquals(199, replicaState.highWatermark) + } + + /** + * Test: Empty Follower Fetch with TieredStorage Enabled, Leader LogStartOffset != 0, and No Local Deletions + * + * Purpose: + * - Simulate TieredStorage enabled and validate follower fetch behavior when the leader's log + * starts at a non-zero offset and no local deletions have occurred. + * + * Conditions: + * - TieredStorage: **Enabled** + * - Leader LogStartOffset: **10** + * - Leader LocalLogStartOffset: **10** (No deletions). + * + * Scenario: + * - The leader log starts at offset 10 with batches at 10, 150, and 199. + * - The follower starts fetching from offset 10. + * + * Expected Outcomes: + * 1. After fetching, the replica log matches the leader: + * - LogStartOffset = 10 + * - LocalLogStartOffset = 10 + * - LogEndOffset = 200 + * - HighWatermark = 199 + */ + @Test + def testEmptyFollowerFetchTieredStorageEnabledLeaderLogStartOffsetNonZeroNoLocalDeletions(): Unit = { + val rlmEnabled = true + val partition = new TopicPartition("topic1", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + + val replicaState = emptyReplicaState(rlmEnabled, partition, fetcher) + + val leaderLog = Seq( + // LogStartOffset = LocalLogStartOffset = 10 + mkBatch(baseOffset = 10, leaderEpoch = 0, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 150, leaderEpoch = 0, new SimpleRecord("d".getBytes)), + mkBatch(baseOffset = 199, leaderEpoch = 0, new SimpleRecord("e".getBytes)) + ) + + val leaderState = PartitionState( + leaderLog, + leaderEpoch = 0, + highWatermark = 199L, + rlmEnabled = rlmEnabled, + ) + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + fetcher.doWork() + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.log.size) + assertEquals(10, replicaState.localLogStartOffset) + assertEquals(10, replicaState.logEndOffset) + assertEquals(Some(10), fetcher.fetchState(partition).map(_.fetchOffset())) + + // Only 1 record batch is returned after a poll so calling 'n' number of times to get the desired result. + for (_ <- 1 to 3) fetcher.doWork() + assertEquals(3, replicaState.log.size) + assertEquals(10, replicaState.logStartOffset) + assertEquals(10, replicaState.localLogStartOffset) + assertEquals(200, replicaState.logEndOffset) + assertEquals(199, replicaState.highWatermark) + } + + /** + * Test: Empty Follower Fetch with TieredStorage Enabled, Leader LogStartOffset != 0, and Local Deletions + * + * Purpose: + * - Validate follower adjustments when the leader has log deletions causing + * LocalLogStartOffset > LogStartOffset. + * + * Conditions: + * - TieredStorage: **Enabled** + * - Leader LogStartOffset: **10** + * - Leader LocalLogStartOffset: **100** (All older segments deleted locally). + * + * Scenario: + * - The leader log starts at offset 10 but the local log starts at offset 100. + * - The follower fetch starts at offset 10 but adjusts for local deletions. + * + * Expected Outcomes: + * 1. Initial fetch offset adjustments: + * - First adjustment: LogEndOffset = 10 (after offset-out-of-range error) + * - Second adjustment: LogEndOffset = 100 (after offset-moved-to-tiered-storage error) + * 2. After successful fetches: + * - 3 record batches fetched + * - LogStartOffset = 10 + * - LocalLogStartOffset = 100 + * - LogEndOffset = 200 + * - HighWatermark = 199 + */ + @Test + def testEmptyFollowerFetchTieredStorageEnabledLeaderLogStartOffsetNonZeroWithLocalDeletions(): Unit = { + val rlmEnabled = true + val partition = new TopicPartition("topic1", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + + val replicaState = emptyReplicaState(rlmEnabled, partition, fetcher) + + val leaderLog = Seq( + // LocalLogStartOffset = 100 + mkBatch(baseOffset = 100, leaderEpoch = 0, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 150, leaderEpoch = 0, new SimpleRecord("d".getBytes)), + mkBatch(baseOffset = 199, leaderEpoch = 0, new SimpleRecord("e".getBytes)) + ) + + val leaderState = PartitionState( + leaderLog, + leaderEpoch = 0, + highWatermark = 199L, + rlmEnabled = rlmEnabled, + ) + leaderState.logStartOffset = 10 + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + fetcher.doWork() + // On offset-out-of-range error, fetch offset is updated + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.log.size) + assertEquals(10, replicaState.localLogStartOffset) + assertEquals(10, replicaState.logEndOffset) + assertEquals(Some(10), fetcher.fetchState(partition).map(_.fetchOffset())) + + fetcher.doWork() + // On offset-moved-to-tiered-storage error, fetch offset is updated + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.log.size) + assertEquals(100, replicaState.localLogStartOffset) + assertEquals(100, replicaState.logEndOffset) + assertEquals(Some(100), fetcher.fetchState(partition).map(_.fetchOffset())) + + // Only 1 record batch is returned after a poll so calling 'n' number of times to get the desired result. + for (_ <- 1 to 3) fetcher.doWork() + assertEquals(3, replicaState.log.size) + assertEquals(10, replicaState.logStartOffset) + assertEquals(100, replicaState.localLogStartOffset) + assertEquals(200, replicaState.logEndOffset) + assertEquals(199, replicaState.highWatermark) + } + + /** + * Test: Empty Follower Fetch with TieredStorage Enabled, All Local Segments Deleted + * + * Purpose: + * - Handle scenarios where all local segments have been deleted: + * - LocalLogStartOffset > LogStartOffset. + * - LocalLogStartOffset = LogEndOffset. + * + * Conditions: + * - TieredStorage: **Enabled** + * - Leader LogStartOffset: **0 or > 0** + * - Leader LocalLogStartOffset: Leader LogEndOffset (all segments deleted locally). + * + * Expected Outcomes: + * 1. Follower state is adjusted to reflect local deletions: + * - LocalLogStartOffset = LogEndOffset. + * - No new data remains to fetch. + */ + @Test + def testEmptyFollowerFetchTieredStorageEnabledLeaderLogStartOffsetZeroAllLocalSegmentsDeleted(): Unit = { + val rlmEnabled = true + val partition = new TopicPartition("topic1", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + + val replicaState = emptyReplicaState(rlmEnabled, partition, fetcher) + + val leaderLog = Seq( + // LocalLogStartOffset = 100 + mkBatch(baseOffset = 100, leaderEpoch = 0, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 150, leaderEpoch = 0, new SimpleRecord("d".getBytes)), + ) + + val leaderState = PartitionState( + leaderLog, + leaderEpoch = 0, + highWatermark = 151L, + rlmEnabled = rlmEnabled + ) + leaderState.logStartOffset = 0 + // Set Local Log Start Offset to Log End Offset + leaderState.localLogStartOffset = 151 + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + fetcher.doWork() + + // On offset-moved-to-tiered-storage error, fetch offset is updated + fetcher.doWork() + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.log.size) + assertEquals(151, replicaState.localLogStartOffset) + assertEquals(151, replicaState.logEndOffset) + assertEquals(151, replicaState.highWatermark) + assertEquals(Some(151), fetcher.fetchState(partition).map(_.fetchOffset())) + + // Call once again to see if new data is received + fetcher.doWork() + // No metadata update expected + assertEquals(0, replicaState.log.size) + assertEquals(0, replicaState.logStartOffset) + assertEquals(151, replicaState.localLogStartOffset) + assertEquals(151, replicaState.logEndOffset) + assertEquals(151, replicaState.highWatermark) + } + + /** + * Test: Empty Follower Fetch with TieredStorage Enabled, Leader LogStartOffset != 0, and All Local Segments Deleted + * + * Purpose: + * - Validate follower behavior when TieredStorage is enabled, the leader's log starts at a non-zero offset, + * and all local log segments have been deleted. + * + * Conditions: + * - TieredStorage: **Enabled** + * - Leader LogStartOffset: **10** + * - Leader LocalLogStartOffset: **151** (all older segments deleted locally). + * + * Scenario: + * - The leader log contains record batches from offset 100, but all local segments up to offset 151 are deleted. + * - The follower starts at LogStartOffset = 10 and adjusts for local segment deletions. + * + * Expected Outcomes: + * 1. Follower detects offset adjustments due to local deletions: + * - LogStartOffset remains 10. + * - LocalLogStartOffset updates to 151. + * - LogEndOffset updates to 151. + * 2. HighWatermark aligns with the leader (151). + * 3. No new data is fetched since all relevant segments are deleted. + */ + @Test + def testEmptyFollowerFetchTieredStorageEnabledLeaderLogStartOffsetNonZeroAllLocalSegmentsDeleted(): Unit = { + val rlmEnabled = true + val partition = new TopicPartition("topic1", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + + val replicaState = emptyReplicaState(rlmEnabled, partition, fetcher) + + val leaderLog = Seq( + // LocalLogStartOffset = 100 + mkBatch(baseOffset = 100, leaderEpoch = 0, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 150, leaderEpoch = 0, new SimpleRecord("d".getBytes)), + ) + + val leaderState = PartitionState( + leaderLog, + leaderEpoch = 0, + highWatermark = 151L, + rlmEnabled = rlmEnabled + ) + leaderState.logStartOffset = 10 + // Set Local Log Start Offset to Log End Offset + leaderState.localLogStartOffset = 151 + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + fetcher.doWork() + + // On offset-out-of-range error, fetch offset is updated + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.log.size) + assertEquals(10, replicaState.localLogStartOffset) + assertEquals(10, replicaState.logEndOffset) + assertEquals(Some(10), fetcher.fetchState(partition).map(_.fetchOffset())) + + // On offset-moved-to-tiered-storage error, fetch offset is updated + fetcher.doWork() + assertEquals(Option(ReplicaState.FETCHING), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.log.size) + assertEquals(151, replicaState.localLogStartOffset) + assertEquals(151, replicaState.logEndOffset) + assertEquals(151, replicaState.highWatermark) + assertEquals(Some(151), fetcher.fetchState(partition).map(_.fetchOffset())) + + // Call once again to see if new data is received + fetcher.doWork() + // No metadata update expected + assertEquals(0, replicaState.log.size) + assertEquals(10, replicaState.logStartOffset) + assertEquals(151, replicaState.localLogStartOffset) + assertEquals(151, replicaState.logEndOffset) + assertEquals(151, replicaState.highWatermark) + } } \ No newline at end of file diff --git a/docker/README.md b/docker/README.md index 9c2916d9293..c4b9d49d0ea 100644 --- a/docker/README.md +++ b/docker/README.md @@ -130,6 +130,10 @@ python docker_build_test.py kafka/test --image-tag=3.6.0 --image-type=jvm --kafk ``` python docker_build_test.py kafka/test --image-tag=3.8.0 --image-type=native --kafka-url=https://archive.apache.org/dist/kafka/3.8.0/kafka_2.13-3.8.0.tgz ``` +- Example(local build archive with jvm or native image type) :- To build and test an image named test with local build archive +``` +python docker_build_test.py kafka/test --image-tag=local-build --image-type= --kafka-archive= +``` Creating a Release Candidate ---------------------------- diff --git a/docker/common.py b/docker/common.py index 9c0f901823f..f04a484a187 100644 --- a/docker/common.py +++ b/docker/common.py @@ -33,12 +33,14 @@ def get_input(message): raise ValueError("This field cannot be empty") return value -def build_docker_image_runner(command, image_type): +def build_docker_image_runner(command, image_type, kafka_archive=None): temp_dir_path = tempfile.mkdtemp() current_dir = os.path.dirname(os.path.realpath(__file__)) copy_tree(f"{current_dir}/{image_type}", f"{temp_dir_path}/{image_type}") copy_tree(f"{current_dir}/resources", f"{temp_dir_path}/{image_type}/resources") copy_file(f"{current_dir}/server.properties", f"{temp_dir_path}/{image_type}") + if kafka_archive: + copy_file(kafka_archive, f"{temp_dir_path}/{image_type}/kafka.tgz") command = command.replace("$DOCKER_FILE", f"{temp_dir_path}/{image_type}/Dockerfile") command = command.replace("$DOCKER_DIR", f"{temp_dir_path}/{image_type}") try: diff --git a/docker/docker_build_test.py b/docker/docker_build_test.py index 793148573f3..fab6e65263d 100755 --- a/docker/docker_build_test.py +++ b/docker/docker_build_test.py @@ -25,9 +25,11 @@ Usage: Example command:- docker_build_test.py --image-tag --image-type --kafka-url + docker_build_test.py --image-tag --image-type --kafka-archive This command will build an image with as image name, as image_tag (it will be latest by default), as image type (jvm by default), for the kafka inside the image and run tests on the image. + can be passed as an alternative to to use a local kafka archive. The path of kafka_archive should be absolute. -b can be passed as additional argument if you just want to build the image. -t can be passed if you just want to run tests on the image. """ @@ -41,10 +43,6 @@ from common import execute, build_docker_image_runner import tempfile import os -def build_docker_image(image, tag, kafka_url, image_type): - image = f'{image}:{tag}' - build_docker_image_runner(f"docker build -f $DOCKER_FILE -t {image} --build-arg kafka_url={kafka_url} --build-arg build_date={date.today()} $DOCKER_DIR", image_type) - def run_docker_tests(image, tag, kafka_url, image_type): temp_dir_path = tempfile.mkdtemp() try: @@ -69,16 +67,20 @@ if __name__ == '__main__': parser.add_argument("image", help="Image name that you want to keep for the Docker image") parser.add_argument("--image-tag", "-tag", default="latest", dest="tag", help="Image tag that you want to add to the image") parser.add_argument("--image-type", "-type", choices=["jvm", "native"], default="jvm", dest="image_type", help="Image type you want to build") - parser.add_argument("--kafka-url", "-u", dest="kafka_url", help="Kafka url to be used to download kafka binary tarball in the docker image") parser.add_argument("--build", "-b", action="store_true", dest="build_only", default=False, help="Only build the image, don't run tests") parser.add_argument("--test", "-t", action="store_true", dest="test_only", default=False, help="Only run the tests, don't build the image") + + archive_group = parser.add_mutually_exclusive_group(required=True) + archive_group.add_argument("--kafka-url", "-u", dest="kafka_url", help="Kafka url to be used to download kafka binary tarball in the docker image") + archive_group.add_argument("--kafka-archive", "-a", dest="kafka_archive", help="Kafka archive to be used to extract kafka binary tarball in the docker image") + args = parser.parse_args() if args.build_only or not (args.build_only or args.test_only): if args.kafka_url: - build_docker_image(args.image, args.tag, args.kafka_url, args.image_type) - else: - raise ValueError("--kafka-url is a required argument for docker image") + build_docker_image_runner(f"docker build -f $DOCKER_FILE -t {args.image}:{args.tag} --build-arg kafka_url={args.kafka_url} --build-arg build_date={date.today()} --no-cache --progress=plain $DOCKER_DIR", args.image_type) + elif args.kafka_archive: + build_docker_image_runner(f"docker build -f $DOCKER_FILE -t {args.image}:{args.tag} --build-arg build_date={date.today()} --no-cache --progress=plain $DOCKER_DIR", args.image_type, args.kafka_archive) if args.test_only or not (args.build_only or args.test_only): run_docker_tests(args.image, args.tag, args.kafka_url, args.image_type) diff --git a/docker/jvm/Dockerfile b/docker/jvm/Dockerfile index f98f50a2e03..3d2f06820d4 100644 --- a/docker/jvm/Dockerfile +++ b/docker/jvm/Dockerfile @@ -23,20 +23,27 @@ USER root # Get kafka from https://archive.apache.org/dist/kafka and pass the url through build arguments ARG kafka_url +ENV KAFKA_URL=$kafka_url + COPY jsa_launch /etc/kafka/docker/jsa_launch COPY server.properties /etc/kafka/docker/server.properties +COPY *kafka.tgz kafka.tgz + RUN set -eux ; \ apk update ; \ apk upgrade ; \ - apk add --no-cache wget gcompat gpg gpg-agent procps bash; \ + apk add --no-cache bash; \ + if [ -n "$KAFKA_URL" ]; then \ + apk add --no-cache wget gcompat gpg gpg-agent procps; \ + wget -nv -O kafka.tgz "$KAFKA_URL"; \ + wget -nv -O kafka.tgz.asc "$KAFKA_URL.asc"; \ + wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ + gpg --import KEYS; \ + gpg --batch --verify kafka.tgz.asc kafka.tgz; \ + fi; \ mkdir opt/kafka; \ - wget -nv -O kafka.tgz "$kafka_url"; \ - wget -nv -O kafka.tgz.asc "$kafka_url.asc"; \ - tar xfz kafka.tgz -C /opt/kafka --strip-components 1; \ - wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ - gpg --import KEYS; \ - gpg --batch --verify kafka.tgz.asc kafka.tgz + tar xfz kafka.tgz -C opt/kafka --strip-components 1; # Generate jsa files using dynamic CDS for kafka server start command and kafka storage format command RUN /etc/kafka/docker/jsa_launch @@ -53,6 +60,9 @@ USER root ARG kafka_url ARG build_date +ENV KAFKA_URL=$kafka_url + +COPY *kafka.tgz kafka.tgz LABEL org.label-schema.name="kafka" \ org.label-schema.description="Apache Kafka" \ @@ -60,17 +70,25 @@ LABEL org.label-schema.name="kafka" \ org.label-schema.vcs-url="https://github.com/apache/kafka" \ maintainer="Apache Kafka" -RUN set -eux ; \ +RUN mkdir opt/kafka; \ + set -eux ; \ apk update ; \ apk upgrade ; \ - apk add --no-cache wget gcompat gpg gpg-agent procps bash; \ - mkdir opt/kafka; \ - wget -nv -O kafka.tgz "$kafka_url"; \ - wget -nv -O kafka.tgz.asc "$kafka_url.asc"; \ - tar xfz kafka.tgz -C /opt/kafka --strip-components 1; \ - wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ - gpg --import KEYS; \ - gpg --batch --verify kafka.tgz.asc kafka.tgz; \ + apk add --no-cache bash; \ + if [ -n "$KAFKA_URL" ]; then \ + apk add --no-cache wget gcompat gpg gpg-agent procps; \ + wget -nv -O kafka.tgz "$KAFKA_URL"; \ + wget -nv -O kafka.tgz.asc "$KAFKA_URL.asc"; \ + tar xfz kafka.tgz -C /opt/kafka --strip-components 1; \ + wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ + gpg --import KEYS; \ + gpg --batch --verify kafka.tgz.asc kafka.tgz; \ + rm kafka.tgz kafka.tgz.asc KEYS; \ + apk del wget gpg gpg-agent; \ + else \ + tar xfz kafka.tgz -C /opt/kafka --strip-components 1; \ + rm kafka.tgz; \ + fi; \ mkdir -p /var/lib/kafka/data /etc/kafka/secrets; \ mkdir -p /etc/kafka/docker /usr/logs /mnt/shared/config; \ adduser -h /home/appuser -D --shell /bin/bash appuser; \ @@ -79,8 +97,6 @@ RUN set -eux ; \ chmod -R ug+w /etc/kafka /var/lib/kafka /etc/kafka/secrets; \ cp /opt/kafka/config/log4j2.yaml /etc/kafka/docker/log4j2.yaml; \ cp /opt/kafka/config/tools-log4j2.yaml /etc/kafka/docker/tools-log4j2.yaml; \ - rm kafka.tgz kafka.tgz.asc KEYS; \ - apk del wget gpg gpg-agent; \ apk cache clean; COPY server.properties /etc/kafka/docker/server.properties diff --git a/docker/native/Dockerfile b/docker/native/Dockerfile index ca85f35562d..010edbcd51c 100644 --- a/docker/native/Dockerfile +++ b/docker/native/Dockerfile @@ -29,15 +29,18 @@ ENV TARGET_PATH="$KAFKA_DIR/kafka.Kafka" COPY native-image-configs $NATIVE_CONFIGS_DIR COPY native_command.sh native_command.sh -RUN mkdir $KAFKA_DIR; \ - microdnf install wget; \ - wget -nv -O kafka.tgz "$KAFKA_URL"; \ - wget -nv -O kafka.tgz.asc "$KAFKA_URL.asc"; \ +COPY *kafka.tgz /app + +RUN if [ -n "$KAFKA_URL" ]; then \ + microdnf install wget; \ + wget -nv -O kafka.tgz "$KAFKA_URL"; \ + wget -nv -O kafka.tgz.asc "$KAFKA_URL.asc"; \ + wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ + gpg --import KEYS; \ + gpg --batch --verify kafka.tgz.asc kafka.tgz; \ + fi; \ + mkdir $KAFKA_DIR; \ tar xfz kafka.tgz -C $KAFKA_DIR --strip-components 1; \ - wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ - gpg --import KEYS; \ - gpg --batch --verify kafka.tgz.asc kafka.tgz; \ - rm kafka.tgz ; \ # Build the native-binary of the apache kafka using graalVM native-image. /app/native_command.sh $NATIVE_IMAGE_PATH $NATIVE_CONFIGS_DIR $KAFKA_LIBS_DIR $TARGET_PATH diff --git a/docs/ops.html b/docs/ops.html index 61007a30b8f..d6c0982854a 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4116,11 +4116,9 @@ Feature: metadata.version SupportedMinVersion: 3.3-IV3 SupportedMaxVers


   $ bin/kafka-storage.sh format -t KAFKA_CLUSTER_ID --feature kraft.version=1 -c controller.properties
-  Cannot set kraft.version to 1 unless KIP-853 configuration is present. Try removing the --feature flag for kraft.version.
 

- Note: Currently it is not possible to convert clusters using a static controller quorum to - use a dynamic controller quorum. This function will be supported in the future release. + Note: To migrate from static voter set to dynamic voter set, please refer to the Upgrade section.

Add New Controller
If a dynamic controller cluster already exists, it can be expanded by first provisioning a new controller using the kafka-storage.sh tool and starting the controller. @@ -4318,7 +4316,7 @@ $ bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:

Try to send messages to the `tieredTopic` topic to roll the log segment:

-
$ bin/kafka-producer-perf-test.sh --topic tieredTopic --num-records 1200 --record-size 1024 --throughput -1 --producer-props bootstrap.servers=localhost:9092
+
$ bin/kafka-producer-perf-test.sh --bootstrap-server localhost:9092 --topic tieredTopic --num-records 1200 --record-size 1024 --throughput -1

Then, after the active segment is rolled, the old segment should be moved to the remote storage and get deleted. This can be verified by checking the remote log directory configured above. For example: diff --git a/docs/upgrade.html b/docs/upgrade.html index f0f3f540738..1dbb7e2d2ee 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -40,6 +40,10 @@

  • The PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG in ProducerConfig was deprecated and will be removed in Kafka 5.0. Please use the PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG instead.
  • +
  • + The ConsumerPerformance command line tool has a new include option that is alternative to the topic option. + This new option allows to pass a regular expression specifying a list of topics to include for consumption, which is useful to test consumer performance across multiple topics or dynamically matching topic sets. +
  • Upgrading to 4.1.0

    diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index bd957d7f8a7..6d333ab03f6 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -105,7 +105,7 @@ versions += [ kafka_38: "3.8.1", kafka_39: "3.9.1", kafka_40: "4.0.0", - log4j2: "2.24.3", + log4j2: "2.25.1", // When updating lz4 make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid lz4: "1.8.0", mavenArtifact: "3.9.6", 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 44380427284..f87af4897a7 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 @@ -833,19 +833,28 @@ public class GroupMetadataManager { * Gets or creates a streams group without updating the groups map. * The group will be materialized during the replay. * + * If there is an empty classic consumer group of the same name, it will be deleted and a new streams + * group will be created. + * * @param groupId The group ID. + * @param records The record list to which the group tombstones are written + * if the group is empty and is a classic group. * * @return A StreamsGroup. * * Package private for testing. */ StreamsGroup getOrCreateStreamsGroup( - String groupId + String groupId, + List records ) { Group group = groups.get(groupId); if (group == null) { return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); + } else if (maybeDeleteEmptyClassicGroup(group, records)) { + log.info("[GroupId {}] Converted the empty classic group to a streams group.", groupId); + return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); } else { return castToStreamsGroup(group); } @@ -1871,7 +1880,7 @@ public class GroupMetadataManager { boolean isJoining = memberEpoch == 0; StreamsGroup group; if (isJoining) { - group = getOrCreateStreamsGroup(groupId); + group = getOrCreateStreamsGroup(groupId, records); throwIfStreamsGroupIsFull(group); } else { group = getStreamsGroupOrThrow(groupId); @@ -6066,7 +6075,11 @@ public class GroupMetadataManager { // classicGroupJoinToConsumerGroup takes the join requests to non-empty consumer groups. // The empty consumer groups should be converted to classic groups in classicGroupJoinToClassicGroup. return classicGroupJoinToConsumerGroup((ConsumerGroup) group, context, request, responseFuture); - } else if (group.type() == CONSUMER || group.type() == CLASSIC) { + } else if (group.type() == CONSUMER || group.type() == CLASSIC || group.type() == STREAMS && group.isEmpty()) { + // classicGroupJoinToClassicGroup accepts: + // - classic groups + // - empty streams groups + // - empty consumer groups return classicGroupJoinToClassicGroup(context, request, responseFuture); } else { // Group exists but it's not a consumer group @@ -6107,6 +6120,8 @@ public class GroupMetadataManager { ClassicGroup group; if (maybeDeleteEmptyConsumerGroup(groupId, records)) { log.info("[GroupId {}] Converted the empty consumer group to a classic group.", groupId); + } else if (maybeDeleteEmptyStreamsGroup(groupId, records)) { + log.info("[GroupId {}] Converted the empty streams group to a classic group.", groupId); } boolean isNewGroup = !groups.containsKey(groupId); try { @@ -8398,6 +8413,13 @@ public class GroupMetadataManager { return group != null && group.type() == CONSUMER && group.isEmpty(); } + /** + * @return true if the group is an empty streams group. + */ + private static boolean isEmptyStreamsGroup(Group group) { + return group != null && group.type() == STREAMS && group.isEmpty(); + } + /** * Write tombstones for the group if it's empty and is a classic group. * @@ -8435,6 +8457,26 @@ public class GroupMetadataManager { } return false; } + + /** + * Delete and write tombstones for the group if it's empty and is a streams group. + * + * @param groupId The group id to be deleted. + * @param records The list of records to delete the group. + * + * @return true if the group is an empty streams group. + */ + private boolean maybeDeleteEmptyStreamsGroup(String groupId, List records) { + Group group = groups.get(groupId, Long.MAX_VALUE); + if (isEmptyStreamsGroup(group)) { + // Add tombstones for the previous streams group. The tombstones won't actually be + // replayed because its coordinator result has a non-null appendFuture. + createGroupTombstoneRecords(group, records); + removeGroup(groupId); + return true; + } + return false; + } /** * Checks whether the given protocol type or name in the request is inconsistent with the group's. 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 3f1e49b955d..efe2ad96435 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 @@ -18633,6 +18633,156 @@ public class GroupMetadataManagerTest { assertNull(result.response().data().partitionsByUserEndpoint()); } + @Test + public void testStreamsGroupHeartbeatWithNonEmptyClassicGroup() { + String classicGroupId = "classic-group-id"; + String memberId = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder().build(); + ClassicGroup classicGroup = new ClassicGroup( + new LogContext(), + classicGroupId, + EMPTY, + context.time + ); + context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(classicGroup, classicGroup.groupAssignment())); + + context.groupMetadataManager.getOrMaybeCreateClassicGroup(classicGroupId, false).transitionTo(PREPARING_REBALANCE); + assertThrows(GroupIdNotFoundException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(classicGroupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(12000) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()))); + } + + @Test + public void testStreamsGroupHeartbeatWithEmptyClassicGroup() { + String classicGroupId = "classic-group-id"; + String memberId = Uuid.randomUuid().toString(); + String fooTopicName = "foo"; + String subtopology1 = "subtopology1"; + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .build(); + ClassicGroup classicGroup = new ClassicGroup( + new LogContext(), + classicGroupId, + EMPTY, + context.time + ); + context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(classicGroup, classicGroup.groupAssignment())); + + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(classicGroupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(12000) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + StreamsGroupMember expectedMember = StreamsGroupMember.Builder.withDefaults(memberId) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(1) + .setPreviousMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setClientId(DEFAULT_CLIENT_ID) + .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) + .setAssignedTasks(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .setRebalanceTimeoutMs(12000) + .setTopologyEpoch(0) + .build(); + + assertEquals(Errors.NONE.code(), result.response().data().errorCode()); + assertEquals( + List.of( + GroupCoordinatorRecordHelpers.newGroupMetadataTombstoneRecord(classicGroupId), + StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(classicGroupId, expectedMember), + StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord(classicGroupId, topology), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(classicGroupId, 1, 0), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(classicGroupId, memberId, TasksTuple.EMPTY), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(classicGroupId, 1), + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(classicGroupId, expectedMember) + ), + result.records() + ); + assertEquals( + Group.GroupType.STREAMS, + context.groupMetadataManager.streamsGroup(classicGroupId).type() + ); + } + + @Test + public void testClassicGroupJoinWithEmptyStreamsGroup() throws Exception { + String streamsGroupId = "streams-group-id"; + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroup(new StreamsGroupBuilder(streamsGroupId, 10)) + .build(); + + JoinGroupRequestData request = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() + .withGroupId(streamsGroupId) + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + GroupMetadataManagerTestContext.JoinResult joinResult = context.sendClassicGroupJoin(request, true); + + List expectedRecords = List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochTombstoneRecord(streamsGroupId), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochTombstoneRecord(streamsGroupId), + StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecordTombstone(streamsGroupId) + ); + + assertEquals(Errors.MEMBER_ID_REQUIRED.code(), joinResult.joinFuture.get().errorCode()); + assertEquals(expectedRecords, joinResult.records.subList(0, expectedRecords.size())); + assertEquals( + Group.GroupType.CLASSIC, + context.groupMetadataManager.getOrMaybeCreateClassicGroup(streamsGroupId, false).type() + ); + } + + @Test + public void testClassicGroupJoinWithNonEmptyStreamsGroup() throws Exception { + String streamsGroupId = "streams-group-id"; + String memberId = Uuid.randomUuid().toString(); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroup(new StreamsGroupBuilder(streamsGroupId, 10) + .withMember(StreamsGroupMember.Builder.withDefaults(memberId) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .build())) + .build(); + + JoinGroupRequestData request = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() + .withGroupId(streamsGroupId) + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + GroupMetadataManagerTestContext.JoinResult joinResult = context.sendClassicGroupJoin(request); + assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), joinResult.joinFuture.get().errorCode()); + } + @Test public void testConsumerGroupDynamicConfigs() { String groupId = "fooup"; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsStickyAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsStickyAssignorBenchmark.java index 3293c0223f2..22863fef5fa 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsStickyAssignorBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsStickyAssignorBenchmark.java @@ -44,7 +44,6 @@ import org.openjdk.jmh.annotations.Threads; import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -107,7 +106,7 @@ public class StreamsStickyAssignorBenchmark { taskAssignor = new StickyTaskAssignor(); Map members = createMembers(); - this.assignmentConfigs = Collections.singletonMap( + this.assignmentConfigs = Map.of( "num.standby.replicas", Integer.toString(standbyReplicas) ); @@ -138,7 +137,7 @@ public class StreamsStickyAssignorBenchmark { for (Map.Entry member : groupSpec.members().entrySet()) { MemberAssignment memberAssignment = members.getOrDefault( member.getKey(), - new MemberAssignment(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()) + new MemberAssignment(Map.of(), Map.of(), Map.of()) ); updatedMemberSpec.put(member.getKey(), new AssignmentMemberSpec( diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/core/TestPurgatoryPerformance.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/core/TestPurgatoryPerformance.java index 68f3a52918c..e0fd3d8c8d4 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/core/TestPurgatoryPerformance.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/core/TestPurgatoryPerformance.java @@ -30,7 +30,6 @@ import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.Comparator; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.Random; import java.util.concurrent.CountDownLatch; @@ -405,12 +404,7 @@ public class TestPurgatoryPerformance { } - private static class Scheduled implements Delayed { - final FakeOperation operation; - - public Scheduled(FakeOperation operation) { - this.operation = operation; - } + private record Scheduled(FakeOperation operation) implements Delayed { @Override public long getDelay(TimeUnit unit) { @@ -429,30 +423,11 @@ public class TestPurgatoryPerformance { } } - private static class FakeOperationKey implements DelayedOperationKey { - private final String key; - - public FakeOperationKey(String key) { - this.key = key; - } - + private record FakeOperationKey(String key) implements DelayedOperationKey { @Override public String keyLabel() { return key; } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - FakeOperationKey that = (FakeOperationKey) o; - return Objects.equals(key, that.key); - } - - @Override - public int hashCode() { - return Objects.hash(key); - } } private static class FakeOperation extends DelayedOperation { @@ -469,7 +444,6 @@ public class TestPurgatoryPerformance { @Override public void onExpiration() { - } @Override diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokerControlStates.java b/metadata/src/main/java/org/apache/kafka/controller/BrokerControlStates.java index b3b4dc9414f..8114c9c6f21 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BrokerControlStates.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BrokerControlStates.java @@ -17,39 +17,5 @@ package org.apache.kafka.controller; -import java.util.Objects; - - -class BrokerControlStates { - private final BrokerControlState current; - private final BrokerControlState next; - - BrokerControlStates(BrokerControlState current, BrokerControlState next) { - this.current = current; - this.next = next; - } - - BrokerControlState current() { - return current; - } - - BrokerControlState next() { - return next; - } - - @Override - public int hashCode() { - return Objects.hash(current, next); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof BrokerControlStates other)) return false; - return other.current == current && other.next == next; - } - - @Override - public String toString() { - return "BrokerControlStates(current=" + current + ", next=" + next + ")"; - } +record BrokerControlStates(BrokerControlState current, BrokerControlState next) { } diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokerIdAndEpoch.java b/metadata/src/main/java/org/apache/kafka/controller/BrokerIdAndEpoch.java index 7ef75c4e3b3..ddb944ee77c 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BrokerIdAndEpoch.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BrokerIdAndEpoch.java @@ -17,41 +17,5 @@ package org.apache.kafka.controller; -import java.util.Objects; - -public class BrokerIdAndEpoch { - private final int id; - private final long epoch; - - public BrokerIdAndEpoch( - int id, - long epoch - ) { - this.id = id; - this.epoch = epoch; - } - - public int id() { - return id; - } - - public long epoch() { - return epoch; - } - - @Override - public boolean equals(Object o) { - if (o == null || (!(o instanceof BrokerIdAndEpoch other))) return false; - return id == other.id && epoch == other.epoch; - } - - @Override - public int hashCode() { - return Objects.hash(id, epoch); - } - - @Override - public String toString() { - return "BrokerIdAndEpoch(id=" + id + ", epoch=" + epoch + ")"; - } +public record BrokerIdAndEpoch(int id, long epoch) { } diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index a1d35293bd1..8697ad00962 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -421,8 +421,8 @@ public class PartitionChangeBuilder { PartitionReassignmentReplicas.CompletedReassignment completedReassignment = completedReassignmentOpt.get(); - targetIsr = completedReassignment.isr; - targetReplicas = completedReassignment.replicas; + targetIsr = completedReassignment.isr(); + targetReplicas = completedReassignment.replicas(); targetRemoving = List.of(); targetAdding = List.of(); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionReassignmentReplicas.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionReassignmentReplicas.java index 51fdf42bb67..5f85dd1a3d6 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionReassignmentReplicas.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionReassignmentReplicas.java @@ -129,14 +129,7 @@ class PartitionReassignmentReplicas { ); } - static class CompletedReassignment { - final List replicas; - final List isr; - - public CompletedReassignment(List replicas, List isr) { - this.replicas = replicas; - this.isr = isr; - } + record CompletedReassignment(List replicas, List isr) { } List originalReplicas() { diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index fe561244dab..a1e93b3f10f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -2441,14 +2441,7 @@ public class ReplicationControlManager { } } - private static final class IneligibleReplica { - private final int replicaId; - private final String reason; - - private IneligibleReplica(int replicaId, String reason) { - this.replicaId = replicaId; - this.reason = reason; - } + private record IneligibleReplica(int replicaId, String reason) { @Override public String toString() { diff --git a/metadata/src/main/java/org/apache/kafka/image/AclsImage.java b/metadata/src/main/java/org/apache/kafka/image/AclsImage.java index 12f778b841e..86112fa22fb 100644 --- a/metadata/src/main/java/org/apache/kafka/image/AclsImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/AclsImage.java @@ -30,14 +30,12 @@ import java.util.Map.Entry; /** * Represents the ACLs in the metadata image. - * + *

    * This class is thread-safe. */ -public final class AclsImage { +public record AclsImage(Map acls) { public static final AclsImage EMPTY = new AclsImage(Map.of()); - private final Map acls; - public AclsImage(Map acls) { this.acls = Collections.unmodifiableMap(acls); } @@ -46,10 +44,6 @@ public final class AclsImage { return acls.isEmpty(); } - public Map acls() { - return acls; - } - public void write(ImageWriter writer) { for (Entry entry : acls.entrySet()) { StandardAclWithId aclWithId = new StandardAclWithId(entry.getKey(), entry.getValue()); @@ -57,17 +51,6 @@ public final class AclsImage { } } - @Override - public int hashCode() { - return acls.hashCode(); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof AclsImage other)) return false; - return acls.equals(other.acls); - } - @Override public String toString() { return new AclsImageNode(this).stringify(); diff --git a/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java b/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java index 6cf23a2b688..e5dfb0433d0 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java @@ -45,14 +45,12 @@ import static org.apache.kafka.common.requests.DescribeClientQuotasRequest.MATCH /** * Represents the client quotas in the metadata image. - * + *

    * This class is thread-safe. */ -public final class ClientQuotasImage { +public record ClientQuotasImage(Map entities) { public static final ClientQuotasImage EMPTY = new ClientQuotasImage(Map.of()); - private final Map entities; - public ClientQuotasImage(Map entities) { this.entities = Collections.unmodifiableMap(entities); } @@ -61,11 +59,6 @@ public final class ClientQuotasImage { return entities.isEmpty(); } - // Visible for testing - public Map entities() { - return entities; - } - public void write(ImageWriter writer) { for (Entry entry : entities.entrySet()) { ClientQuotaEntity entity = entry.getKey(); @@ -82,14 +75,14 @@ public final class ClientQuotasImage { if (component.entityType().isEmpty()) { throw new InvalidRequestException("Invalid empty entity type."); } else if (exactMatch.containsKey(component.entityType()) || - typeMatch.contains(component.entityType())) { + typeMatch.contains(component.entityType())) { throw new InvalidRequestException("Entity type " + component.entityType() + " cannot appear more than once in the filter."); } if (!(component.entityType().equals(IP) || component.entityType().equals(USER) || - component.entityType().equals(CLIENT_ID))) { + component.entityType().equals(CLIENT_ID))) { throw new UnsupportedVersionException("Unsupported entity type " + - component.entityType()); + component.entityType()); } switch (component.matchType()) { case MATCH_TYPE_EXACT: @@ -119,7 +112,7 @@ public final class ClientQuotasImage { } if (exactMatch.containsKey(IP) || typeMatch.contains(IP)) { if ((exactMatch.containsKey(USER) || typeMatch.contains(USER)) || - (exactMatch.containsKey(CLIENT_ID) || typeMatch.contains(CLIENT_ID))) { + (exactMatch.containsKey(CLIENT_ID) || typeMatch.contains(CLIENT_ID))) { throw new InvalidRequestException("Invalid entity filter component " + "combination. IP filter component should not be used with " + "user or clientId filter component."); @@ -173,17 +166,6 @@ public final class ClientQuotasImage { return data; } - @Override - public boolean equals(Object o) { - if (!(o instanceof ClientQuotasImage other)) return false; - return entities.equals(other.entities); - } - - @Override - public int hashCode() { - return Objects.hash(entities); - } - @Override public String toString() { return new ClientQuotasImageNode(this).stringify(); diff --git a/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java b/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java index 22166f1c2a3..1e4dbe20d7a 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java @@ -25,22 +25,16 @@ import org.apache.kafka.metadata.ControllerRegistration; import java.util.Collections; import java.util.Map; -import java.util.Objects; - /** * Represents the cluster in the metadata image. - * + *

    * This class is thread-safe. */ -public final class ClusterImage { +public record ClusterImage(Map brokers, Map controllers) { public static final ClusterImage EMPTY = new ClusterImage( - Map.of(), - Map.of()); - - private final Map brokers; - - private final Map controllers; + Map.of(), + Map.of()); public ClusterImage( Map brokers, @@ -54,18 +48,10 @@ public final class ClusterImage { return brokers.isEmpty(); } - public Map brokers() { - return brokers; - } - public BrokerRegistration broker(int nodeId) { return brokers.get(nodeId); } - public Map controllers() { - return controllers; - } - public long brokerEpoch(int brokerId) { BrokerRegistration brokerRegistration = broker(brokerId); if (brokerRegistration == null) { @@ -89,18 +75,6 @@ public final class ClusterImage { } } - @Override - public int hashCode() { - return Objects.hash(brokers, controllers); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof ClusterImage other)) return false; - return brokers.equals(other.brokers) && - controllers.equals(other.controllers); - } - @Override public String toString() { return new ClusterImageNode(this).stringify(); diff --git a/metadata/src/main/java/org/apache/kafka/image/ConfigurationImage.java b/metadata/src/main/java/org/apache/kafka/image/ConfigurationImage.java index c0757d351a7..8147a4b58a8 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ConfigurationImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ConfigurationImage.java @@ -30,29 +30,10 @@ import java.util.Properties; /** * Represents the configuration of a resource. - * + *

    * This class is thread-safe. */ -public final class ConfigurationImage { - private final ConfigResource resource; - - private final Map data; - - public ConfigurationImage( - ConfigResource resource, - Map data - ) { - this.resource = resource; - this.data = data; - } - - public ConfigResource resource() { - return resource; - } - - public Map data() { - return data; - } +public record ConfigurationImage(ConfigResource resource, Map data) { public boolean isEmpty() { return data.isEmpty(); diff --git a/metadata/src/main/java/org/apache/kafka/image/DelegationTokenImage.java b/metadata/src/main/java/org/apache/kafka/image/DelegationTokenImage.java index c2c90a4bbb9..41339fc667c 100644 --- a/metadata/src/main/java/org/apache/kafka/image/DelegationTokenImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/DelegationTokenImage.java @@ -31,16 +31,14 @@ import java.util.Map.Entry; /** * Represents the DelegationToken credentials in the metadata image. - * + *

    * This class is thread-safe. + * + * @param tokens Map TokenID to TokenInformation. The TokenID is also contained in the TokenInformation inside the DelegationTokenData */ -public final class DelegationTokenImage { +public record DelegationTokenImage(Map tokens) { public static final DelegationTokenImage EMPTY = new DelegationTokenImage(Map.of()); - // Map TokenID to TokenInformation. - // The TokenID is also contained in the TokenInformation inside the DelegationTokenData - private final Map tokens; - public DelegationTokenImage(Map tokens) { this.tokens = Collections.unmodifiableMap(tokens); } @@ -55,31 +53,14 @@ public final class DelegationTokenImage { List tokenIds = new ArrayList<>(tokens.keySet()); String delegationTokenImageString = "DelegationTokenImage(" + String.join(", ", tokenIds) + ")"; options.handleLoss(delegationTokenImageString); - } + } } } - public Map tokens() { - return tokens; - } - public boolean isEmpty() { return tokens.isEmpty(); } - @Override - public int hashCode() { - return tokens.hashCode(); - } - - @Override - public boolean equals(Object o) { - if (o == null) return false; - if (!o.getClass().equals(DelegationTokenImage.class)) return false; - DelegationTokenImage other = (DelegationTokenImage) o; - return tokens.equals(other.tokens); - } - @Override public String toString() { return new DelegationTokenImageNode(this).stringify(); diff --git a/metadata/src/main/java/org/apache/kafka/image/LocalReplicaChanges.java b/metadata/src/main/java/org/apache/kafka/image/LocalReplicaChanges.java index e24b1455cb2..317641e4534 100644 --- a/metadata/src/main/java/org/apache/kafka/image/LocalReplicaChanges.java +++ b/metadata/src/main/java/org/apache/kafka/image/LocalReplicaChanges.java @@ -92,26 +92,6 @@ public final class LocalReplicaChanges { ); } - public static final class PartitionInfo { - private final Uuid topicId; - private final PartitionRegistration partition; - - public PartitionInfo(Uuid topicId, PartitionRegistration partition) { - this.topicId = topicId; - this.partition = partition; - } - - public Uuid topicId() { - return topicId; - } - - public PartitionRegistration partition() { - return partition; - } - - @Override - public String toString() { - return String.format("PartitionInfo(topicId = %s, partition = %s)", topicId, partition); - } + public record PartitionInfo(Uuid topicId, PartitionRegistration partition) { } } diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java index da245f00519..99364c039e7 100644 --- a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java @@ -22,15 +22,16 @@ import org.apache.kafka.image.writer.ImageWriter; import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.server.common.OffsetAndEpoch; -import java.util.Objects; - /** * The broker metadata image. - * + *

    * This class is thread-safe. */ -public final class MetadataImage { +public record MetadataImage(MetadataProvenance provenance, FeaturesImage features, ClusterImage cluster, + TopicsImage topics, ConfigurationsImage configs, ClientQuotasImage clientQuotas, + ProducerIdsImage producerIds, AclsImage acls, ScramImage scram, + DelegationTokenImage delegationTokens) { public static final MetadataImage EMPTY = new MetadataImage( MetadataProvenance.EMPTY, FeaturesImage.EMPTY, @@ -43,50 +44,6 @@ public final class MetadataImage { ScramImage.EMPTY, DelegationTokenImage.EMPTY); - private final MetadataProvenance provenance; - - private final FeaturesImage features; - - private final ClusterImage cluster; - - private final TopicsImage topics; - - private final ConfigurationsImage configs; - - private final ClientQuotasImage clientQuotas; - - private final ProducerIdsImage producerIds; - - private final AclsImage acls; - - private final ScramImage scram; - - private final DelegationTokenImage delegationTokens; - - public MetadataImage( - MetadataProvenance provenance, - FeaturesImage features, - ClusterImage cluster, - TopicsImage topics, - ConfigurationsImage configs, - ClientQuotasImage clientQuotas, - ProducerIdsImage producerIds, - AclsImage acls, - ScramImage scram, - DelegationTokenImage delegationTokens - ) { - this.provenance = provenance; - this.features = features; - this.cluster = cluster; - this.topics = topics; - this.configs = configs; - this.clientQuotas = clientQuotas; - this.producerIds = producerIds; - this.acls = acls; - this.scram = scram; - this.delegationTokens = delegationTokens; - } - public boolean isEmpty() { return features.isEmpty() && cluster.isEmpty() && @@ -99,10 +56,6 @@ public final class MetadataImage { delegationTokens.isEmpty(); } - public MetadataProvenance provenance() { - return provenance; - } - public OffsetAndEpoch highestOffsetAndEpoch() { return new OffsetAndEpoch(provenance.lastContainedOffset(), provenance.lastContainedEpoch()); } @@ -111,42 +64,6 @@ public final class MetadataImage { return provenance.lastContainedOffset(); } - public FeaturesImage features() { - return features; - } - - public ClusterImage cluster() { - return cluster; - } - - public TopicsImage topics() { - return topics; - } - - public ConfigurationsImage configs() { - return configs; - } - - public ClientQuotasImage clientQuotas() { - return clientQuotas; - } - - public ProducerIdsImage producerIds() { - return producerIds; - } - - public AclsImage acls() { - return acls; - } - - public ScramImage scram() { - return scram; - } - - public DelegationTokenImage delegationTokens() { - return delegationTokens; - } - public void write(ImageWriter writer, ImageWriterOptions options) { // Features should be written out first so we can include the metadata.version at the beginning of the // snapshot @@ -162,37 +79,6 @@ public final class MetadataImage { writer.close(true); } - @Override - public boolean equals(Object o) { - if (o == null || !o.getClass().equals(this.getClass())) return false; - MetadataImage other = (MetadataImage) o; - return provenance.equals(other.provenance) && - features.equals(other.features) && - cluster.equals(other.cluster) && - topics.equals(other.topics) && - configs.equals(other.configs) && - clientQuotas.equals(other.clientQuotas) && - producerIds.equals(other.producerIds) && - acls.equals(other.acls) && - scram.equals(other.scram) && - delegationTokens.equals(other.delegationTokens); - } - - @Override - public int hashCode() { - return Objects.hash( - provenance, - features, - cluster, - topics, - configs, - clientQuotas, - producerIds, - acls, - scram, - delegationTokens); - } - @Override public String toString() { return new MetadataImageNode(this).stringify(); diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java b/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java index e9cef3e70e6..fe6bf6f5333 100644 --- a/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java +++ b/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java @@ -20,48 +20,18 @@ package org.apache.kafka.image; import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.Snapshots; -import java.util.Objects; - /** * Information about the source of a metadata image. */ -public final class MetadataProvenance { +public record MetadataProvenance(long lastContainedOffset, int lastContainedEpoch, long lastContainedLogTimeMs, + boolean isOffsetBatchAligned) { public static final MetadataProvenance EMPTY = new MetadataProvenance(-1L, -1, -1L, false); - private final long lastContainedOffset; - private final int lastContainedEpoch; - private final long lastContainedLogTimeMs; - private final boolean isOffsetBatchAligned; - - public MetadataProvenance( - long lastContainedOffset, - int lastContainedEpoch, - long lastContainedLogTimeMs, - boolean isOffsetBatchAligned - ) { - this.lastContainedOffset = lastContainedOffset; - this.lastContainedEpoch = lastContainedEpoch; - this.lastContainedLogTimeMs = lastContainedLogTimeMs; - this.isOffsetBatchAligned = isOffsetBatchAligned; - } - public OffsetAndEpoch snapshotId() { return new OffsetAndEpoch(lastContainedOffset + 1, lastContainedEpoch); } - public long lastContainedOffset() { - return lastContainedOffset; - } - - public int lastContainedEpoch() { - return lastContainedEpoch; - } - - public long lastContainedLogTimeMs() { - return lastContainedLogTimeMs; - } - /** * Returns whether lastContainedOffset is the last offset in a record batch */ @@ -75,32 +45,4 @@ public final class MetadataProvenance { public String snapshotName() { return String.format("snapshot %s", Snapshots.filenameFromSnapshotId(snapshotId())); } - - @Override - public boolean equals(Object o) { - if (o == null || !o.getClass().equals(this.getClass())) return false; - MetadataProvenance other = (MetadataProvenance) o; - return lastContainedOffset == other.lastContainedOffset && - lastContainedEpoch == other.lastContainedEpoch && - lastContainedLogTimeMs == other.lastContainedLogTimeMs && - isOffsetBatchAligned == other.isOffsetBatchAligned; - } - - @Override - public int hashCode() { - return Objects.hash(lastContainedOffset, - lastContainedEpoch, - lastContainedLogTimeMs, - isOffsetBatchAligned); - } - - @Override - public String toString() { - return "MetadataProvenance(" + - "lastContainedOffset=" + lastContainedOffset + - ", lastContainedEpoch=" + lastContainedEpoch + - ", lastContainedLogTimeMs=" + lastContainedLogTimeMs + - ", isOffsetBatchAligned=" + isOffsetBatchAligned + - ")"; - } } diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChange.java b/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChange.java index d9b43a306e5..f03e1719950 100644 --- a/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChange.java +++ b/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChange.java @@ -21,28 +21,13 @@ import org.apache.kafka.server.common.MetadataVersion; import java.util.Objects; - /** * A change in the MetadataVersion. */ -public final class MetadataVersionChange { - private final MetadataVersion oldVersion; - private final MetadataVersion newVersion; - - public MetadataVersionChange( - MetadataVersion oldVersion, - MetadataVersion newVersion - ) { - this.oldVersion = Objects.requireNonNull(oldVersion); - this.newVersion = Objects.requireNonNull(newVersion); - } - - public MetadataVersion oldVersion() { - return oldVersion; - } - - public MetadataVersion newVersion() { - return newVersion; +public record MetadataVersionChange(MetadataVersion oldVersion, MetadataVersion newVersion) { + public MetadataVersionChange { + Objects.requireNonNull(oldVersion); + Objects.requireNonNull(newVersion); } public boolean isUpgrade() { @@ -52,26 +37,4 @@ public final class MetadataVersionChange { public boolean isDowngrade() { return newVersion.isLessThan(oldVersion); } - - @Override - public boolean equals(Object o) { - if (o == null || !o.getClass().equals(this.getClass())) return false; - MetadataVersionChange other = (MetadataVersionChange) o; - return oldVersion.equals(other.oldVersion) && - newVersion.equals(other.newVersion); - } - - @Override - public int hashCode() { - return Objects.hash(oldVersion, - newVersion); - } - - @Override - public String toString() { - return "MetadataVersionChange(" + - "oldVersion=" + oldVersion + - ", newVersion=" + newVersion + - ")"; - } } diff --git a/metadata/src/main/java/org/apache/kafka/image/ProducerIdsImage.java b/metadata/src/main/java/org/apache/kafka/image/ProducerIdsImage.java index ea3f76fdfa8..d1a0a4a7d12 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ProducerIdsImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ProducerIdsImage.java @@ -21,36 +21,22 @@ import org.apache.kafka.common.metadata.ProducerIdsRecord; import org.apache.kafka.image.node.ProducerIdsImageNode; import org.apache.kafka.image.writer.ImageWriter; -import java.util.Objects; - - /** * Stores the highest seen producer ID in the metadata image. - * + *

    * This class is thread-safe. + * + * @param nextProducerId The next producer ID, or -1 in the special case where no producer IDs have been issued. */ -public final class ProducerIdsImage { +public record ProducerIdsImage(long nextProducerId) { public static final ProducerIdsImage EMPTY = new ProducerIdsImage(-1L); - /** - * The next producer ID, or -1 in the special case where no producer IDs have been issued. - */ - private final long nextProducerId; - - public ProducerIdsImage(long nextProducerId) { - this.nextProducerId = nextProducerId; - } - - public long nextProducerId() { - return nextProducerId; - } - public void write(ImageWriter writer) { if (nextProducerId >= 0) { writer.write(0, new ProducerIdsRecord(). - setBrokerId(-1). - setBrokerEpoch(-1). - setNextProducerId(nextProducerId)); + setBrokerId(-1). + setBrokerEpoch(-1). + setNextProducerId(nextProducerId)); } } @@ -58,17 +44,6 @@ public final class ProducerIdsImage { return nextProducerId == EMPTY.nextProducerId; } - @Override - public boolean equals(Object o) { - if (!(o instanceof ProducerIdsImage other)) return false; - return nextProducerId == other.nextProducerId; - } - - @Override - public int hashCode() { - return Objects.hash(nextProducerId); - } - @Override public String toString() { return new ProducerIdsImageNode(this).stringify(); diff --git a/metadata/src/main/java/org/apache/kafka/image/ScramImage.java b/metadata/src/main/java/org/apache/kafka/image/ScramImage.java index c0c17a2a482..60c28d8f72e 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ScramImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ScramImage.java @@ -39,29 +39,27 @@ import java.util.Map.Entry; /** * Represents the SCRAM credentials in the metadata image. - * + *

    * This class is thread-safe. */ -public final class ScramImage { +public record ScramImage(Map> mechanisms) { public static final ScramImage EMPTY = new ScramImage(Map.of()); - private final Map> mechanisms; - - public ScramImage(Map> mechanisms) { - this.mechanisms = Collections.unmodifiableMap(mechanisms); + public ScramImage { + mechanisms = Collections.unmodifiableMap(mechanisms); } public void write(ImageWriter writer, ImageWriterOptions options) { if (options.metadataVersion().isScramSupported()) { - for (Entry> mechanismEntry : mechanisms.entrySet()) { - for (Entry userEntry : mechanismEntry.getValue().entrySet()) { + for (var mechanismEntry : mechanisms.entrySet()) { + for (var userEntry : mechanismEntry.getValue().entrySet()) { writer.write(0, userEntry.getValue().toRecord(userEntry.getKey(), mechanismEntry.getKey())); } } } else { boolean isEmpty = true; StringBuilder scramImageString = new StringBuilder("ScramImage({"); - for (Entry> mechanismEntry : mechanisms.entrySet()) { + for (var mechanismEntry : mechanisms.entrySet()) { if (!mechanismEntry.getValue().isEmpty()) { scramImageString.append(mechanismEntry.getKey()).append(":"); List users = new ArrayList<>(mechanismEntry.getValue().keySet()); @@ -87,7 +85,7 @@ public final class ScramImage { if ((users == null) || (users.isEmpty())) { // If there are no users listed then get all the users - for (Map scramCredentialDataSet : mechanisms.values()) { + for (var scramCredentialDataSet : mechanisms.values()) { for (String user : scramCredentialDataSet.keySet()) { uniqueUsers.put(user, false); } @@ -105,17 +103,17 @@ public final class ScramImage { DescribeUserScramCredentialsResponseData retval = new DescribeUserScramCredentialsResponseData(); - for (Map.Entry user : uniqueUsers.entrySet()) { + for (Entry user : uniqueUsers.entrySet()) { DescribeUserScramCredentialsResult result = new DescribeUserScramCredentialsResult().setUser(user.getKey()); if (!user.getValue()) { boolean dataFound = false; List credentialInfos = new ArrayList<>(); - for (Map.Entry> mechanismsEntry : mechanisms.entrySet()) { + for (var mechanismsEntry : mechanisms.entrySet()) { Map credentialDataSet = mechanismsEntry.getValue(); if (credentialDataSet.containsKey(user.getKey())) { credentialInfos.add(new CredentialInfo().setMechanism(mechanismsEntry.getKey().type()) - .setIterations(credentialDataSet.get(user.getKey()).iterations())); + .setIterations(credentialDataSet.get(user.getKey()).iterations())); dataFound = true; } } @@ -123,38 +121,21 @@ public final class ScramImage { result.setCredentialInfos(credentialInfos); } else { result.setErrorCode(Errors.RESOURCE_NOT_FOUND.code()) - .setErrorMessage(DESCRIBE_USER_THAT_DOES_NOT_EXIST + user.getKey()); + .setErrorMessage(DESCRIBE_USER_THAT_DOES_NOT_EXIST + user.getKey()); } } else { result.setErrorCode(Errors.DUPLICATE_RESOURCE.code()) - .setErrorMessage(DESCRIBE_DUPLICATE_USER + user.getKey()); + .setErrorMessage(DESCRIBE_DUPLICATE_USER + user.getKey()); } retval.results().add(result); } return retval; } - public Map> mechanisms() { - return mechanisms; - } - public boolean isEmpty() { return mechanisms.isEmpty(); } - @Override - public int hashCode() { - return mechanisms.hashCode(); - } - - @Override - public boolean equals(Object o) { - if (o == null) return false; - if (!o.getClass().equals(ScramImage.class)) return false; - ScramImage other = (ScramImage) o; - return mechanisms.equals(other.mechanisms); - } - @Override public String toString() { return new ScramImageNode(this).stringify(); diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicsImage.java b/metadata/src/main/java/org/apache/kafka/image/TopicsImage.java index 21dba62576a..36e73921932 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicsImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicsImage.java @@ -26,26 +26,14 @@ import org.apache.kafka.server.immutable.ImmutableMap; import org.apache.kafka.server.util.TranslatedValueMapView; import java.util.Map; -import java.util.Objects; /** * Represents the topics in the metadata image. - * + *

    * This class is thread-safe. */ -public final class TopicsImage { - public static final TopicsImage EMPTY = new TopicsImage(ImmutableMap.empty(), ImmutableMap.empty()); - - private final ImmutableMap topicsById; - private final ImmutableMap topicsByName; - - public TopicsImage( - ImmutableMap topicsById, - ImmutableMap topicsByName - ) { - this.topicsById = topicsById; - this.topicsByName = topicsByName; - } +public record TopicsImage(ImmutableMap topicsById, ImmutableMap topicsByName) { + public static final TopicsImage EMPTY = new TopicsImage(ImmutableMap.empty(), ImmutableMap.empty()); public TopicsImage including(TopicImage topic) { return new TopicsImage( @@ -57,14 +45,6 @@ public final class TopicsImage { return topicsById.isEmpty() && topicsByName.isEmpty(); } - public ImmutableMap topicsById() { - return topicsById; - } - - public ImmutableMap topicsByName() { - return topicsByName; - } - public PartitionRegistration getPartition(Uuid id, int partitionId) { TopicImage topicImage = topicsById.get(id); if (topicImage == null) return null; @@ -85,21 +65,9 @@ public final class TopicsImage { } } - @Override - public boolean equals(Object o) { - if (!(o instanceof TopicsImage other)) return false; - return topicsById.equals(other.topicsById) && - topicsByName.equals(other.topicsByName); - } - - @Override - public int hashCode() { - return Objects.hash(topicsById, topicsByName); - } - /** * Expose a view of this TopicsImage as a map from topic names to IDs. - * + *

    * Like TopicsImage itself, this map is immutable. */ public Map topicNameToIdView() { @@ -108,7 +76,7 @@ public final class TopicsImage { /** * Expose a view of this TopicsImage as a map from IDs to names. - * + *

    * Like TopicsImage itself, this map is immutable. */ public Map topicIdToNameView() { diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java b/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java index 5653a4689ea..cbd5e660c19 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java @@ -19,65 +19,16 @@ package org.apache.kafka.image.loader; import org.apache.kafka.image.MetadataProvenance; -import java.util.Objects; - - /** * Contains information about a snapshot that was loaded. + * + * @param provenance The source of this snapshot. + * @param elapsedNs The time in microseconds that it took to load the snapshot. */ -public class SnapshotManifest implements LoaderManifest { - /** - * The source of this snapshot. - */ - private final MetadataProvenance provenance; - - /** - * The time in microseconds that it took to load the snapshot. - */ - private final long elapsedNs; - - public SnapshotManifest( - MetadataProvenance provenance, - long elapsedNs - ) { - this.provenance = provenance; - this.elapsedNs = elapsedNs; - } +public record SnapshotManifest(MetadataProvenance provenance, long elapsedNs) implements LoaderManifest { @Override public LoaderManifestType type() { return LoaderManifestType.SNAPSHOT; } - - @Override - public MetadataProvenance provenance() { - return provenance; - } - - public long elapsedNs() { - return elapsedNs; - } - - @Override - public int hashCode() { - return Objects.hash( - provenance, - elapsedNs); - } - - @Override - public boolean equals(Object o) { - if (o == null || !o.getClass().equals(this.getClass())) return false; - SnapshotManifest other = (SnapshotManifest) o; - return provenance.equals(other.provenance) && - elapsedNs == other.elapsedNs; - } - - @Override - public String toString() { - return "SnapshotManifest(" + - "provenance=" + provenance + - ", elapsedNs=" + elapsedNs + - ")"; - } } diff --git a/metadata/src/main/java/org/apache/kafka/image/node/MetadataImageNode.java b/metadata/src/main/java/org/apache/kafka/image/node/MetadataImageNode.java index d9b0ae880f8..0eebac81521 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/MetadataImageNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/MetadataImageNode.java @@ -24,17 +24,15 @@ import java.util.Map; import java.util.function.Function; -public class MetadataImageNode implements MetadataNode { +/** + * @param image The metadata image. + */ +public record MetadataImageNode(MetadataImage image) implements MetadataNode { /** * The name of this node. */ public static final String NAME = "image"; - /** - * The metadata image. - */ - private final MetadataImage image; - private static final Map> CHILDREN = Map.of( ProvenanceNode.NAME, image -> new ProvenanceNode(image.provenance()), FeaturesImageNode.NAME, image -> new FeaturesImageNode(image.features()), @@ -48,14 +46,6 @@ public class MetadataImageNode implements MetadataNode { DelegationTokenImageNode.NAME, image -> new DelegationTokenImageNode(image.delegationTokens()) ); - public MetadataImageNode(MetadataImage image) { - this.image = image; - } - - public MetadataImage image() { - return image; - } - @Override public Collection childNames() { return CHILDREN.keySet(); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java index 9380bccbcfd..011c3c09952 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java @@ -17,76 +17,13 @@ package org.apache.kafka.metadata; -import java.util.Objects; - -public class BrokerHeartbeatReply { - /** - * True if the heartbeat reply should tell the broker that it has caught up. - */ - private final boolean isCaughtUp; - - /** - * True if the heartbeat reply should tell the broker that it is fenced. - */ - private final boolean isFenced; - - /** - * True if the broker is currently in a controlled shutdown state. - */ - private final boolean inControlledShutdown; - - /** - * True if the heartbeat reply should tell the broker that it should shut down. - */ - private final boolean shouldShutDown; - - public BrokerHeartbeatReply(boolean isCaughtUp, - boolean isFenced, - boolean inControlledShutdown, - boolean shouldShutDown) { - this.isCaughtUp = isCaughtUp; - this.isFenced = isFenced; - this.inControlledShutdown = inControlledShutdown; - this.shouldShutDown = shouldShutDown; - } - - public boolean isCaughtUp() { - return isCaughtUp; - } - - public boolean isFenced() { - return isFenced; - } - - public boolean inControlledShutdown() { - return inControlledShutdown; - } - - public boolean shouldShutDown() { - return shouldShutDown; - } - - @Override - public int hashCode() { - return Objects.hash(isCaughtUp, isFenced, inControlledShutdown, shouldShutDown); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof BrokerHeartbeatReply other)) return false; - return other.isCaughtUp == isCaughtUp && - other.isFenced == isFenced && - other.inControlledShutdown == inControlledShutdown && - other.shouldShutDown == shouldShutDown; - } - - @Override - public String toString() { - return "BrokerHeartbeatReply(isCaughtUp=" + isCaughtUp + - ", isFenced=" + isFenced + - ", inControlledShutdown=" + inControlledShutdown + - ", shouldShutDown = " + shouldShutDown + - ")"; - } +/** + * @param isCaughtUp True if the heartbeat reply should tell the broker that it has caught up. + * @param isFenced True if the heartbeat reply should tell the broker that it is fenced. + * @param inControlledShutdown True if the broker is currently in a controlled shutdown state. + * @param shouldShutDown True if the heartbeat reply should tell the broker that it should shut down. + */ +public record BrokerHeartbeatReply(boolean isCaughtUp, boolean isFenced, boolean inControlledShutdown, + boolean shouldShutDown) { } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistrationReply.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistrationReply.java index cfd86e67ab5..977ec2f9863 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistrationReply.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistrationReply.java @@ -17,33 +17,6 @@ package org.apache.kafka.metadata; -import java.util.Objects; - -public class BrokerRegistrationReply { - private final long epoch; - - public BrokerRegistrationReply(long epoch) { - this.epoch = epoch; - } - - public long epoch() { - return epoch; - } - - @Override - public int hashCode() { - return Objects.hash(epoch); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof BrokerRegistrationReply other)) return false; - return other.epoch == epoch; - } - - @Override - public String toString() { - return "BrokerRegistrationReply(epoch=" + epoch + ")"; - } +public record BrokerRegistrationReply(long epoch) { } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/DelegationTokenData.java b/metadata/src/main/java/org/apache/kafka/metadata/DelegationTokenData.java index df853169e19..9ed69f01081 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/DelegationTokenData.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/DelegationTokenData.java @@ -24,16 +24,13 @@ import org.apache.kafka.common.utils.SecurityUtils; import java.util.ArrayList; import java.util.List; -import java.util.Objects; /** * Represents the Delegation Tokens in the metadata image. - * + *

    * This class is thread-safe. */ -public final class DelegationTokenData { - - private final TokenInformation tokenInformation; +public record DelegationTokenData(TokenInformation tokenInformation) { public static DelegationTokenData fromRecord(DelegationTokenRecord record) { List renewers = new ArrayList<>(); @@ -50,14 +47,6 @@ public final class DelegationTokenData { record.expirationTimestamp())); } - public DelegationTokenData(TokenInformation tokenInformation) { - this.tokenInformation = tokenInformation; - } - - public TokenInformation tokenInformation() { - return tokenInformation; - } - public DelegationTokenRecord toRecord() { return new DelegationTokenRecord() .setOwner(tokenInformation.ownerAsString()) @@ -69,19 +58,6 @@ public final class DelegationTokenData { .setTokenId(tokenInformation.tokenId()); } - @Override - public int hashCode() { - return Objects.hash(tokenInformation); - } - - @Override - public boolean equals(Object o) { - if (o == null) return false; - if (!o.getClass().equals(DelegationTokenData.class)) return false; - DelegationTokenData other = (DelegationTokenData) o; - return tokenInformation.equals(other.tokenInformation); - } - /* * We explicitly hide tokenInformation when converting DelegationTokenData to string * For legacy reasons, we did not change TokenInformation to hide sensitive data. diff --git a/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java b/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java index 0a5af620f2a..d66f6ebe02c 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java @@ -19,18 +19,13 @@ package org.apache.kafka.metadata; import java.util.Collections; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; - /** * A map of feature names to their supported versions. */ -public class FinalizedControllerFeatures { - private final Map featureMap; - private final long epoch; - +public record FinalizedControllerFeatures(Map featureMap, long epoch) { public FinalizedControllerFeatures(Map featureMap, long epoch) { this.featureMap = Collections.unmodifiableMap(featureMap); this.epoch = epoch; @@ -47,31 +42,4 @@ public class FinalizedControllerFeatures { public Set featureNames() { return featureMap.keySet(); } - - public Map featureMap() { - return featureMap; - } - - public long epoch() { - return epoch; - } - - @Override - public int hashCode() { - return Objects.hash(featureMap, epoch); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof FinalizedControllerFeatures other)) return false; - return featureMap.equals(other.featureMap) && epoch == other.epoch; - } - - @Override - public String toString() { - return "FinalizedControllerFeatures(" + - "featureMap=" + featureMap.toString() + - ", epoch=" + epoch + - ")"; - } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/ScramCredentialData.java b/metadata/src/main/java/org/apache/kafka/metadata/ScramCredentialData.java index 4339a572b71..ff35881baaa 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/ScramCredentialData.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/ScramCredentialData.java @@ -26,51 +26,18 @@ import java.util.Objects; /** * Represents the ACLs in the metadata image. - * + *

    * This class is thread-safe. */ -public final class ScramCredentialData { - private final byte[] salt; - private final byte[] storedKey; - private final byte[] serverKey; - private final int iterations; - +public record ScramCredentialData(byte[] salt, byte[] storedKey, byte[] serverKey, int iterations) { public static ScramCredentialData fromRecord( UserScramCredentialRecord record ) { return new ScramCredentialData( - record.salt(), - record.storedKey(), - record.serverKey(), - record.iterations()); - } - - public ScramCredentialData( - byte[] salt, - byte[] storedKey, - byte[] serverKey, - int iterations - ) { - this.salt = salt; - this.storedKey = storedKey; - this.serverKey = serverKey; - this.iterations = iterations; - } - - public byte[] salt() { - return salt; - } - - public byte[] storedKey() { - return storedKey; - } - - public byte[] serverKey() { - return serverKey; - } - - public int iterations() { - return iterations; + record.salt(), + record.storedKey(), + record.serverKey(), + record.iterations()); } public UserScramCredentialRecord toRecord( @@ -78,12 +45,12 @@ public final class ScramCredentialData { ScramMechanism mechanism ) { return new UserScramCredentialRecord(). - setName(userName). - setMechanism(mechanism.type()). - setSalt(salt). - setStoredKey(storedKey). - setServerKey(serverKey). - setIterations(iterations); + setName(userName). + setMechanism(mechanism.type()). + setSalt(salt). + setStoredKey(storedKey). + setServerKey(serverKey). + setIterations(iterations); } public ScramCredential toCredential() { @@ -106,9 +73,9 @@ public final class ScramCredentialData { if (!o.getClass().equals(ScramCredentialData.class)) return false; ScramCredentialData other = (ScramCredentialData) o; return Arrays.equals(salt, other.salt) && - Arrays.equals(storedKey, other.storedKey) && - Arrays.equals(serverKey, other.serverKey) && - iterations == other.iterations; + Arrays.equals(storedKey, other.storedKey) && + Arrays.equals(serverKey, other.serverKey) && + iterations == other.iterations; } @Override diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAcl.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAcl.java index 8817b93bae4..21a142d3b80 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAcl.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAcl.java @@ -27,13 +27,13 @@ import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.security.auth.KafkaPrincipal; -import java.util.Objects; - /** * A Kafka ACLs which is identified by a UUID and stored in the metadata log. */ -public final class StandardAcl implements Comparable { +public record StandardAcl(ResourceType resourceType, String resourceName, PatternType patternType, String principal, + String host, AclOperation operation, + AclPermissionType permissionType) implements Comparable { public static StandardAcl fromRecord(AccessControlEntryRecord record) { return new StandardAcl( ResourceType.fromCode(record.resourceType()), @@ -56,47 +56,6 @@ public final class StandardAcl implements Comparable { acl.entry().permissionType()); } - private final ResourceType resourceType; - private final String resourceName; - private final PatternType patternType; - private final String principal; - private final String host; - private final AclOperation operation; - private final AclPermissionType permissionType; - - public StandardAcl( - ResourceType resourceType, - String resourceName, - PatternType patternType, - String principal, - String host, - AclOperation operation, - AclPermissionType permissionType) { - this.resourceType = resourceType; - this.resourceName = resourceName; - this.patternType = patternType; - this.principal = principal; - this.host = host; - this.operation = operation; - this.permissionType = permissionType; - } - - public ResourceType resourceType() { - return resourceType; - } - - public String resourceName() { - return resourceName; - } - - public PatternType patternType() { - return patternType; - } - - public String principal() { - return principal; - } - public KafkaPrincipal kafkaPrincipal() { int colonIndex = principal.indexOf(":"); if (colonIndex == -1) { @@ -108,18 +67,6 @@ public final class StandardAcl implements Comparable { return new KafkaPrincipal(principalType, principalName); } - public String host() { - return host; - } - - public AclOperation operation() { - return operation; - } - - public AclPermissionType permissionType() { - return permissionType; - } - public AclBinding toBinding() { ResourcePattern resourcePattern = new ResourcePattern(resourceType, resourceName, patternType); @@ -128,32 +75,6 @@ public final class StandardAcl implements Comparable { return new AclBinding(resourcePattern, accessControlEntry); } - @Override - public boolean equals(Object o) { - if (o == null || !o.getClass().equals(StandardAcl.class)) return false; - if (o == this) return true; - StandardAcl other = (StandardAcl) o; - return resourceType.equals(other.resourceType) && - resourceName.equals(other.resourceName) && - patternType.equals(other.patternType) && - principal.equals(other.principal) && - host.equals(other.host) && - operation.equals(other.operation) && - permissionType.equals(other.permissionType); - } - - @Override - public int hashCode() { - return Objects.hash( - resourceType, - resourceName, - patternType, - principal, - host, - operation, - permissionType); - } - /** * Compare two StandardAcl objects. See {@link StandardAuthorizerData#authorize} for an * explanation of why we want this particular sort order. @@ -176,17 +97,4 @@ public final class StandardAcl implements Comparable { result = permissionType.compareTo(other.permissionType); return result; } - - @Override - public String toString() { - return "StandardAcl(" + - "resourceType=" + resourceType + - ", resourceName=" + resourceName + - ", patternType=" + patternType + - ", principal=" + principal + - ", host=" + host + - ", operation=" + operation + - ", permissionType=" + permissionType + - ")"; - } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAclWithId.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAclWithId.java index 822a7752e0e..83a9d3f2bc9 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAclWithId.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAclWithId.java @@ -21,34 +21,14 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.metadata.AccessControlEntryRecord; -import java.util.Objects; - - /** * A tuple of (id, acl) */ -public final class StandardAclWithId { +public record StandardAclWithId(Uuid id, StandardAcl acl) { public static StandardAclWithId fromRecord(AccessControlEntryRecord record) { return new StandardAclWithId(record.id(), StandardAcl.fromRecord(record)); } - private final Uuid id; - private final StandardAcl acl; - - public StandardAclWithId(Uuid id, - StandardAcl acl) { - this.id = id; - this.acl = acl; - } - - public Uuid id() { - return id; - } - - public StandardAcl acl() { - return acl; - } - public AccessControlEntryRecord toRecord() { return new AccessControlEntryRecord(). setId(id). @@ -64,26 +44,4 @@ public final class StandardAclWithId { public AclBinding toBinding() { return acl.toBinding(); } - - @Override - public boolean equals(Object o) { - if (o == null || !o.getClass().equals(StandardAclWithId.class)) return false; - if (o == this) return true; - StandardAclWithId other = (StandardAclWithId) o; - return id.equals(other.id) && - acl.equals(other.acl); - } - - @Override - public int hashCode() { - return Objects.hash(id, acl); - } - - @Override - public String toString() { - return "StandardAclWithId(" + - "id=" + id + - ", acl=" + acl + - ")"; - } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java index be50a0c895b..54bb6ea4784 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java @@ -550,17 +550,7 @@ public class StandardAuthorizerData { } } - private static class DefaultRule implements MatchingRule { - private final AuthorizationResult result; - - private DefaultRule(AuthorizationResult result) { - this.result = result; - } - - @Override - public AuthorizationResult result() { - return result; - } + private record DefaultRule(AuthorizationResult result) implements MatchingRule { @Override public String toString() { @@ -568,19 +558,7 @@ public class StandardAuthorizerData { } } - private static class MatchingAclRule implements MatchingRule { - private final StandardAcl acl; - private final AuthorizationResult result; - - private MatchingAclRule(StandardAcl acl, AuthorizationResult result) { - this.acl = acl; - this.result = result; - } - - @Override - public AuthorizationResult result() { - return result; - } + private record MatchingAclRule(StandardAcl acl, AuthorizationResult result) implements MatchingRule { @Override public String toString() { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/placement/PlacementSpec.java b/metadata/src/main/java/org/apache/kafka/metadata/placement/PlacementSpec.java index 85daaf59e5d..c3247f1981c 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/placement/PlacementSpec.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/placement/PlacementSpec.java @@ -19,65 +19,9 @@ package org.apache.kafka.metadata.placement; import org.apache.kafka.common.annotation.InterfaceStability; -import java.util.Objects; - - /** * Specifies a replica placement that we want to make. */ @InterfaceStability.Unstable -public class PlacementSpec { - private final int startPartition; - - private final int numPartitions; - - private final short numReplicas; - - public PlacementSpec( - int startPartition, - int numPartitions, - short numReplicas - ) { - this.startPartition = startPartition; - this.numPartitions = numPartitions; - this.numReplicas = numReplicas; - } - - public int startPartition() { - return startPartition; - } - - public int numPartitions() { - return numPartitions; - } - - public short numReplicas() { - return numReplicas; - } - - @Override - public boolean equals(Object o) { - if (o == null) return false; - if (!(o.getClass().equals(this.getClass()))) return false; - PlacementSpec other = (PlacementSpec) o; - return startPartition == other.startPartition && - numPartitions == other.numPartitions && - numReplicas == other.numReplicas; - } - - @Override - public int hashCode() { - return Objects.hash(startPartition, - numPartitions, - numReplicas); - } - - @Override - public String toString() { - return "PlacementSpec" + - "(startPartition=" + startPartition + - ", numPartitions=" + numPartitions + - ", numReplicas=" + numReplicas + - ")"; - } +public record PlacementSpec(int startPartition, int numPartitions, short numReplicas) { } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java b/metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java index 88bdc5df96c..e34ae025174 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java @@ -18,42 +18,14 @@ package org.apache.kafka.metadata.placement; import java.util.List; -import java.util.Objects; /** * The topic assignment. - * + *

    * This class is immutable. It's internal state does not change. */ -public class TopicAssignment { - private final List assignments; - - public TopicAssignment(List assignments) { - this.assignments = List.copyOf(assignments); - } - - /** - * @return The replica assignment for each partition, where the index in the list corresponds to different partition. - */ - public List assignments() { - return assignments; - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof TopicAssignment other)) return false; - return assignments.equals(other.assignments); - } - - @Override - public int hashCode() { - return Objects.hash(assignments); - } - - @Override - public String toString() { - return "TopicAssignment" + - "(assignments=" + assignments + - ")"; +public record TopicAssignment(List assignments) { + public TopicAssignment { + assignments = List.copyOf(assignments); } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/placement/UsableBroker.java b/metadata/src/main/java/org/apache/kafka/metadata/placement/UsableBroker.java index 17f531d023f..2c24d937f9e 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/placement/UsableBroker.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/placement/UsableBroker.java @@ -19,58 +19,11 @@ package org.apache.kafka.metadata.placement; import org.apache.kafka.common.annotation.InterfaceStability; -import java.util.Objects; import java.util.Optional; - /** * A broker where a replica can be placed. */ @InterfaceStability.Unstable -public class UsableBroker { - private final int id; - - private final Optional rack; - - private final boolean fenced; - - public UsableBroker(int id, Optional rack, boolean fenced) { - this.id = id; - this.rack = rack; - this.fenced = fenced; - } - - public int id() { - return id; - } - - public Optional rack() { - return rack; - } - - public boolean fenced() { - return fenced; - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof UsableBroker other)) return false; - return other.id == id && other.rack.equals(rack) && other.fenced == fenced; - } - - @Override - public int hashCode() { - return Objects.hash(id, - rack, - fenced); - } - - @Override - public String toString() { - return "UsableBroker" + - "(id=" + id + - ", rack=" + rack + - ", fenced=" + fenced + - ")"; - } +public record UsableBroker(int id, Optional rack, boolean fenced) { } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileReader.java b/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileReader.java index 11f6403a850..7ac61b9e502 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileReader.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileReader.java @@ -69,22 +69,7 @@ public final class BatchFileReader implements Iterator batch; - private final boolean isControl; - - public BatchAndType(Batch batch, boolean isControl) { - this.batch = batch; - this.isControl = isControl; - } - - public Batch batch() { - return batch; - } - - public boolean isControl() { - return isControl; - } + public record BatchAndType(Batch batch, boolean isControl) { } private final FileRecords fileRecords; diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java index 1a9cd50bd1d..33c540b9d14 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java @@ -113,8 +113,8 @@ public class PartitionReassignmentReplicasTest { replicas.maybeCompleteReassignment(List.of(0, 1, 2, 3, 4, 5)); assertTrue(reassignmentOptional.isPresent()); PartitionReassignmentReplicas.CompletedReassignment completedReassignment = reassignmentOptional.get(); - assertEquals(List.of(3, 4, 5), completedReassignment.isr); - assertEquals(List.of(3, 4, 5), completedReassignment.replicas); + assertEquals(List.of(3, 4, 5), completedReassignment.isr()); + assertEquals(List.of(3, 4, 5), completedReassignment.replicas()); } @Test @@ -126,8 +126,8 @@ public class PartitionReassignmentReplicasTest { replicas.maybeCompleteReassignment(List.of(0, 1, 2, 3)); assertTrue(reassignmentOptional.isPresent()); PartitionReassignmentReplicas.CompletedReassignment completedReassignment = reassignmentOptional.get(); - assertEquals(List.of(0, 1, 3), completedReassignment.isr); - assertEquals(List.of(0, 1, 3), completedReassignment.replicas); + assertEquals(List.of(0, 1, 3), completedReassignment.isr()); + assertEquals(List.of(0, 1, 3), completedReassignment.replicas()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/PeriodicTaskControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/PeriodicTaskControlManagerTest.java index aebdb596c6c..2d62fdd02b1 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PeriodicTaskControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PeriodicTaskControlManagerTest.java @@ -64,20 +64,7 @@ public class PeriodicTaskControlManagerTest { } } - static class TrackedTask { - final String tag; - final long deadlineNs; - final Supplier> op; - - TrackedTask( - String tag, - long deadlineNs, - Supplier> op - ) { - this.tag = tag; - this.deadlineNs = deadlineNs; - this.op = op; - } + record TrackedTask(String tag, long deadlineNs, Supplier> op) { } static class PeriodicTaskControlManagerTestEnv implements PeriodicTaskControlManager.QueueAccessor { diff --git a/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java b/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java index 323cc5abf9a..4cfd4334694 100644 --- a/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java @@ -24,6 +24,7 @@ import org.junit.jupiter.api.Timeout; 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; @@ -57,4 +58,22 @@ public class MetadataVersionChangeTest { "is changing from " + MetadataVersion.latestProduction() + " to " + MetadataVersion.MINIMUM_VERSION, new MetadataVersionChangeException(CHANGE_LATEST_TO_MINIMUM).toString()); } + + @Test + public void testConstructorThrowsExceptionWhenOldVersionIsNull() { + assertThrows(NullPointerException.class, () -> + new MetadataVersionChange(null, MetadataVersion.MINIMUM_VERSION)); + } + + @Test + public void testConstructorThrowsExceptionWhenNewVersionIsNull() { + assertThrows(NullPointerException.class, () -> + new MetadataVersionChange(MetadataVersion.MINIMUM_VERSION, null)); + } + + @Test + public void testConstructorThrowsExceptionWhenBothVersionsAreNull() { + assertThrows(NullPointerException.class, () -> + new MetadataVersionChange(null, null)); + } } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java index c022ae79893..bd15406cbb0 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java @@ -106,22 +106,7 @@ public class RecordTestUtils { } } - public static class ImageDeltaPair { - private final Supplier imageSupplier; - private final Function deltaCreator; - - public ImageDeltaPair(Supplier imageSupplier, Function deltaCreator) { - this.imageSupplier = imageSupplier; - this.deltaCreator = deltaCreator; - } - - public Supplier imageSupplier() { - return imageSupplier; - } - - public Function deltaCreator() { - return deltaCreator; - } + public record ImageDeltaPair(Supplier imageSupplier, Function deltaCreator) { } public static class TestThroughAllIntermediateImagesLeadingToFinalImageHelper { diff --git a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java index b2f9fa68b3a..74d5a49d7ea 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java @@ -569,9 +569,9 @@ public class StandardAuthorizerTest { String expectedAuditLog = "Principal = User:bob is Denied operation = READ " + "from host = 127.0.0.1 on resource = Topic:LITERAL:alpha for request = Fetch " + - "with resourceRefCount = 1 based on rule MatchingAcl(acl=StandardAcl(resourceType=TOPIC, " + + "with resourceRefCount = 1 based on rule MatchingAcl(acl=StandardAcl[resourceType=TOPIC, " + "resourceName=alp, patternType=PREFIXED, principal=User:bob, host=*, operation=READ, " + - "permissionType=DENY))"; + "permissionType=DENY])"; if (logIfDenied) { Mockito.verify(auditLog).info(expectedAuditLog); @@ -611,9 +611,9 @@ public class StandardAuthorizerTest { String expectedAuditLog = "Principal = User:bob is Allowed operation = READ " + "from host = 127.0.0.1 on resource = Topic:LITERAL:green1 for request = Fetch " + - "with resourceRefCount = 1 based on rule MatchingAcl(acl=StandardAcl(resourceType=TOPIC, " + + "with resourceRefCount = 1 based on rule MatchingAcl(acl=StandardAcl[resourceType=TOPIC, " + "resourceName=green, patternType=PREFIXED, principal=User:bob, host=*, operation=READ, " + - "permissionType=ALLOW))"; + "permissionType=ALLOW])"; if (logIfAllowed) { Mockito.verify(auditLog).debug(expectedAuditLog); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java index 3551e2ce7b0..81625151dba 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java @@ -85,7 +85,7 @@ public class TopicAssignmentTest { new PartitionAssignment(replicas, directories::get) ); TopicAssignment topicAssignment = new TopicAssignment(partitionAssignments); - assertEquals("TopicAssignment(assignments=[PartitionAssignment(replicas=[0, 1, 2], " + - "directories=[v56qeYzNRrqNtXsxzcReog, MvUIAsOiRlSePeiBHdZrSQ, jUqCchHtTHqMxeVv4dw1RA])])", topicAssignment.toString()); + assertEquals("TopicAssignment[assignments=[PartitionAssignment(replicas=[0, 1, 2], " + + "directories=[v56qeYzNRrqNtXsxzcReog, MvUIAsOiRlSePeiBHdZrSQ, jUqCchHtTHqMxeVv4dw1RA])]]", topicAssignment.toString()); } } diff --git a/release/README.md b/release/README.md index 1f56f679011..bde487fff35 100644 --- a/release/README.md +++ b/release/README.md @@ -25,7 +25,7 @@ pip install -r requirements.txt # Usage -To start a release, first activate the virutalenv, and then run +To start a release, first activate the virtualenv, and then run the release script. ``` diff --git a/release/git.py b/release/git.py index 9cb106df2fd..02d7dc8cbe2 100644 --- a/release/git.py +++ b/release/git.py @@ -136,4 +136,3 @@ def push_ref(ref, remote=push_remote_name, **kwargs): def merge_ref(ref, **kwargs): __defaults(kwargs) cmd(f"Merging ref {ref}", f"git merge {ref}") - diff --git a/release/gpg.py b/release/gpg.py index 6b396119265..347389e3089 100644 --- a/release/gpg.py +++ b/release/gpg.py @@ -32,7 +32,7 @@ def key_exists(key_id): """ try: execute(f"gpg --list-keys {key_id}") - except Exception as e: + except Exception: return False return True @@ -70,13 +70,13 @@ def valid_passphrase(key_id, passphrase): with tempfile.TemporaryDirectory() as tmpdir: content = __file__ signature = tmpdir + '/sig.asc' - # if the agent is running, the suplied passphrase may be ignored + # if the agent is running, the supplied passphrase may be ignored agent_kill() try: sign(key_id, passphrase, content, signature) verify(content, signature) - except subprocess.CalledProcessError as e: - False + except subprocess.CalledProcessError: + return False return True @@ -88,5 +88,3 @@ def key_pass_id(key_id, passphrase): h.update(key_id.encode()) h.update(passphrase.encode()) return h.hexdigest() - - diff --git a/release/notes.py b/release/notes.py index 529c27b3728..e561fa03a20 100644 --- a/release/notes.py +++ b/release/notes.py @@ -41,13 +41,13 @@ def query(query, **kwargs): Any additional keyword arguments are forwarded to jira.search_issues. """ results = [] - startAt = 0 + start_at = 0 new_results = None jira = JIRA(JIRA_BASE_URL) while new_results is None or len(new_results) == MAX_RESULTS: - new_results = jira.search_issues(query, startAt=startAt, maxResults=MAX_RESULTS, **kwargs) + new_results = jira.search_issues(query, startAt=start_at, maxResults=MAX_RESULTS, **kwargs) results += new_results - startAt += len(new_results) + start_at += len(new_results) return results @@ -172,5 +172,3 @@ if __name__ == "__main__": except Exception as e: print(e, file=sys.stderr) sys.exit(1) - - diff --git a/release/preferences.py b/release/preferences.py index 9b57554585a..f32f1650eed 100644 --- a/release/preferences.py +++ b/release/preferences.py @@ -89,5 +89,3 @@ def as_json(): Export all saved preferences in JSON format. """ json.dumps(prefs, indent=2) - - diff --git a/release/release.py b/release/release.py index d0cba6f1782..92b76dee1e3 100644 --- a/release/release.py +++ b/release/release.py @@ -218,7 +218,7 @@ def verify_gpg_key(): if not gpg.key_exists(gpg_key_id): fail(f"GPG key {gpg_key_id} not found") if not gpg.valid_passphrase(gpg_key_id, gpg_passphrase): - fail(f"GPG passprase not valid for key {gpg_key_id}") + fail(f"GPG passphrase not valid for key {gpg_key_id}") preferences.once("verify_requirements", lambda: confirm_or_fail(templates.requirements_instructions(preferences.FILE, preferences.as_json()))) @@ -232,12 +232,12 @@ apache_id = preferences.get('apache_id', lambda: prompt("Please enter your apach jdk21_env = get_jdk(21) -def verify_prerequeisites(): +def verify_prerequisites(): print("Begin to check if you have met all the pre-requisites for the release process") def prereq(name, soft_check): try: result = soft_check() - if result == False: + if not result: fail(f"Pre-requisite not met: {name}") else: print(f"Pre-requisite met: {name}") @@ -250,7 +250,7 @@ def verify_prerequeisites(): return True -preferences.once(f"verify_prerequeisites", verify_prerequeisites) +preferences.once(f"verify_prerequisites", verify_prerequisites) # Validate that the release doesn't already exist git.fetch_tags() @@ -360,7 +360,7 @@ cmd("Building and uploading archives", "mvn deploy -Pgpg-signing", cwd=os.path.j # TODO: Many of these suggested validation steps could be automated # and would help pre-validate a lot of the stuff voters test -print(templates.sanity_check_instructions(release_version, rc_tag, apache_id)) +print(templates.sanity_check_instructions(release_version, rc_tag)) confirm_or_fail("Have you sufficiently verified the release artifacts?") # TODO: Can we close the staging repository via a REST API since we @@ -376,6 +376,5 @@ git.reset_hard_head() git.switch_branch(starting_branch) git.delete_branch(release_version) -rc_vote_email_text = templates.rc_vote_email_text(release_version, rc, rc_tag, dev_branch, docs_release_version, apache_id) +rc_vote_email_text = templates.rc_vote_email_text(release_version, rc, rc_tag, dev_branch, docs_release_version) print(templates.rc_email_instructions(rc_vote_email_text)) - diff --git a/release/runtime.py b/release/runtime.py index c0d5d67eafb..28765f0d92b 100644 --- a/release/runtime.py +++ b/release/runtime.py @@ -108,7 +108,7 @@ def _prefix(prefix_str, value_str): def cmd(action, cmd_arg, *args, **kwargs): """ - Execute an external command. This should be preferered over execute() + Execute an external command. This should be preferred over execute() when returning the output is not necessary, as the user will be given the option of retrying in case of a failure. """ @@ -144,5 +144,3 @@ def cmd(action, cmd_arg, *args, **kwargs): print(templates.cmd_failed()) fail("") - - diff --git a/release/svn.py b/release/svn.py index 4869f79ddf1..41c2fc6d28a 100644 --- a/release/svn.py +++ b/release/svn.py @@ -27,7 +27,7 @@ from runtime import cmd SVN_DEV_URL="https://dist.apache.org/repos/dist/dev/kafka" -def delete_old_rc_directory_if_needed(rc_tag, src, work_dir): +def delete_old_rc_directory_if_needed(rc_tag, work_dir): svn_dev = os.path.join(work_dir, "svn_dev") cmd_desc = f"Check if {rc_tag} exists in the subversion repository." cmd_str = f"svn info --show-item revision {SVN_DEV_URL}/{rc_tag}" @@ -39,7 +39,7 @@ def delete_old_rc_directory_if_needed(rc_tag, src, work_dir): cmd(cmd_desc, cmd_str, cwd = svn_dev) def commit_artifacts(rc_tag, src, work_dir): - delete_old_rc_directory_if_needed(rc_tag, src, work_dir) + delete_old_rc_directory_if_needed(rc_tag, work_dir) svn_dev = os.path.join(work_dir, "svn_dev") dst = os.path.join(svn_dev, rc_tag) print(f"Copying {src} to {dst}") diff --git a/release/templates.py b/release/templates.py index e067f1eb475..2f9f07b0e3c 100644 --- a/release/templates.py +++ b/release/templates.py @@ -154,11 +154,11 @@ Go to https://repository.apache.org/#stagingRepositories and hit 'Close' for the There will be more than one repository entries created, please close all of them. In some cases, you may get errors on some repositories while closing them, see KAFKA-15033. If this is not the first RC, you need to 'Drop' the previous artifacts. -Confirm the correct artifacts are visible at https://repository.apache.org/content/groups/staging/org/apache/kafka/ +Confirm the correct artifacts are visible at https://repository.apache.org/content/groups/staging/org/apache/kafka/ and build the +jvm and native Docker images following these instructions: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=34840886#ReleaseProcess-CreateJVMApacheKafkaDockerArtifacts(Forversions>=3.7.0) """ - -def sanity_check_instructions(release_version, rc_tag, apache_id): +def sanity_check_instructions(release_version, rc_tag): return f""" ******************************************************************************************************************************************************* Ok. We've built and staged everything for the {rc_tag}. @@ -189,14 +189,14 @@ Some suggested steps: """ -def rc_vote_email_text(release_version, rc, rc_tag, dev_branch, docs_version, apache_id): +def rc_vote_email_text(release_version, rc, rc_tag, dev_branch, docs_version): return f""" To: dev@kafka.apache.org, users@kafka.apache.org, kafka-clients@googlegroups.com Subject: [VOTE] {release_version} RC{rc} Hello Kafka users, developers and client-developers, -This is the first candidate for release of Apache Kafka {release_version}. +This is the candidate for release of Apache Kafka {release_version}. @@ -221,7 +221,7 @@ apache/kafka-native:{rc_tag} https://repository.apache.org/content/groups/staging/org/apache/kafka/ * Javadoc: -https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/javadoc/ +https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/javadoc/index.html * Tag to be voted upon (off {dev_branch} branch) is the {release_version} tag: https://github.com/apache/kafka/releases/tag/{rc_tag} @@ -233,17 +233,16 @@ https://kafka.apache.org/{docs_version}/documentation.html https://kafka.apache.org/{docs_version}/protocol.html * Successful CI builds for the {dev_branch} branch: -Unit/integration tests: https://ci-builds.apache.org/job/Kafka/job/kafka/job/{dev_branch}// --- Confluent engineers can access the semphore build to provide the build number -System tests: https://confluent-open-source-kafka-system-test-results.s3-us-west-2.amazonaws.com/{dev_branch}//report.html +Unit/integration tests: https://github.com/apache/kafka/actions/runs/ +System tests: +/report.html> * Successful Docker Image Github Actions Pipeline for {dev_branch} branch: Docker Build Test Pipeline (JVM): https://github.com/apache/kafka/actions/runs/ Docker Build Test Pipeline (Native): https://github.com/apache/kafka/actions/runs/ -/************************************** - Thanks, """ @@ -294,5 +293,3 @@ IMPORTANT: Note that there are still some substitutions that need to be made in - Finally, validate all the links before shipping! Note that all substitutions are annotated with <> around them. """ - - diff --git a/release/textfiles.py b/release/textfiles.py index 7598b96067a..20752659c4b 100644 --- a/release/textfiles.py +++ b/release/textfiles.py @@ -71,5 +71,3 @@ def replace(path, pattern, replacement, **kwargs): with open(path, "w") as f: for line in updated: f.write(line) - - 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 dd7c5937bdc..ceca9a6a7de 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 @@ -283,7 +283,9 @@ public enum MetadataVersion { } public short listOffsetRequestVersion() { - if (this.isAtLeast(IBP_4_0_IV3)) { + if (this.isAtLeast(IBP_4_2_IV1)) { + return 11; + } else if (this.isAtLeast(IBP_4_0_IV3)) { return 10; } else if (this.isAtLeast(IBP_3_9_IV0)) { return 9; diff --git a/server-common/src/main/java/org/apache/kafka/server/util/CommandLineUtils.java b/server-common/src/main/java/org/apache/kafka/server/util/CommandLineUtils.java index 6146daeb45c..c5b973f78e7 100644 --- a/server-common/src/main/java/org/apache/kafka/server/util/CommandLineUtils.java +++ b/server-common/src/main/java/org/apache/kafka/server/util/CommandLineUtils.java @@ -23,10 +23,12 @@ import org.apache.kafka.common.utils.Exit; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.stream.Collectors; import joptsimple.OptionParser; import joptsimple.OptionSet; @@ -135,6 +137,29 @@ public class CommandLineUtils { Exit.exit(1, message); } + /** + * Check that exactly one of a set of mutually exclusive arguments is present. + */ + public static void checkOneOfArgs(OptionParser parser, OptionSet options, OptionSpec... optionSpecs) { + if (optionSpecs == null || optionSpecs.length == 0) { + throw new IllegalArgumentException("At least one option must be provided"); + } + + int presentCount = 0; + for (OptionSpec spec : optionSpecs) { + if (options.has(spec)) { + presentCount++; + } + } + + if (presentCount != 1) { + printUsageAndExit(parser, "Exactly one of the following arguments is required: " + + Arrays.stream(optionSpecs) + .map(Object::toString) + .collect(Collectors.joining(", "))); + } + } + public static void printUsageAndExit(OptionParser parser, String message) { System.err.println(message); try { 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 508d4bd900b..49a200f6225 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 @@ -266,8 +266,8 @@ class MetadataVersionTest { @ParameterizedTest @EnumSource(value = MetadataVersion.class) public void testListOffsetsValueVersion(MetadataVersion metadataVersion) { - final short expectedVersion = 10; - if (metadataVersion.isAtLeast(IBP_4_0_IV3)) { + final short expectedVersion = 11; + if (metadataVersion.isAtLeast(IBP_4_2_IV1)) { assertEquals(expectedVersion, metadataVersion.listOffsetRequestVersion()); } else { assertTrue(metadataVersion.listOffsetRequestVersion() < expectedVersion); diff --git a/server-common/src/test/java/org/apache/kafka/server/util/CommandLineUtilsTest.java b/server-common/src/test/java/org/apache/kafka/server/util/CommandLineUtilsTest.java index 8fdc6c89d06..a634b21403e 100644 --- a/server-common/src/test/java/org/apache/kafka/server/util/CommandLineUtilsTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/util/CommandLineUtilsTest.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.server.util; +import org.apache.kafka.common.utils.Exit; + import org.junit.jupiter.api.Test; import java.util.List; @@ -26,9 +28,12 @@ import joptsimple.OptionParser; import joptsimple.OptionSet; import joptsimple.OptionSpec; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class CommandLineUtilsTest { @Test @@ -266,4 +271,105 @@ public class CommandLineUtilsTest { () -> CommandLineUtils.initializeBootstrapProperties(createTestProps(), Optional.of("127.0.0.2:9094"), Optional.of("127.0.0.3:9095"))).getMessage()); } + + private OptionSpec createMockOptionSpec(String name) { + OptionSpec spec = mock(OptionSpec.class); + when(spec.toString()).thenReturn("[" + name.replaceAll("--", "") + "]"); + return spec; + } + + @Test + void testCheckOneOfArgsNoOptions() { + OptionParser parser = mock(OptionParser.class); + OptionSet options = mock(OptionSet.class); + + IllegalArgumentException e = assertThrows(IllegalArgumentException.class, () -> + CommandLineUtils.checkOneOfArgs(parser, options) + ); + + assertEquals("At least one option must be provided", e.getMessage()); + } + + @Test + void testCheckOneOfArgsOnePresent() { + OptionParser parser = mock(OptionParser.class); + OptionSet options = mock(OptionSet.class); + OptionSpec opt1 = createMockOptionSpec("--first-option"); + OptionSpec opt2 = createMockOptionSpec("--second-option"); + OptionSpec opt3 = createMockOptionSpec("--third-option"); + + when(options.has(opt1)).thenReturn(true); + when(options.has(opt2)).thenReturn(false); + when(options.has(opt3)).thenReturn(false); + + assertDoesNotThrow(() -> + CommandLineUtils.checkOneOfArgs(parser, options, opt1, opt2, opt3) + ); + + when(options.has(opt1)).thenReturn(false); + when(options.has(opt2)).thenReturn(true); + + assertDoesNotThrow(() -> + CommandLineUtils.checkOneOfArgs(parser, options, opt1, opt2, opt3) + ); + + when(options.has(opt2)).thenReturn(false); + when(options.has(opt3)).thenReturn(true); + + assertDoesNotThrow(() -> + CommandLineUtils.checkOneOfArgs(parser, options, opt1, opt2, opt3) + ); + } + + @Test + void testCheckOneOfArgsNonePresent() { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + OptionParser parser = mock(OptionParser.class); + OptionSet options = mock(OptionSet.class); + OptionSpec opt1 = createMockOptionSpec("--first-option"); + OptionSpec opt2 = createMockOptionSpec("--second-option"); + OptionSpec opt3 = createMockOptionSpec("--third-option"); + + when(options.has(opt1)).thenReturn(false); + when(options.has(opt2)).thenReturn(false); + when(options.has(opt3)).thenReturn(false); + + try { + IllegalArgumentException e = assertThrows(IllegalArgumentException.class, + () -> CommandLineUtils.checkOneOfArgs(parser, options, opt1, opt2, opt3)); + assertEquals("Exactly one of the following arguments is required: " + + "[first-option], [second-option], [third-option]", e.getMessage()); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + void testCheckOneOfArgsMultiplePresent() { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + OptionParser parser = mock(OptionParser.class); + OptionSet options = mock(OptionSet.class); + OptionSpec opt1 = createMockOptionSpec("--first-option"); + OptionSpec opt2 = createMockOptionSpec("--second-option"); + OptionSpec opt3 = createMockOptionSpec("--third-option"); + + when(options.has(opt1)).thenReturn(true); + when(options.has(opt2)).thenReturn(true); + when(options.has(opt3)).thenReturn(false); + + try { + IllegalArgumentException e = assertThrows(IllegalArgumentException.class, + () -> CommandLineUtils.checkOneOfArgs(parser, options, opt1, opt2, opt3)); + assertEquals("Exactly one of the following arguments is required: " + + "[first-option], [second-option], [third-option]", e.getMessage()); + } finally { + Exit.resetExitProcedure(); + } + } } diff --git a/server/src/main/java/org/apache/kafka/server/share/acknowledge/ShareAcknowledgementBatch.java b/server/src/main/java/org/apache/kafka/server/share/acknowledge/ShareAcknowledgementBatch.java index bc29f37c62f..96efa5eb29e 100644 --- a/server/src/main/java/org/apache/kafka/server/share/acknowledge/ShareAcknowledgementBatch.java +++ b/server/src/main/java/org/apache/kafka/server/share/acknowledge/ShareAcknowledgementBatch.java @@ -24,6 +24,9 @@ import java.util.List; * The class abstracts the acknowledgement request for SharePartition class constructed * from {@link org.apache.kafka.common.message.ShareFetchRequestData.AcknowledgementBatch} and * {@link org.apache.kafka.common.message.ShareAcknowledgeRequestData.AcknowledgementBatch} classes. + *

    + * Acknowledge types are represented as a list of bytes, where each byte corresponds to an acknowledge + * type defined in {@link org.apache.kafka.clients.consumer.AcknowledgeType}. */ public record ShareAcknowledgementBatch( long firstOffset, diff --git a/server/src/test/java/org/apache/kafka/server/EligibleLeaderReplicasIntegrationTest.java b/server/src/test/java/org/apache/kafka/server/EligibleLeaderReplicasIntegrationTest.java new file mode 100644 index 00000000000..61863688f2e --- /dev/null +++ b/server/src/test/java/org/apache/kafka/server/EligibleLeaderReplicasIntegrationTest.java @@ -0,0 +1,384 @@ +/* + * 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; + +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.FeatureUpdate; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.OffsetSpec; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.admin.UpdateFeaturesOptions; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.ClusterTestDefaults; +import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.server.common.EligibleLeaderReplicasVersion; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.config.ReplicationConfigs; +import org.apache.kafka.server.config.ServerConfigs; +import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler; +import org.apache.kafka.test.TestUtils; + +import java.io.File; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +@ClusterTestDefaults( + brokers = 5, + serverProperties = { + @ClusterConfigProperty(key = ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, value = "4") + } +) +public class EligibleLeaderReplicasIntegrationTest { + private final ClusterInstance clusterInstance; + + EligibleLeaderReplicasIntegrationTest(ClusterInstance clusterInstance) { + this.clusterInstance = clusterInstance; + } + + @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_4_0_IV1) + public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr() throws ExecutionException, InterruptedException { + try (var admin = clusterInstance.admin(); + var producer = clusterInstance.producer(Map.of( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName(), + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName(), + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers(), + ProducerConfig.ACKS_CONFIG, "1")); + var consumer = clusterInstance.consumer(Map.of( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers(), + ConsumerConfig.GROUP_ID_CONFIG, "test", + ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, "10", + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName(), + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()))) { + String testTopicName = String.format("%s-%s", "testHighWatermarkShouldNotAdvanceIfUnderMinIsr", "ELR-test"); + admin.updateFeatures( + Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, + new FeatureUpdate(EligibleLeaderReplicasVersion.ELRV_1.featureLevel(), FeatureUpdate.UpgradeType.UPGRADE)), + new UpdateFeaturesOptions()).all().get(); + + admin.createTopics(List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); + clusterInstance.waitTopicCreation(testTopicName, 1); + + ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); + Collection ops = new ArrayList<>(); + ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET)); + Map> configOps = Map.of(configResource, ops); + // alter configs on target cluster + admin.incrementalAlterConfigs(configOps).all().get(); + + TopicDescription testTopicDescription = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName); + TopicPartitionInfo topicPartitionInfo = testTopicDescription.partitions().get(0); + List initialReplicas = topicPartitionInfo.replicas(); + assertEquals(4, topicPartitionInfo.isr().size()); + assertEquals(0, topicPartitionInfo.elr().size()); + assertEquals(0, topicPartitionInfo.lastKnownElr().size()); + + consumer.subscribe(Set.of(testTopicName)); + producer.send(new ProducerRecord<>(testTopicName, "0", "0")).get(); + waitUntilOneMessageIsConsumed(consumer); + + clusterInstance.shutdownBroker(initialReplicas.get(0).id()); + clusterInstance.shutdownBroker(initialReplicas.get(1).id()); + + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 2 && elrSize == 1, admin, testTopicName); + + TopicPartition partition = new TopicPartition(testTopicName, 0); + long leoBeforeSend = admin.listOffsets(Map.of(partition, OffsetSpec.latest())).partitionResult(partition).get().offset(); + // Now the partition is under min ISR. HWM should not advance. + producer.send(new ProducerRecord<>(testTopicName, "1", "1")).get(); + long leoAfterSend = admin.listOffsets(Map.of(partition, OffsetSpec.latest())).partitionResult(partition).get().offset(); + assertEquals(leoBeforeSend, leoAfterSend); + + // Restore the min ISR and the previous log should be visible. + clusterInstance.startBroker(initialReplicas.get(1).id()); + clusterInstance.startBroker(initialReplicas.get(0).id()); + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 4 && elrSize == 0, admin, testTopicName); + + waitUntilOneMessageIsConsumed(consumer); + } + } + + void waitUntilOneMessageIsConsumed(Consumer consumer) throws InterruptedException { + TestUtils.waitForCondition( + () -> { + try { + return consumer.poll(Duration.ofMillis(100L)).count() >= 1; + } catch (Exception e) { + return false; + } + }, + DEFAULT_MAX_WAIT_MS, + () -> "fail to consume messages" + ); + } + + @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_4_0_IV1) + public void testElrMemberCanBeElected() throws ExecutionException, InterruptedException { + try (var admin = clusterInstance.admin()) { + String testTopicName = String.format("%s-%s", "testElrMemberCanBeElected", "ELR-test"); + + admin.updateFeatures( + Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, + new FeatureUpdate(EligibleLeaderReplicasVersion.ELRV_1.featureLevel(), FeatureUpdate.UpgradeType.UPGRADE)), + new UpdateFeaturesOptions()).all().get(); + admin.createTopics(List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); + clusterInstance.waitTopicCreation(testTopicName, 1); + + ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); + Collection ops = new ArrayList<>(); + ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET)); + Map> configOps = Map.of(configResource, ops); + // alter configs on target cluster + admin.incrementalAlterConfigs(configOps).all().get(); + + TopicDescription testTopicDescription = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName); + TopicPartitionInfo topicPartitionInfo = testTopicDescription.partitions().get(0); + List initialReplicas = topicPartitionInfo.replicas(); + assertEquals(4, topicPartitionInfo.isr().size()); + assertEquals(0, topicPartitionInfo.elr().size()); + assertEquals(0, topicPartitionInfo.lastKnownElr().size()); + + clusterInstance.shutdownBroker(initialReplicas.get(0).id()); + clusterInstance.shutdownBroker(initialReplicas.get(1).id()); + clusterInstance.shutdownBroker(initialReplicas.get(2).id()); + + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 1 && elrSize == 2, admin, testTopicName); + + clusterInstance.shutdownBroker(initialReplicas.get(3).id()); + + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 0 && elrSize == 3, admin, testTopicName); + + topicPartitionInfo = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions().get(0); + assertEquals(1, topicPartitionInfo.lastKnownElr().size(), topicPartitionInfo.toString()); + int expectLastKnownLeader = initialReplicas.get(3).id(); + assertEquals(expectLastKnownLeader, topicPartitionInfo.lastKnownElr().get(0).id(), topicPartitionInfo.toString()); + + // At this point, all the replicas are failed and the last know leader is No.3 and 3 members in the ELR. + // Restart one broker of the ELR and it should be the leader. + + int expectLeader = topicPartitionInfo.elr().stream() + .filter(node -> node.id() != expectLastKnownLeader).toList().get(0).id(); + + clusterInstance.startBroker(expectLeader); + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 1 && elrSize == 2, admin, testTopicName); + + topicPartitionInfo = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions().get(0); + assertEquals(0, topicPartitionInfo.lastKnownElr().size(), topicPartitionInfo.toString()); + assertEquals(expectLeader, topicPartitionInfo.leader().id(), topicPartitionInfo.toString()); + + // Start another 2 brokers and the ELR fields should be cleaned. + topicPartitionInfo.replicas().stream().filter(node -> node.id() != expectLeader).limit(2) + .forEach(node -> clusterInstance.startBroker(node.id())); + + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 3 && elrSize == 0, admin, testTopicName); + + topicPartitionInfo = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions().get(0); + assertEquals(0, topicPartitionInfo.lastKnownElr().size(), topicPartitionInfo.toString()); + assertEquals(expectLeader, topicPartitionInfo.leader().id(), topicPartitionInfo.toString()); + } + } + + @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_4_0_IV1) + public void testElrMemberShouldBeKickOutWhenUncleanShutdown() throws ExecutionException, InterruptedException { + try (var admin = clusterInstance.admin()) { + String testTopicName = String.format("%s-%s", "testElrMemberShouldBeKickOutWhenUncleanShutdown", "ELR-test"); + + admin.updateFeatures( + Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, + new FeatureUpdate(EligibleLeaderReplicasVersion.ELRV_1.featureLevel(), FeatureUpdate.UpgradeType.UPGRADE)), + new UpdateFeaturesOptions()).all().get(); + admin.createTopics(List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); + clusterInstance.waitTopicCreation(testTopicName, 1); + + ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); + Collection ops = new ArrayList<>(); + ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET)); + Map> configOps = Map.of(configResource, ops); + // alter configs on target cluster + admin.incrementalAlterConfigs(configOps).all().get(); + + TopicDescription testTopicDescription = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName); + TopicPartitionInfo topicPartitionInfo = testTopicDescription.partitions().get(0); + List initialReplicas = topicPartitionInfo.replicas(); + assertEquals(4, topicPartitionInfo.isr().size()); + assertEquals(0, topicPartitionInfo.elr().size()); + assertEquals(0, topicPartitionInfo.lastKnownElr().size()); + + clusterInstance.shutdownBroker(initialReplicas.get(0).id()); + clusterInstance.shutdownBroker(initialReplicas.get(1).id()); + clusterInstance.shutdownBroker(initialReplicas.get(2).id()); + clusterInstance.shutdownBroker(initialReplicas.get(3).id()); + + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 0 && elrSize == 3, admin, testTopicName); + topicPartitionInfo = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions().get(0); + + int brokerToBeUncleanShutdown = topicPartitionInfo.elr().get(0).id(); + var broker = clusterInstance.brokers().values().stream().filter(b -> b.config().brokerId() == brokerToBeUncleanShutdown) + .findFirst().get(); + List dirs = new ArrayList<>(); + broker.logManager().liveLogDirs().foreach(dirs::add); + assertEquals(1, dirs.size()); + CleanShutdownFileHandler handler = new CleanShutdownFileHandler(dirs.get(0).toString()); + assertTrue(handler.exists()); + assertDoesNotThrow(handler::delete); + + // After remove the clean shutdown file, the broker should report unclean shutdown during restart. + clusterInstance.startBroker(brokerToBeUncleanShutdown); + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 0 && elrSize == 2, admin, testTopicName); + topicPartitionInfo = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions().get(0); + assertNull(topicPartitionInfo.leader()); + assertEquals(1, topicPartitionInfo.lastKnownElr().size()); + } + } + + /* + This test is only valid for KIP-966 part 1. When the unclean recovery is implemented, it should be removed. + */ + @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_4_0_IV1) + public void testLastKnownLeaderShouldBeElectedIfEmptyElr() throws ExecutionException, InterruptedException { + try (var admin = clusterInstance.admin()) { + String testTopicName = String.format("%s-%s", "testLastKnownLeaderShouldBeElectedIfEmptyElr", "ELR-test"); + + admin.updateFeatures( + Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, + new FeatureUpdate(EligibleLeaderReplicasVersion.ELRV_1.featureLevel(), FeatureUpdate.UpgradeType.UPGRADE)), + new UpdateFeaturesOptions()).all().get(); + admin.createTopics(List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); + clusterInstance.waitTopicCreation(testTopicName, 1); + + ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); + Collection ops = new ArrayList<>(); + ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET)); + Map> configOps = Map.of(configResource, ops); + // alter configs on target cluster + admin.incrementalAlterConfigs(configOps).all().get(); + + + TopicDescription testTopicDescription = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName); + TopicPartitionInfo topicPartitionInfo = testTopicDescription.partitions().get(0); + List initialReplicas = topicPartitionInfo.replicas(); + assertEquals(4, topicPartitionInfo.isr().size()); + assertEquals(0, topicPartitionInfo.elr().size()); + assertEquals(0, topicPartitionInfo.lastKnownElr().size()); + + clusterInstance.shutdownBroker(initialReplicas.get(0).id()); + clusterInstance.shutdownBroker(initialReplicas.get(1).id()); + clusterInstance.shutdownBroker(initialReplicas.get(2).id()); + clusterInstance.shutdownBroker(initialReplicas.get(3).id()); + + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 0 && elrSize == 3, admin, testTopicName); + topicPartitionInfo = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions().get(0); + int lastKnownLeader = topicPartitionInfo.lastKnownElr().get(0).id(); + + Set initialReplicaSet = initialReplicas.stream().map(node -> node.id()).collect(Collectors.toSet()); + clusterInstance.brokers().forEach((id, broker) -> { + if (initialReplicaSet.contains(id)) { + List dirs = new ArrayList<>(); + broker.logManager().liveLogDirs().foreach(dirs::add); + assertEquals(1, dirs.size()); + CleanShutdownFileHandler handler = new CleanShutdownFileHandler(dirs.get(0).toString()); + assertDoesNotThrow(handler::delete); + } + }); + + // After remove the clean shutdown file, the broker should report unclean shutdown during restart. + topicPartitionInfo.replicas().forEach(replica -> { + if (replica.id() != lastKnownLeader) clusterInstance.startBroker(replica.id()); + }); + waitForIsrAndElr((isrSize, elrSize) -> isrSize == 0 && elrSize == 1, admin, testTopicName); + topicPartitionInfo = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions().get(0); + assertNull(topicPartitionInfo.leader()); + assertEquals(1, topicPartitionInfo.lastKnownElr().size()); + + // Now if the last known leader goes through unclean shutdown, it will still be elected. + clusterInstance.startBroker(lastKnownLeader); + waitForIsrAndElr((isrSize, elrSize) -> isrSize > 0 && elrSize == 0, admin, testTopicName); + TestUtils.waitForCondition( + () -> { + try { + TopicPartitionInfo partition = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName).partitions().get(0); + if (partition.leader() == null) return false; + return partition.lastKnownElr().isEmpty() && partition.elr().isEmpty() && partition.leader().id() == lastKnownLeader; + } catch (Exception e) { + return false; + } + }, + DEFAULT_MAX_WAIT_MS, + () -> String.format("Partition metadata for %s is not correct", testTopicName) + ); + } + } + + void waitForIsrAndElr(BiFunction isIsrAndElrSizeSatisfied, Admin admin, String testTopicName) throws InterruptedException { + TestUtils.waitForCondition( + () -> { + try { + TopicDescription topicDescription = admin.describeTopics(List.of(testTopicName)) + .allTopicNames().get().get(testTopicName); + TopicPartitionInfo partition = topicDescription.partitions().get(0); + return isIsrAndElrSizeSatisfied.apply(partition.isr().size(), partition.elr().size()); + } catch (Exception e) { + return false; + } + }, + DEFAULT_MAX_WAIT_MS, + () -> String.format("Partition metadata for %s is not propagated", testTopicName) + ); + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetrics.java b/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetrics.java index 4e365e1a13d..f813f31adea 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetrics.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetrics.java @@ -24,7 +24,7 @@ import org.apache.kafka.server.quota.SensorAccess; import java.util.concurrent.locks.ReentrantReadWriteLock; -public class RLMQuotaMetrics { +public class RLMQuotaMetrics implements AutoCloseable { private final SensorAccess sensorAccess; private final Metrics metrics; @@ -51,4 +51,9 @@ public class RLMQuotaMetrics { String.format(descriptionFormat, "maximum")), new Max()); }); } + + @Override + public void close() { + this.metrics.removeSensor(name); + } } diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index a9b2c67ba79..f6480b8668c 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -314,6 +314,8 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader { metricsGroup.removeMetric(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC); metricsGroup.removeMetric(REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC); remoteStorageReaderThreadPool.removeMetrics(); + Utils.closeQuietly(fetchQuotaMetrics, "fetchQuotaMetrics"); + Utils.closeQuietly(copyQuotaMetrics, "copyQuotaMetrics"); } // Visible for testing @@ -2044,17 +2046,18 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader { followerThreadPool.close(); try { shutdownAndAwaitTermination(remoteStorageReaderThreadPool, "RemoteStorageReaderThreadPool", 10, TimeUnit.SECONDS); + + leaderCopyRLMTasks.clear(); + leaderExpirationRLMTasks.clear(); + followerRLMTasks.clear(); + + Utils.closeQuietly(indexCache, "RemoteIndexCache"); + Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin"); + Utils.closeQuietly(remoteStorageManagerPlugin, "remoteStorageManagerPlugin"); + closed = true; } finally { removeMetrics(); } - leaderCopyRLMTasks.clear(); - leaderExpirationRLMTasks.clear(); - followerRLMTasks.clear(); - - Utils.closeQuietly(indexCache, "RemoteIndexCache"); - Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin"); - Utils.closeQuietly(remoteStorageManagerPlugin, "remoteStorageManagerPlugin"); - closed = true; } } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index ca32e4f086a..769f59d56dc 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1667,6 +1667,8 @@ public class UnifiedLog implements AutoCloseable { } else { return new OffsetResultHolder(new FileRecords.TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1))); } + } else if (targetTimestamp == ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) { + return fetchEarliestPendingUploadOffset(remoteOffsetReader); } else if (targetTimestamp == ListOffsetsRequest.MAX_TIMESTAMP) { // Cache to avoid race conditions. List segments = logSegments(); @@ -1709,6 +1711,31 @@ public class UnifiedLog implements AutoCloseable { }); } + private OffsetResultHolder fetchEarliestPendingUploadOffset(Optional remoteOffsetReader) { + if (remoteLogEnabled()) { + long curHighestRemoteOffset = highestOffsetInRemoteStorage(); + + if (curHighestRemoteOffset == -1L) { + if (localLogStartOffset() == logStartOffset()) { + // No segments have been uploaded yet + return fetchOffsetByTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP, remoteOffsetReader); + } else { + // Leader currently does not know about the already uploaded segments + return new OffsetResultHolder(Optional.of(new FileRecords.TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1)))); + } + } else { + long earliestPendingUploadOffset = Math.max(curHighestRemoteOffset + 1, logStartOffset()); + OptionalInt epochForOffset = leaderEpochCache.epochForOffset(earliestPendingUploadOffset); + Optional epochResult = epochForOffset.isPresent() + ? Optional.of(epochForOffset.getAsInt()) + : Optional.empty(); + return new OffsetResultHolder(new FileRecords.TimestampAndOffset(RecordBatch.NO_TIMESTAMP, earliestPendingUploadOffset, epochResult)); + } + } else { + return new OffsetResultHolder(new FileRecords.TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1))); + } + } + /** * Checks if the log is empty. * @return Returns True when the log is empty. Otherwise, false. diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetricsTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetricsTest.java index bf2cddd0f31..3c90c1bbc86 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetricsTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/quota/RLMQuotaMetricsTest.java @@ -28,6 +28,8 @@ import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; public class RLMQuotaMetricsTest { private final MockTime time = new MockTime(); @@ -49,4 +51,22 @@ public class RLMQuotaMetricsTest { Sensor newSensor = rlmQuotaMetrics.sensor(); assertNotEquals(sensor, newSensor); } + + @Test + public void testClose() { + RLMQuotaMetrics quotaMetrics = new RLMQuotaMetrics(metrics, "metric", "group", "format", 5); + + // Register the sensor + quotaMetrics.sensor(); + var avg = metrics.metricName("metric" + "-avg", "group", String.format("format", "average")); + + // Verify that metrics are created + assertNotNull(metrics.metric(avg)); + + // Close the quotaMetrics instance + quotaMetrics.close(); + + // After closing, the metrics should be removed + assertNull(metrics.metric(avg)); + } } diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index 92231da7e6e..2012cfdf092 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -16,9 +16,14 @@ */ package org.apache.kafka.streams.integration; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.ListOffsetsResult; +import org.apache.kafka.clients.admin.OffsetSpec; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; @@ -32,6 +37,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.GroupProtocol; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KafkaStreams.State; import org.apache.kafka.streams.KeyValue; @@ -68,10 +74,10 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,6 +88,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -109,6 +116,7 @@ import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; @Timeout(600) @Tag("integration") @@ -121,13 +129,20 @@ public class RestoreIntegrationTest { public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); + private static Admin admin; + @BeforeAll public static void startCluster() throws IOException { CLUSTER.start(); + + final Properties adminConfig = new Properties(); + adminConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + admin = Admin.create(adminConfig); } @AfterAll public static void closeCluster() { + Utils.closeQuietly(admin, "admin"); CLUSTER.stop(); } @@ -160,7 +175,7 @@ public class RestoreIntegrationTest { streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L); - streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsConfiguration.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest"); streamsConfiguration.putAll(extraProperties); streamsConfigurations.add(streamsConfiguration); @@ -178,11 +193,12 @@ public class RestoreIntegrationTest { streamsConfigurations.clear(); } - @Test - public void shouldRestoreNullRecord() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void shouldRestoreNullRecord(final boolean useNewProtocol) throws Exception { final StreamsBuilder builder = new StreamsBuilder(); - final String applicationId = "restoration-test-app"; + final String applicationId = appId; final String stateStoreName = "stateStore"; final String inputTopic = "input"; final String outputTopic = "output"; @@ -196,6 +212,10 @@ public class RestoreIntegrationTest { Serdes.BytesSerde.class.getName(), props); + if (useNewProtocol) { + streamsConfiguration.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } + CLUSTER.createTopics(inputTopic); CLUSTER.createTopics(outputTopic); @@ -244,18 +264,28 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void shouldRestoreStateFromSourceTopicForReadOnlyStore(final boolean stateUpdaterEnabled) throws Exception { + @CsvSource({ + "true,true", + "true,false", + "false,true", + "false,false" + }) + public void shouldRestoreStateFromSourceTopicForReadOnlyStore(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws Exception { final AtomicInteger numReceived = new AtomicInteger(0); final Topology topology = new Topology(); final Properties props = props(stateUpdaterEnabled); + if (useNewProtocol) { + props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } // restoring from 1000 to 4000 (committed), and then process from 4000 to 5000 on each of the two partitions final int offsetLimitDelta = 1000; final int offsetCheckpointed = 1000; createStateForRestoration(inputStream, 0); - setCommittedOffset(inputStream, offsetLimitDelta); + if (!useNewProtocol) { + setCommittedOffset(inputStream, offsetLimitDelta, useNewProtocol); + } final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true, false); // note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1 @@ -264,7 +294,6 @@ public class RestoreIntegrationTest { new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 1)), ".checkpoint")) .write(Collections.singletonMap(new TopicPartition(inputStream, 1), (long) offsetCheckpointed - 1)); - final CountDownLatch startupLatch = new CountDownLatch(1); final CountDownLatch shutdownLatch = new CountDownLatch(1); topology.addReadOnlyStateStore( @@ -282,17 +311,23 @@ public class RestoreIntegrationTest { ); kafkaStreams = new KafkaStreams(topology, props); - kafkaStreams.setStateListener((newState, oldState) -> { - if (newState == KafkaStreams.State.RUNNING && oldState == KafkaStreams.State.REBALANCING) { - startupLatch.countDown(); - } - }); final AtomicLong restored = new AtomicLong(0); kafkaStreams.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored)); - kafkaStreams.start(); + startApplicationAndWaitUntilRunning(kafkaStreams); + + if (useNewProtocol) { + // For new protocol, we need to stop the streams instance before altering offsets + kafkaStreams.close(Duration.ofSeconds(60)); + setCommittedOffset(inputStream, offsetLimitDelta, useNewProtocol); + + // Restart the streams instance with a new startup latch + + kafkaStreams = new KafkaStreams(topology, props); + kafkaStreams.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored)); + startApplicationAndWaitUntilRunning(kafkaStreams); + } - assertTrue(startupLatch.await(30, TimeUnit.SECONDS)); assertThat(restored.get(), equalTo((long) numberOfKeys - offsetLimitDelta * 2 - offsetCheckpointed * 2)); assertTrue(shutdownLatch.await(30, TimeUnit.SECONDS)); @@ -300,19 +335,29 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void shouldRestoreStateFromSourceTopicForGlobalTable(final boolean stateUpdaterEnabled) throws Exception { + @CsvSource({ + "true,true", + "true,false", + "false,true", + "false,false" + }) + public void shouldRestoreStateFromSourceTopicForGlobalTable(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws Exception { final AtomicInteger numReceived = new AtomicInteger(0); final StreamsBuilder builder = new StreamsBuilder(); final Properties props = props(stateUpdaterEnabled); props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); + if (useNewProtocol) { + props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } // restoring from 1000 to 4000 (committed), and then process from 4000 to 5000 on each of the two partitions final int offsetLimitDelta = 1000; final int offsetCheckpointed = 1000; createStateForRestoration(inputStream, 0); - setCommittedOffset(inputStream, offsetLimitDelta); + if (!useNewProtocol) { + setCommittedOffset(inputStream, offsetLimitDelta, useNewProtocol); + } final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true, false); // note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1 @@ -344,6 +389,20 @@ public class RestoreIntegrationTest { kafkaStreams.start(); assertTrue(startupLatch.await(30, TimeUnit.SECONDS)); + + if (useNewProtocol) { + // For new protocol, we need to stop the streams instance before altering offsets + kafkaStreams.close(); + setCommittedOffset(inputStream, offsetLimitDelta, useNewProtocol); + + // Restart the streams instance with a new startup latch + kafkaStreams = new KafkaStreams(builder.build(props), props); + + kafkaStreams.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored)); + startApplicationAndWaitUntilRunning(kafkaStreams); + + } + assertThat(restored.get(), equalTo((long) numberOfKeys - offsetLimitDelta * 2 - offsetCheckpointed * 2)); assertTrue(shutdownLatch.await(30, TimeUnit.SECONDS)); @@ -351,8 +410,13 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void shouldRestoreStateFromChangelogTopic(final boolean stateUpdaterEnabled) throws Exception { + @CsvSource({ + "true,true", + "true,false", + "false,true", + "false,false" + }) + public void shouldRestoreStateFromChangelogTopic(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws Exception { final String changelog = appId + "-store-changelog"; CLUSTER.createTopic(changelog, 2, 1); @@ -361,6 +425,10 @@ public class RestoreIntegrationTest { final Properties props = props(stateUpdaterEnabled); + if (useNewProtocol) { + props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } + // restoring from 1000 to 5000, and then process from 5000 to 10000 on each of the two partitions final int offsetCheckpointed = 1000; createStateForRestoration(changelog, 0); @@ -403,8 +471,13 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void shouldSuccessfullyStartWhenLoggingDisabled(final boolean stateUpdaterEnabled) throws InterruptedException { + @CsvSource({ + "true,true", + "true,false", + "false,true", + "false,false" + }) + public void shouldSuccessfullyStartWhenLoggingDisabled(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws InterruptedException { final StreamsBuilder builder = new StreamsBuilder(); final KStream stream = builder.stream(inputStream); @@ -414,23 +487,26 @@ public class RestoreIntegrationTest { Integer::sum, Materialized.>as("reduce-store").withLoggingDisabled() ); - - final CountDownLatch startupLatch = new CountDownLatch(1); - kafkaStreams = new KafkaStreams(builder.build(), props(stateUpdaterEnabled)); - kafkaStreams.setStateListener((newState, oldState) -> { - if (newState == KafkaStreams.State.RUNNING && oldState == KafkaStreams.State.REBALANCING) { - startupLatch.countDown(); - } - }); - - kafkaStreams.start(); - - assertTrue(startupLatch.await(30, TimeUnit.SECONDS)); + final Properties props = props(stateUpdaterEnabled); + if (useNewProtocol) { + props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } + kafkaStreams = new KafkaStreams(builder.build(), props); + try { + startApplicationAndWaitUntilRunning(kafkaStreams); + } catch (final Exception e) { + fail("Failed to start KafkaStreams", e); + } } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void shouldProcessDataFromStoresWithLoggingDisabled(final boolean stateUpdaterEnabled) throws InterruptedException { + @CsvSource({ + "true,true", + "true,false", + "false,true", + "false,false" + }) + public void shouldProcessDataFromStoresWithLoggingDisabled(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws InterruptedException { IntegrationTestUtils.produceKeyValuesSynchronously(inputStream, asList(KeyValue.pair(1, 1), KeyValue.pair(2, 2), @@ -458,7 +534,12 @@ public class RestoreIntegrationTest { final Topology topology = streamsBuilder.build(); - kafkaStreams = new KafkaStreams(topology, props(stateUpdaterEnabled)); + final Properties props = props(stateUpdaterEnabled); + + if (useNewProtocol) { + props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } + kafkaStreams = new KafkaStreams(topology, props); final CountDownLatch latch = new CountDownLatch(1); kafkaStreams.setStateListener((newState, oldState) -> { @@ -474,8 +555,13 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void shouldRecycleStateFromStandbyTaskPromotedToActiveTaskAndNotRestore(final boolean stateUpdaterEnabled) throws Exception { + @CsvSource({ + "true,true", + "true,false", + "false,true", + "false,false" + }) + public void shouldRecycleStateFromStandbyTaskPromotedToActiveTaskAndNotRestore(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws Exception { final StreamsBuilder builder = new StreamsBuilder(); builder.table( inputStream, @@ -483,15 +569,25 @@ public class RestoreIntegrationTest { ); createStateForRestoration(inputStream, 0); + if (useNewProtocol) { + CLUSTER.setStandbyReplicas(appId, 1); + } + final Properties props1 = props(stateUpdaterEnabled); props1.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); props1.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId + "-1").getPath()); + if (useNewProtocol) { + props1.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } purgeLocalStreamsState(props1); final KafkaStreams streams1 = new KafkaStreams(builder.build(), props1); final Properties props2 = props(stateUpdaterEnabled); props2.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); props2.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId + "-2").getPath()); + if (useNewProtocol) { + props2.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } purgeLocalStreamsState(props2); final KafkaStreams streams2 = new KafkaStreams(builder.build(), props2); @@ -513,19 +609,19 @@ public class RestoreIntegrationTest { waitForStandbyCompletion(streams1, 1, 30 * 1000L); waitForStandbyCompletion(streams2, 1, 30 * 1000L); } catch (final Exception e) { - streams1.close(); - streams2.close(); + streams1.close(Duration.ofSeconds(60)); + streams2.close(Duration.ofSeconds(60)); + throw e; } // Sometimes the store happens to have already been closed sometime during startup, so just keep track // of where it started and make sure it doesn't happen more times from there final int initialStoreCloseCount = CloseCountingInMemoryStore.numStoresClosed(); final long initialNunRestoredCount = restoreListener.totalNumRestored(); - transitionedStates1.clear(); transitionedStates2.clear(); try { - streams2.close(); + streams2.close(Duration.ofSeconds(60)); waitForTransitionTo(transitionedStates2, State.NOT_RUNNING, Duration.ofSeconds(60)); waitForTransitionTo(transitionedStates1, State.REBALANCING, Duration.ofSeconds(60)); waitForTransitionTo(transitionedStates1, State.RUNNING, Duration.ofSeconds(60)); @@ -535,18 +631,20 @@ public class RestoreIntegrationTest { assertThat(restoreListener.totalNumRestored(), CoreMatchers.equalTo(initialNunRestoredCount)); - // After stopping instance 2 and letting instance 1 take over its tasks, we should have closed just two stores - // total: the active and standby tasks on instance 2 - assertThat(CloseCountingInMemoryStore.numStoresClosed(), equalTo(initialStoreCloseCount + 2)); + // After stopping instance 2 and letting instance 1 take over its tasks, we should have closed the stores on instance 2. + // Under the new group protocol, an extra store close can occur during rebalance; account for that here. + final int expectedAfterStreams2Close = initialStoreCloseCount + (useNewProtocol ? 3 : 2); + assertThat(CloseCountingInMemoryStore.numStoresClosed(), equalTo(expectedAfterStreams2Close)); } finally { - streams1.close(); + streams1.close(Duration.ofSeconds(60)); } waitForTransitionTo(transitionedStates1, State.NOT_RUNNING, Duration.ofSeconds(60)); assertThat(CloseCountingInMemoryStore.numStoresClosed(), CoreMatchers.equalTo(initialStoreCloseCount + 4)); } - @Test - public void shouldInvokeUserDefinedGlobalStateRestoreListener() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void shouldInvokeUserDefinedGlobalStateRestoreListener(final boolean useNewProtocol) throws Exception { final String inputTopic = "inputTopic"; final String outputTopic = "outputTopic"; CLUSTER.createTopic(inputTopic, 5, 1); @@ -575,7 +673,7 @@ public class RestoreIntegrationTest { sendEvents(inputTopic, sampleData); - kafkaStreams = startKafkaStreams(builder, null, kafkaStreams1Configuration); + kafkaStreams = startKafkaStreams(builder, null, kafkaStreams1Configuration, useNewProtocol); validateReceivedMessages(sampleData, outputTopic); @@ -584,7 +682,7 @@ public class RestoreIntegrationTest { IntegrationTestUtils.purgeLocalStreamsState(streamsConfigurations); final TestStateRestoreListener kafkaStreams1StateRestoreListener = new TestStateRestoreListener("ks1", RESTORATION_DELAY); - kafkaStreams = startKafkaStreams(builder, kafkaStreams1StateRestoreListener, kafkaStreams1Configuration); + kafkaStreams = startKafkaStreams(builder, kafkaStreams1StateRestoreListener, kafkaStreams1Configuration, useNewProtocol); // Ensure all the restoring tasks are in active state before starting the new instance. // Otherwise, the tasks which assigned to first kafka streams won't encounter "restoring suspend" after being reassigned to the second instance. @@ -600,7 +698,8 @@ public class RestoreIntegrationTest { try (final KafkaStreams kafkaStreams2 = startKafkaStreams(builder, kafkaStreams2StateRestoreListener, - kafkaStreams2Configuration)) { + kafkaStreams2Configuration, + useNewProtocol)) { waitForCondition(() -> State.RUNNING == kafkaStreams2.state(), 90_000, @@ -639,8 +738,12 @@ public class RestoreIntegrationTest { private KafkaStreams startKafkaStreams(final StreamsBuilder streamsBuilder, final StateRestoreListener stateRestoreListener, - final Map extraConfiguration) { + final Map extraConfiguration, + final boolean useNewProtocol) { final Properties streamsConfiguration = props(mkObjectProperties(extraConfiguration)); + if (useNewProtocol) { + streamsConfiguration.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } final KafkaStreams kafkaStreams = new KafkaStreams(streamsBuilder.build(), streamsConfiguration); kafkaStreams.setGlobalStateRestoreListener(stateRestoreListener); @@ -814,29 +917,54 @@ public class RestoreIntegrationTest { } } - private void setCommittedOffset(final String topic, final int limitDelta) { - final Properties consumerConfig = new Properties(); - consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, appId); - consumerConfig.put(ConsumerConfig.CLIENT_ID_CONFIG, "commit-consumer"); - consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); - consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); + private void setCommittedOffset(final String topic, final int limitDelta, final boolean useNewProtocol) { + if (!useNewProtocol) { + final Properties consumerConfig = new Properties(); + consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, appId); + consumerConfig.put(ConsumerConfig.CLIENT_ID_CONFIG, "commit-consumer"); + consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); + consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); - final Consumer consumer = new KafkaConsumer<>(consumerConfig); - final List partitions = asList( - new TopicPartition(topic, 0), - new TopicPartition(topic, 1)); + try (final Consumer consumer = new KafkaConsumer<>(consumerConfig)) { + final List partitions = asList( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1)); - consumer.assign(partitions); - consumer.seekToEnd(partitions); + consumer.assign(partitions); + consumer.seekToEnd(partitions); - for (final TopicPartition partition : partitions) { - final long position = consumer.position(partition); - consumer.seek(partition, position - limitDelta); + for (final TopicPartition partition : partitions) { + final long position = consumer.position(partition); + consumer.seek(partition, position - limitDelta); + } + + consumer.commitSync(); + } + } else { + try { + final List partitions = asList( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1)); + + final Map offsetSpecs = partitions.stream() + .collect(Collectors.toMap(tp -> tp, tp -> OffsetSpec.latest())); + + final Map endOffsets = + admin.listOffsets(offsetSpecs).all().get(); + + final Map offsetsToCommit = new HashMap<>(); + for (final TopicPartition partition : partitions) { + final long endOffset = endOffsets.get(partition).offset(); + final long targetOffset = Math.max(0, endOffset - limitDelta); + offsetsToCommit.put(partition, new OffsetAndMetadata(targetOffset)); + } + + admin.alterStreamsGroupOffsets(appId, offsetsToCommit).all().get(); + } catch (final Exception e) { + fail("Failed to set committed offsets", e); + } } - - consumer.commitSync(); - consumer.close(); } private void waitForTransitionTo(final Set observed, final KafkaStreams.State state, final Duration timeout) throws Exception { diff --git a/tests/kafkatest/services/performance/producer_performance.py b/tests/kafkatest/services/performance/producer_performance.py index acfe4790d73..4a93541c7cb 100644 --- a/tests/kafkatest/services/performance/producer_performance.py +++ b/tests/kafkatest/services/performance/producer_performance.py @@ -91,7 +91,7 @@ class ProducerPerformanceService(HttpMetricsCollector, PerformanceService): cmd += " export KAFKA_LOG4J_OPTS=\"%s%s\"; " % (get_log4j_config_param(node), get_log4j_config_for_tools(node)) cmd += "KAFKA_OPTS=%(kafka_opts)s KAFKA_HEAP_OPTS=\"-XX:+HeapDumpOnOutOfMemoryError\" %(kafka_run_class)s org.apache.kafka.tools.ProducerPerformance " \ - "--topic %(topic)s --num-records %(num_records)d --record-size %(record_size)d --throughput %(throughput)d --producer-props bootstrap.servers=%(bootstrap_servers)s client.id=%(client_id)s %(metrics_props)s" % args + "--topic %(topic)s --num-records %(num_records)d --record-size %(record_size)d --throughput %(throughput)d --command-property bootstrap.servers=%(bootstrap_servers)s client.id=%(client_id)s %(metrics_props)s" % args self.security_config.setup_node(node) if self.security_config.security_protocol != SecurityConfig.PLAINTEXT: diff --git a/tests/kafkatest/services/verifiable_client.py b/tests/kafkatest/services/verifiable_client.py index 16617d621aa..b6c19267328 100644 --- a/tests/kafkatest/services/verifiable_client.py +++ b/tests/kafkatest/services/verifiable_client.py @@ -74,7 +74,8 @@ Command line arguments: * `--enable-autocommit` * `--max-messages ` * `--assignment-strategy ` - * `--consumer.config ` - consumer config properties (typically empty) + * `--consumer.config ` - (DEPRECATED) consumer config properties (typically empty). This option will be removed in a future version. Use --command-config instead. + * `--command-config ` - command config properties Environment variables: * `LOG_DIR` - log output directory. Typically not needed if logs are written to stderr. @@ -97,7 +98,8 @@ Command line arguments: * `--broker-list ` * `--max-messages ` * `--throughput ` - * `--producer.config ` - producer config properties (typically empty) + * `--producer.config ` - producer config properties (typically empty). This option will be removed in a future version. Use --command-config instead. + * `--command-config ` - command config properties Environment variables: * `LOG_DIR` - log output directory. Typically not needed if logs are written to stderr. diff --git a/tests/kafkatest/services/verifiable_consumer.py b/tests/kafkatest/services/verifiable_consumer.py index 8264566f1c2..99c403ce190 100644 --- a/tests/kafkatest/services/verifiable_consumer.py +++ b/tests/kafkatest/services/verifiable_consumer.py @@ -424,7 +424,7 @@ class VerifiableConsumer(KafkaPathResolverMixin, VerifiableClientMixin, Backgrou if self.max_messages > 0: cmd += " --max-messages %s" % str(self.max_messages) - cmd += " --consumer.config %s" % VerifiableConsumer.CONFIG_FILE + cmd += " --command-config %s" % VerifiableConsumer.CONFIG_FILE cmd += " 2>> %s | tee -a %s &" % (VerifiableConsumer.STDOUT_CAPTURE, VerifiableConsumer.STDOUT_CAPTURE) return cmd diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py index e7a8f411f3a..049de4ce986 100644 --- a/tests/kafkatest/services/verifiable_producer.py +++ b/tests/kafkatest/services/verifiable_producer.py @@ -249,7 +249,7 @@ class VerifiableProducer(KafkaPathResolverMixin, VerifiableClientMixin, Backgrou if self.repeating_keys is not None: cmd += " --repeating-keys %s " % str(self.repeating_keys) - cmd += " --producer.config %s" % VerifiableProducer.CONFIG_FILE + cmd += " --command-config %s" % VerifiableProducer.CONFIG_FILE cmd += " 2>> %s | tee -a %s &" % (VerifiableProducer.STDOUT_CAPTURE, VerifiableProducer.STDOUT_CAPTURE) return cmd diff --git a/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java index 60b4b37abe4..0892693801a 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java @@ -37,11 +37,13 @@ import java.io.IOException; import java.text.SimpleDateFormat; import java.time.Duration; import java.util.Collection; +import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.Random; -import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; import joptsimple.OptionException; import joptsimple.OptionSpec; @@ -134,8 +136,13 @@ public class ConsumerPerformance { long reportingIntervalMs = options.reportingIntervalMs(); boolean showDetailedStats = options.showDetailedStats(); SimpleDateFormat dateFormat = options.dateFormat(); - consumer.subscribe(options.topic(), - new ConsumerPerfRebListener(joinTimeMs, joinStartMs, joinTimeMsInSingleRound)); + + ConsumerPerfRebListener listener = new ConsumerPerfRebListener(joinTimeMs, joinStartMs, joinTimeMsInSingleRound); + if (options.topic().isPresent()) { + consumer.subscribe(options.topic().get(), listener); + } else { + consumer.subscribe(options.include().get(), listener); + } // now start the benchmark long currentTimeMs = System.currentTimeMillis(); @@ -246,6 +253,7 @@ public class ConsumerPerformance { protected static class ConsumerPerfOptions extends CommandDefaultOptions { private final OptionSpec bootstrapServerOpt; private final OptionSpec topicOpt; + private final OptionSpec includeOpt; private final OptionSpec groupIdOpt; private final OptionSpec fetchSizeOpt; private final OptionSpec resetBeginningOffsetOpt; @@ -265,10 +273,14 @@ public class ConsumerPerformance { .withRequiredArg() .describedAs("server to connect to") .ofType(String.class); - topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") + topicOpt = parser.accepts("topic", "The topic to consume from.") .withRequiredArg() .describedAs("topic") .ofType(String.class); + includeOpt = parser.accepts("include", "Regular expression specifying list of topics to include for consumption.") + .withRequiredArg() + .describedAs("Java regex (String)") + .ofType(String.class); groupIdOpt = parser.accepts("group", "The group id to consume on.") .withRequiredArg() .describedAs("gid") @@ -323,7 +335,8 @@ public class ConsumerPerformance { } if (options != null) { CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the consumer performance."); - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt); + CommandLineUtils.checkRequiredArgs(parser, options, numMessagesOpt); + CommandLineUtils.checkOneOfArgs(parser, options, topicOpt, includeOpt); } } @@ -353,8 +366,16 @@ public class ConsumerPerformance { return props; } - public Set topic() { - return Set.of(options.valueOf(topicOpt)); + public Optional> topic() { + return options.has(topicOpt) + ? Optional.of(List.of(options.valueOf(topicOpt))) + : Optional.empty(); + } + + public Optional include() { + return options.has(includeOpt) + ? Optional.of(Pattern.compile(options.valueOf(includeOpt))) + : Optional.empty(); } public long numMessages() { diff --git a/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java b/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java index 8cc9428afbd..ae16d11d8ed 100644 --- a/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java +++ b/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java @@ -126,7 +126,7 @@ public class GetOffsetShell { .ofType(String.class); timeOpt = parser.accepts("time", "timestamp of the offsets before that. [Note: No offset is returned, if the timestamp greater than recently committed record timestamp is given.]") .withRequiredArg() - .describedAs(" / -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered") + .describedAs(" / -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered / -6 or earliest-pending-upload") .ofType(String.class) .defaultsTo("latest"); commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client.") @@ -276,6 +276,8 @@ public class GetOffsetShell { return OffsetSpec.earliestLocal(); case "latest-tiered": return OffsetSpec.latestTiered(); + case "earliest-pending-upload": + return OffsetSpec.earliestPendingUpload(); default: long timestamp; @@ -283,7 +285,7 @@ public class GetOffsetShell { timestamp = Long.parseLong(listOffsetsTimestamp); } catch (NumberFormatException e) { throw new TerseException("Malformed time argument " + listOffsetsTimestamp + ". " + - "Please use -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered, or a specified long format timestamp"); + "Please use -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered / -6 or earliest-pending-upload, or a specified long format timestamp"); } if (timestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP) { @@ -296,6 +298,8 @@ public class GetOffsetShell { return OffsetSpec.earliestLocal(); } else if (timestamp == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) { return OffsetSpec.latestTiered(); + } else if (timestamp == ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP) { + return OffsetSpec.earliestPendingUpload(); } else { return OffsetSpec.forTimestamp(timestamp); } diff --git a/tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java b/tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java index accf5241a35..497b8b92b27 100644 --- a/tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java +++ b/tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java @@ -43,8 +43,8 @@ import java.nio.file.Files; import java.nio.file.Path; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.Iterator; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -61,6 +61,8 @@ import joptsimple.OptionParser; import joptsimple.OptionSet; import joptsimple.OptionSpec; +import static java.util.stream.Collectors.toCollection; + /** * This is a torture test that runs against an existing broker @@ -246,9 +248,9 @@ public class LogCompactionTester { int sleepSecs = optionSet.valueOf(options.sleepSecsOpt); long testId = RANDOM.nextLong(); - String[] topics = IntStream.range(0, topicCount) + Set topics = IntStream.range(0, topicCount) .mapToObj(i -> "log-cleaner-test-" + testId + "-" + i) - .toArray(String[]::new); + .collect(toCollection(LinkedHashSet::new)); createTopics(brokerUrl, topics); System.out.println("Producing " + messages + " messages..to topics " + String.join(",", topics)); @@ -278,7 +280,7 @@ public class LogCompactionTester { } - private static void createTopics(String brokerUrl, String[] topics) throws Exception { + private static void createTopics(String brokerUrl, Set topics) throws Exception { Properties adminConfig = new Properties(); adminConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerUrl); @@ -286,7 +288,7 @@ public class LogCompactionTester { Map topicConfigs = Map.of( TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT ); - List newTopics = Arrays.stream(topics) + List newTopics = topics.stream() .map(name -> new NewTopic(name, 1, (short) 1).configs(topicConfigs)).toList(); adminClient.createTopics(newTopics).all().get(); @@ -296,7 +298,7 @@ public class LogCompactionTester { Set allTopics = adminClient.listTopics().names().get(); pendingTopics.clear(); pendingTopics.addAll( - Arrays.stream(topics) + topics.stream() .filter(topicName -> !allTopics.contains(topicName)) .toList() ); @@ -392,7 +394,7 @@ public class LogCompactionTester { } } - private static Path produceMessages(String brokerUrl, String[] topics, long messages, + private static Path produceMessages(String brokerUrl, Set topics, long messages, String compressionType, int dups, int percentDeletes) throws IOException { Map producerProps = Map.of( ProducerConfig.MAX_BLOCK_MS_CONFIG, String.valueOf(Long.MAX_VALUE), @@ -408,8 +410,10 @@ public class LogCompactionTester { try (BufferedWriter producedWriter = Files.newBufferedWriter( producedFilePath, StandardCharsets.UTF_8)) { - for (long i = 0; i < messages * topics.length; i++) { - String topic = topics[(int) (i % topics.length)]; + List topicsList = List.copyOf(topics); + int size = topicsList.size(); + for (long i = 0; i < messages * size; i++) { + String topic = topicsList.get((int) (i % size)); int key = RANDOM.nextInt(keyCount); boolean delete = (i % 100) < percentDeletes; ProducerRecord record; @@ -430,14 +434,14 @@ public class LogCompactionTester { } } - private static Path consumeMessages(String brokerUrl, String[] topics) throws IOException { + private static Path consumeMessages(String brokerUrl, Set topics) throws IOException { Path consumedFilePath = Files.createTempFile("kafka-log-cleaner-consumed-", ".txt"); System.out.println("Logging consumed messages to " + consumedFilePath); try (Consumer consumer = createConsumer(brokerUrl); BufferedWriter consumedWriter = Files.newBufferedWriter(consumedFilePath, StandardCharsets.UTF_8)) { - consumer.subscribe(Arrays.asList(topics)); + consumer.subscribe(topics); while (true) { ConsumerRecords consumerRecords = consumer.poll(Duration.ofSeconds(20)); if (consumerRecords.isEmpty()) return consumedFilePath; diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java index d6ed1d0a4ef..fa21432b587 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java @@ -81,7 +81,7 @@ public class ProducerPerformance { System.out.println("Warmup first " + config.warmupRecords + " records. Steady state results will print after the complete test summary."); } boolean isSteadyState = false; - stats = new Stats(config.numRecords, isSteadyState); + stats = new Stats(config.numRecords, config.reportingInterval, isSteadyState); long startMs = System.currentTimeMillis(); ThroughputThrottler throttler = new ThroughputThrottler(config.throughput, startMs); @@ -101,7 +101,7 @@ public class ProducerPerformance { long sendStartMs = System.currentTimeMillis(); if ((isSteadyState = config.warmupRecords > 0) && i == config.warmupRecords) { - steadyStateStats = new Stats(config.numRecords - config.warmupRecords, isSteadyState); + steadyStateStats = new Stats(config.numRecords - config.warmupRecords, config.reportingInterval, isSteadyState); stats.suppressPrinting(); } cb = new PerfCallback(sendStartMs, payload.length, stats, steadyStateStats); @@ -131,7 +131,7 @@ public class ProducerPerformance { steadyStateStats.printTotal(); } } else { - // Make sure all messages are sent before printing out the stats and the metrics + // Make sure all records are sent before printing out the stats and the metrics // We need to do this in a different branch for now since tests/kafkatest/sanity_checks/test_performance_services.py // expects this class to work with older versions of the client jar that don't support flush(). producer.flush(); @@ -177,7 +177,7 @@ public class ProducerPerformance { } else if (payloadMonotonic) { payload = Long.toString(recordValue).getBytes(StandardCharsets.UTF_8); } else { - throw new IllegalArgumentException("no payload File Path or record Size or payload-monotonic option provided"); + throw new IllegalArgumentException("No payload file, record size or payload-monotonic option provided."); } return payload; } @@ -221,7 +221,7 @@ public class ProducerPerformance { } } - System.out.println("Number of messages read: " + payloadByteList.size()); + System.out.println("Number of records read: " + payloadByteList.size()); } return payloadByteList; @@ -230,24 +230,34 @@ public class ProducerPerformance { /** Get the command-line argument parser. */ static ArgumentParser argParser() { ArgumentParser parser = ArgumentParsers - .newArgumentParser("producer-performance") + .newArgumentParser("kafka-producer-perf-test") .defaultHelp(true) .description("This tool is used to verify the producer performance. To enable transactions, " + - "you can specify a transaction id or set a transaction duration using --transaction-duration-ms. " + - "There are three ways to specify the transaction id: set transaction.id= via --producer-props, " + - "set transaction.id= in the config file via --producer.config, or use --transaction-id ."); + "you can specify a transactional id or set a transaction duration using --transaction-duration-ms. " + + "There are three ways to specify the transactional id: set transactional.id= via --command-property, " + + "set transactional.id= in the config file via --command-config, or use --transactional-id ."); + + parser.addArgument("--bootstrap-server") + .action(store()) + .required(false) + .type(String.class) + .metavar("BOOTSTRAP-SERVERS") + .dest("bootstrapServers") + .help("The server(s) to connect to. This config takes precedence over bootstrap.servers specified " + + "via --command-property or --command-config."); MutuallyExclusiveGroup payloadOptions = parser .addMutuallyExclusiveGroup() .required(true) - .description("either --record-size or --payload-file must be specified but not both."); + .description("Note that you must provide exactly one of --record-size, --payload-file " + + "or --payload-monotonic."); parser.addArgument("--topic") .action(store()) .required(true) .type(String.class) .metavar("TOPIC") - .help("produce messages to this topic"); + .help("Produce records to this topic."); parser.addArgument("--num-records") .action(store()) @@ -255,7 +265,7 @@ public class ProducerPerformance { .type(Long.class) .metavar("NUM-RECORDS") .dest("numRecords") - .help("number of messages to produce"); + .help("Number of records to produce."); payloadOptions.addArgument("--record-size") .action(store()) @@ -263,7 +273,7 @@ public class ProducerPerformance { .type(Integer.class) .metavar("RECORD-SIZE") .dest("recordSize") - .help("message size in bytes. Note that you must provide exactly one of --record-size or --payload-file " + + .help("Record size in bytes. Note that you must provide exactly one of --record-size, --payload-file " + "or --payload-monotonic."); payloadOptions.addArgument("--payload-file") @@ -272,17 +282,17 @@ public class ProducerPerformance { .type(String.class) .metavar("PAYLOAD-FILE") .dest("payloadFile") - .help("file to read the message payloads from. This works only for UTF-8 encoded text files. " + - "Payloads will be read from this file and a payload will be randomly selected when sending messages. " + - "Note that you must provide exactly one of --record-size or --payload-file or --payload-monotonic."); + .help("File to read the record payloads from. This works only for UTF-8 encoded text files. " + + "Payloads will be read from this file and a payload will be randomly selected when sending records. " + + "Note that you must provide exactly one of --record-size, --payload-file or --payload-monotonic."); payloadOptions.addArgument("--payload-monotonic") .action(storeTrue()) .type(Boolean.class) .metavar("PAYLOAD-MONOTONIC") .dest("payloadMonotonic") - .help("payload is monotonically increasing integer. Note that you must provide exactly one of --record-size " + - "or --payload-file or --payload-monotonic."); + .help("Payload is a monotonically increasing integer. Note that you must provide exactly one of --record-size, " + + "--payload-file or --payload-monotonic."); parser.addArgument("--payload-delimiter") .action(store()) @@ -291,8 +301,7 @@ public class ProducerPerformance { .metavar("PAYLOAD-DELIMITER") .dest("payloadDelimiter") .setDefault("\\n") - .help("provides delimiter to be used when --payload-file is provided. " + - "Defaults to new line. " + + .help("Provides the delimiter to be used when --payload-file is provided. Defaults to new line. " + "Note that this parameter will be ignored if --payload-file is not provided."); parser.addArgument("--throughput") @@ -300,16 +309,26 @@ public class ProducerPerformance { .required(true) .type(Double.class) .metavar("THROUGHPUT") - .help("throttle maximum message throughput to *approximately* THROUGHPUT messages/sec. Set this to -1 to disable throttling."); + .help("Throttle maximum record throughput to *approximately* THROUGHPUT records/sec. Set this to -1 to disable throttling."); parser.addArgument("--producer-props") - .nargs("+") - .required(false) - .metavar("PROP-NAME=PROP-VALUE") - .type(String.class) - .dest("producerConfig") - .help("kafka producer related configuration properties like bootstrap.servers,client.id etc. " + - "These configs take precedence over those passed via --producer.config."); + .nargs("+") + .required(false) + .metavar("PROP-NAME=PROP-VALUE") + .type(String.class) + .dest("producerConfig") + .help("(DEPRECATED) Kafka producer related configuration properties like client.id. " + + "These configs take precedence over those passed via --command-config or --producer.config. " + + "This option will be removed in a future version. Use --command-property instead."); + + parser.addArgument("--command-property") + .nargs("+") + .required(false) + .metavar("PROP-NAME=PROP-VALUE") + .type(String.class) + .dest("commandProperties") + .help("Kafka producer related configuration properties like client.id. " + + "These configs take precedence over those passed via --command-config or --producer.config."); parser.addArgument("--producer.config") .action(store()) @@ -317,46 +336,64 @@ public class ProducerPerformance { .type(String.class) .metavar("CONFIG-FILE") .dest("producerConfigFile") - .help("producer config properties file."); + .help("(DEPRECATED) Producer config properties file. " + + "This option will be removed in a future version. Use --command-config instead."); + + parser.addArgument("--command-config") + .action(store()) + .required(false) + .type(String.class) + .metavar("CONFIG-FILE") + .dest("commandConfigFile") + .help("Producer config properties file."); parser.addArgument("--print-metrics") .action(storeTrue()) .type(Boolean.class) .metavar("PRINT-METRICS") .dest("printMetrics") - .help("print out metrics at the end of the test."); + .help("Print out metrics at the end of the test."); parser.addArgument("--transactional-id") - .action(store()) - .required(false) - .type(String.class) - .metavar("TRANSACTIONAL-ID") - .dest("transactionalId") - .help("The transactional id to use. This config takes precedence over the transactional.id " + - "specified via --producer.config or --producer-props. Note that if the transactional id " + - "is not specified while --transaction-duration-ms is provided, the default value for the " + - "transactional id will be performance-producer- followed by a random uuid."); + .action(store()) + .required(false) + .type(String.class) + .metavar("TRANSACTIONAL-ID") + .dest("transactionalId") + .help("The transactional id to use. This config takes precedence over the transactional.id " + + "specified via --command-property or --command-config. Note that if the transactional id " + + "is not specified while --transaction-duration-ms is provided, the default value for the " + + "transactional id will be performance-producer- followed by a random uuid."); parser.addArgument("--transaction-duration-ms") - .action(store()) - .required(false) - .type(Long.class) - .metavar("TRANSACTION-DURATION") - .dest("transactionDurationMs") - .help("The max age of each transaction. The commitTransaction will be called after this time has elapsed. " + - "The value should be greater than 0. If the transactional id is specified via --producer-props, " + - "--producer.config, or --transactional-id but --transaction-duration-ms is not specified, " + - "the default value will be 3000."); + .action(store()) + .required(false) + .type(Long.class) + .metavar("TRANSACTION-DURATION") + .dest("transactionDurationMs") + .help("The maximum duration of each transaction. The commitTransaction will be called after this time has elapsed. " + + "The value should be greater than 0. If the transactional id is specified via --command-property, " + + "--command-config or --transactional-id but --transaction-duration-ms is not specified, " + + "the default value will be 3000."); parser.addArgument("--warmup-records") - .action(store()) - .required(false) - .type(Long.class) - .metavar("WARMUP-RECORDS") - .dest("warmupRecords") - .setDefault(0L) - .help("The number of records to treat as warmup; these initial records will not be included in steady-state statistics. " + - "An additional summary line will be printed describing the steady-state statistics. (default: 0)."); + .action(store()) + .required(false) + .type(Long.class) + .metavar("WARMUP-RECORDS") + .dest("warmupRecords") + .setDefault(0L) + .help("The number of records to treat as warmup. These initial records will not be included in steady-state statistics. " + + "An additional summary line will be printed describing the steady-state statistics."); + + parser.addArgument("--reporting-interval") + .action(store()) + .required(false) + .type(Long.class) + .metavar("INTERVAL-MS") + .dest("reportingInterval") + .setDefault(5_000L) + .help("Interval in milliseconds at which to print progress info."); return parser; } @@ -381,7 +418,7 @@ public class ProducerPerformance { private final boolean isSteadyState; private boolean suppressPrint; - public Stats(long numRecords, boolean isSteadyState) { + public Stats(long numRecords, long reportingInterval, boolean isSteadyState) { this.start = System.currentTimeMillis(); this.windowStart = System.currentTimeMillis(); this.iteration = 0; @@ -394,7 +431,7 @@ public class ProducerPerformance { this.windowTotalLatency = 0; this.windowBytes = 0; this.totalLatency = 0; - this.reportingInterval = 5000; + this.reportingInterval = reportingInterval; this.isSteadyState = isSteadyState; this.suppressPrint = false; } @@ -529,6 +566,7 @@ public class ProducerPerformance { } static final class ConfigPostProcessor { + final String bootstrapServers; final String topicName; final long numRecords; final long warmupRecords; @@ -540,9 +578,11 @@ public class ProducerPerformance { final Long transactionDurationMs; final boolean transactionsEnabled; final List payloadByteList; + final long reportingInterval; public ConfigPostProcessor(ArgumentParser parser, String[] args) throws IOException, ArgumentParserException { Namespace namespace = parser.parseArgs(args); + this.bootstrapServers = namespace.getString("bootstrapServers"); this.topicName = namespace.getString("topic"); this.numRecords = namespace.getLong("numRecords"); this.warmupRecords = Math.max(namespace.getLong("warmupRecords"), 0); @@ -550,33 +590,56 @@ public class ProducerPerformance { this.throughput = namespace.getDouble("throughput"); this.payloadMonotonic = namespace.getBoolean("payloadMonotonic"); this.shouldPrintMetrics = namespace.getBoolean("printMetrics"); + this.reportingInterval = namespace.getLong("reportingInterval"); List producerConfigs = namespace.getList("producerConfig"); String producerConfigFile = namespace.getString("producerConfigFile"); + List commandProperties = namespace.getList("commandProperties"); + String commandConfigFile = namespace.getString("commandConfigFile"); String payloadFilePath = namespace.getString("payloadFile"); Long transactionDurationMsArg = namespace.getLong("transactionDurationMs"); String transactionIdArg = namespace.getString("transactionalId"); if (numRecords <= 0) { - throw new ArgumentParserException("--num-records should be greater than zero", parser); + throw new ArgumentParserException("--num-records should be greater than zero.", parser); } if (warmupRecords >= numRecords) { throw new ArgumentParserException("The value for --warmup-records must be strictly fewer than the number of records in the test, --num-records.", parser); } if (recordSize != null && recordSize <= 0) { - throw new ArgumentParserException("--record-size should be greater than zero", parser); + throw new ArgumentParserException("--record-size should be greater than zero.", parser); } - if (producerConfigs == null && producerConfigFile == null) { - throw new ArgumentParserException("Either --producer-props or --producer.config must be specified.", parser); + if (bootstrapServers == null && commandProperties == null && producerConfigs == null && producerConfigFile == null && commandConfigFile == null) { + throw new ArgumentParserException("At least one of --bootstrap-server, --command-property, --producer-props, --producer.config or --command-config must be specified.", parser); + } + if (commandProperties != null && producerConfigs != null) { + throw new ArgumentParserException("--command-property and --producer-props cannot be specified together.", parser); + } + if (commandConfigFile != null && producerConfigFile != null) { + throw new ArgumentParserException("--command-config and --producer.config cannot be specified together.", parser); } if (transactionDurationMsArg != null && transactionDurationMsArg <= 0) { - throw new ArgumentParserException("--transaction-duration-ms should be greater than zero", parser); + throw new ArgumentParserException("--transaction-duration-ms should be greater than zero.", parser); + } + if (reportingInterval <= 0) { + throw new ArgumentParserException("--reporting-interval should be greater than zero.", parser); } // since default value gets printed with the help text, we are escaping \n there and replacing it with correct value here. String payloadDelimiter = namespace.getString("payloadDelimiter").equals("\\n") ? "\n" : namespace.getString("payloadDelimiter"); this.payloadByteList = readPayloadFile(payloadFilePath, payloadDelimiter); - this.producerProps = readProps(producerConfigs, producerConfigFile); + if (producerConfigs != null) { + System.out.println("Option --producer-props has been deprecated and will be removed in a future version. Use --command-property instead."); + commandProperties = producerConfigs; + } + if (producerConfigFile != null) { + System.out.println("Option --producer.config has been deprecated and will be removed in a future version. Use --command-config instead."); + commandConfigFile = producerConfigFile; + } + this.producerProps = readProps(commandProperties, commandConfigFile); + if (bootstrapServers != null) { + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + } // setup transaction related configs this.transactionsEnabled = transactionDurationMsArg != null || transactionIdArg != null diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java index 1fa1aed8c4f..1ecea5331dd 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java @@ -597,7 +597,7 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons .setDefault("earliest") .type(String.class) .dest("resetPolicy") - .help("Set reset policy (must be either 'earliest', 'latest', or 'none'"); + .help("Set reset policy (must be either 'earliest', 'latest', or 'none')"); parser.addArgument("--assignment-strategy") .action(store()) @@ -611,8 +611,17 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons .action(store()) .required(false) .type(String.class) - .metavar("CONFIG_FILE") - .help("Consumer config properties file (config options shared with command line parameters will be overridden)."); + .metavar("CONFIG-FILE") + .help("(DEPRECATED) Consumer config properties file" + + "This option will be removed in a future version. Use --command-config instead"); + + parser.addArgument("--command-config") + .action(store()) + .required(false) + .type(String.class) + .metavar("CONFIG-FILE") + .dest("commandConfigFile") + .help("Config properties file (config options shared with command line parameters will be overridden)."); return parser; } @@ -622,16 +631,28 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons boolean useAutoCommit = res.getBoolean("useAutoCommit"); String configFile = res.getString("consumer.config"); + String commandConfigFile = res.getString("commandConfigFile"); String brokerHostAndPort = res.getString("bootstrapServer"); Properties consumerProps = new Properties(); + if (configFile != null && commandConfigFile != null) { + throw new ArgumentParserException("Options --consumer.config and --command-config are mutually exclusive.", parser); + } if (configFile != null) { + System.out.println("Option --consumer.config has been deprecated and will be removed in a future version. Use --command-config instead."); try { consumerProps.putAll(Utils.loadProps(configFile)); } catch (IOException e) { throw new ArgumentParserException(e.getMessage(), parser); } } + if (commandConfigFile != null) { + try { + consumerProps.putAll(Utils.loadProps(res.getString(commandConfigFile))); + } catch (IOException e) { + throw new ArgumentParserException(e.getMessage(), parser); + } + } GroupProtocol groupProtocol = GroupProtocol.of(res.getString("groupProtocol")); consumerProps.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name()); diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java index 33f0b3142e8..bfbbb8a4854 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java @@ -161,8 +161,9 @@ public class VerifiableProducer implements AutoCloseable { .action(store()) .required(false) .type(String.class) - .metavar("CONFIG_FILE") - .help("Producer config properties file."); + .metavar("CONFIG-FILE") + .help("(DEPRECATED) Producer config properties file. " + + "This option will be removed in a future version. Use --command-config instead."); parser.addArgument("--message-create-time") .action(store()) @@ -189,6 +190,14 @@ public class VerifiableProducer implements AutoCloseable { .dest("repeatingKeys") .help("If specified, each produced record will have a key starting at 0 increment by 1 up to the number specified (exclusive), then the key is set to 0 again"); + parser.addArgument("--command-config") + .action(store()) + .required(false) + .type(String.class) + .metavar("CONFIG-FILE") + .dest("commandConfigFile") + .help("Config properties file (config options shared with command line parameters will be overridden)."); + return parser; } @@ -217,6 +226,7 @@ public class VerifiableProducer implements AutoCloseable { String topic = res.getString("topic"); int throughput = res.getInt("throughput"); String configFile = res.getString("producer.config"); + String commandConfigFile = res.getString("commandConfigFile"); Integer valuePrefix = res.getInt("valuePrefix"); Long createTime = res.getLong("createTime"); Integer repeatingKeys = res.getInt("repeatingKeys"); @@ -240,14 +250,25 @@ public class VerifiableProducer implements AutoCloseable { producerProps.put(ProducerConfig.ACKS_CONFIG, Integer.toString(res.getInt("acks"))); // No producer retries producerProps.put(ProducerConfig.RETRIES_CONFIG, "0"); + if (configFile != null && commandConfigFile != null) { + throw new ArgumentParserException("Options --producer.config and --command-config are mutually exclusive.", parser); + } + if (configFile != null) { + System.out.println("Option --producer.config has been deprecated and will be removed in a future version. Use --command-config instead."); try { producerProps.putAll(loadProps(configFile)); } catch (IOException e) { throw new ArgumentParserException(e.getMessage(), parser); } } - + if (commandConfigFile != null) { + try { + producerProps.putAll(loadProps(commandConfigFile)); + } catch (IOException e) { + throw new ArgumentParserException(e.getMessage(), parser); + } + } StringSerializer serializer = new StringSerializer(); KafkaProducer producer = new KafkaProducer<>(producerProps, serializer, serializer); diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java index cf4daa0c636..60ee8f61ffa 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java @@ -25,10 +25,8 @@ import org.apache.kafka.server.util.CommandDefaultOptions; import org.apache.kafka.server.util.CommandLineUtils; import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; @@ -185,12 +183,8 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { } private void checkRequiredArgs() { - List> topicOrFilterArgs = new ArrayList<>(List.of(topicArg(), includedTopicsArg())); - topicOrFilterArgs.removeIf(Optional::isEmpty); // user need to specify value for either --topic or --include options - if (topicOrFilterArgs.size() != 1) { - CommandLineUtils.printUsageAndExit(parser, "Exactly one of --include/--topic is required. "); - } + CommandLineUtils.checkOneOfArgs(parser, options, topicOpt, includeOpt); if (partitionArg().isPresent()) { if (!options.has(topicOpt)) { diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java index 4628c34fe18..8e1f6114c24 100644 --- a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java @@ -566,10 +566,11 @@ public class ReassignPartitionsCommand { List topicsToReassign = t0.getValue(); Map> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign); + Map currentReplicaLogDirs = getReplicaToLogDir(adminClient, currentAssignments); List usableBrokers = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness); Map> proposedAssignments = calculateAssignment(currentAssignments, usableBrokers); System.out.printf("Current partition replica assignment%n%s%n%n", - formatAsReassignmentJson(currentAssignments, Map.of())); + formatAsReassignmentJson(currentAssignments, currentReplicaLogDirs)); System.out.printf("Proposed partition reassignment configuration%n%s%n", formatAsReassignmentJson(proposedAssignments, Map.of())); return Map.entry(proposedAssignments, currentAssignments); @@ -775,7 +776,7 @@ public class ReassignPartitionsCommand { verifyBrokerIds(adminClient, brokers); Map> currentParts = getReplicaAssignmentForPartitions(adminClient, proposedParts.keySet()); - System.out.println(currentPartitionReplicaAssignmentToString(proposedParts, currentParts)); + System.out.println(currentPartitionReplicaAssignmentToString(adminClient, proposedParts, currentParts)); if (interBrokerThrottle >= 0 || logDirThrottle >= 0) { System.out.println(YOU_MUST_RUN_VERIFY_PERIODICALLY_MESSAGE); @@ -916,20 +917,23 @@ public class ReassignPartitionsCommand { /** * Return the string which we want to print to describe the current partition assignment. * + * @param adminClient The admin client object to use. * @param proposedParts The proposed partition assignment. * @param currentParts The current partition assignment. * * @return The string to print. We will only print information about * partitions that appear in the proposed partition assignment. */ - static String currentPartitionReplicaAssignmentToString(Map> proposedParts, - Map> currentParts) throws JsonProcessingException { + static String currentPartitionReplicaAssignmentToString(Admin adminClient, + Map> proposedParts, + Map> currentParts) throws JsonProcessingException, ExecutionException, InterruptedException { Map> partitionsToBeReassigned = currentParts.entrySet().stream() .filter(e -> proposedParts.containsKey(e.getKey())) .collect(Collectors.toMap(Entry::getKey, Entry::getValue)); + Map currentReplicaLogDirs = getReplicaToLogDir(adminClient, partitionsToBeReassigned); return String.format("Current partition replica assignment%n%n%s%n%nSave this to use as the %s", - formatAsReassignmentJson(partitionsToBeReassigned, Map.of()), + formatAsReassignmentJson(partitionsToBeReassigned, currentReplicaLogDirs), "--reassignment-json-file option during rollback"); } @@ -1514,4 +1518,26 @@ public class ReassignPartitionsCommand { } return results; } + + static Map getReplicaToLogDir( + Admin adminClient, + Map> topicPartitionToReplicas + ) throws InterruptedException, ExecutionException { + var replicaLogDirs = topicPartitionToReplicas + .entrySet() + .stream() + .flatMap(entry -> entry.getValue() + .stream() + .map(id -> new TopicPartitionReplica(entry.getKey().topic(), entry.getKey().partition(), id))) + .collect(Collectors.toUnmodifiableSet()); + + return adminClient.describeReplicaLogDirs(replicaLogDirs).all().get() + .entrySet() + .stream() + .filter(entry -> entry.getValue().getCurrentReplicaLogDir() != null) + .collect(Collectors.toMap( + Entry::getKey, + entry -> entry.getValue().getCurrentReplicaLogDir() + )); + } } diff --git a/tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java b/tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java index 0f68bf82900..0c54f6c53f9 100644 --- a/tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java @@ -881,6 +881,7 @@ public class StreamsGroupCommand { List topics = opts.options.valuesOf(opts.inputTopicOpt); List partitions = offsetsUtils.parseTopicPartitionsToReset(topics); + offsetsUtils.checkAllTopicPartitionsValid(partitions); // if the user specified topics that do not belong to this group, we filter them out partitions = filterExistingGroupTopics(groupId, partitions); return partitions; diff --git a/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java b/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java index 270fab2cf80..d78b65e54a3 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java @@ -75,7 +75,7 @@ public class ConsumerPerformanceTest { ConsumerPerformance.ConsumerPerfOptions config = new ConsumerPerformance.ConsumerPerfOptions(args); assertEquals("localhost:9092", config.brokerHostsAndPorts()); - assertTrue(config.topic().contains("test")); + assertTrue(config.topic().get().contains("test")); assertEquals(10, config.numMessages()); } @@ -93,6 +93,47 @@ public class ConsumerPerformanceTest { assertTrue(err.contains("new-consumer is not a recognized option")); } + @Test + public void testConfigWithInclude() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--include", "test.*", + "--messages", "10" + }; + + ConsumerPerformance.ConsumerPerfOptions config = new ConsumerPerformance.ConsumerPerfOptions(args); + + assertEquals("localhost:9092", config.brokerHostsAndPorts()); + assertTrue(config.include().get().toString().contains("test.*")); + assertEquals(10, config.numMessages()); + } + + @Test + public void testConfigWithTopicAndInclude() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--include", "test.*", + "--messages", "10" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> new ConsumerPerformance.ConsumerPerfOptions(args)); + + assertTrue(err.contains("Exactly one of the following arguments is required: [topic], [include]")); + } + + @Test + public void testConfigWithoutTopicAndInclude() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--messages", "10" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> new ConsumerPerformance.ConsumerPerfOptions(args)); + + assertTrue(err.contains("Exactly one of the following arguments is required: [topic], [include]")); + } + @Test public void testClientIdOverride() throws IOException { File tempFile = Files.createFile(tempDir.resolve("test_consumer_config.conf")).toFile(); diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellParsingTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellParsingTest.java index 53c1c4d79c9..db53695a7be 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellParsingTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellParsingTest.java @@ -247,7 +247,7 @@ public class GetOffsetShellParsingTest { @Test public void testInvalidOffset() { assertEquals("Malformed time argument foo. " + - "Please use -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered, or a specified long format timestamp", + "Please use -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered / -6 or earliest-pending-upload, or a specified long format timestamp", assertThrows(TerseException.class, () -> GetOffsetShell.parseOffsetSpec("foo")).getMessage()); } diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 9986daa7f3b..c1c7b27639a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -367,6 +367,30 @@ public class GetOffsetShellTest { } } + @ClusterTemplate("withRemoteStorage") + public void testGetOffsetsByEarliestTieredSpec() throws InterruptedException { + setUp(); + setUpRemoteLogTopics(); + + for (String time : new String[] {"-6", "earliest-pending-upload"}) { + // test topics disable remote log storage + // as remote log disabled, broker returns unknown offset of each topic partition and these + // unknown offsets are ignore by GetOffsetShell, hence we have empty result here. + assertEquals(List.of(), + executeAndParse("--topic-partitions", "topic\\d+:0", "--time", time)); + + // test topics enable remote log storage + TestUtils.waitForCondition(() -> + List.of( + new Row("topicRLS1", 0, 0L), + new Row("topicRLS2", 0, 1L), + new Row("topicRLS3", 0, 2L), + new Row("topicRLS4", 0, 3L)) + .equals(executeAndParse("--topic-partitions", "topicRLS.*:0", "--time", time)), + "testGetOffsetsByEarliestTieredSpec result not match"); + } + } + @ClusterTest public void testGetOffsetsByTimestamp() { setUp(); diff --git a/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java b/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java index 7217c2101a0..007a22a8020 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java @@ -188,7 +188,7 @@ public class ProducerPerformanceTest { "--num-records", "5", "--throughput", "100", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; producerPerformanceSpy.start(args); verify(producerMock, times(5)).send(any(), any()); verify(producerMock, times(1)).close(); @@ -205,7 +205,7 @@ public class ProducerPerformanceTest { "--throughput", "100", "--record-size", "100", "--transactional-id", "foobar", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; producerPerformanceSpy.start(args); verify(producerMock, times(1)).beginTransaction(); verify(producerMock, times(1)).commitTransaction(); @@ -225,7 +225,7 @@ public class ProducerPerformanceTest { "--num-records", "10", "--throughput", "1", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; producerPerformanceSpy.start(args); verify(producerMock, times(10)).send(any(), any()); @@ -246,7 +246,7 @@ public class ProducerPerformanceTest { "--num-records", "10", "--throughput", "1", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; producerPerformanceSpy.start(args); verify(producerMock, times(10)).send(any(), any()); @@ -263,7 +263,7 @@ public class ProducerPerformanceTest { "--throughput", "100", "--record-size", "100", "--payload-monotonic", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; ArgumentParser parser1 = ProducerPerformance.argParser(); ArgumentParserException thrown = assertThrows(ArgumentParserException.class, () -> parser1.parseArgs(args1)); assertEquals("argument --payload-monotonic: not allowed with argument --record-size", thrown.getMessage()); @@ -274,7 +274,7 @@ public class ProducerPerformanceTest { "--throughput", "100", "--payload-file", "abc.txt", "--payload-monotonic", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; ArgumentParser parser2 = ProducerPerformance.argParser(); thrown = assertThrows(ArgumentParserException.class, () -> parser2.parseArgs(args2)); assertEquals("argument --payload-monotonic: not allowed with argument --payload-file", thrown.getMessage()); @@ -287,8 +287,8 @@ public class ProducerPerformanceTest { "--topic", "Hello-Kafka", "--num-records", "5", "--throughput", "100", - "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--record-size", "100", + "--bootstrap-server", "localhost:9000"}; ArgumentParser parser = ProducerPerformance.argParser(); ArgumentParserException thrown = assertThrows(ArgumentParserException.class, () -> parser.parseArgs(args)); assertEquals("unrecognized arguments: '--test'", thrown.getMessage()); @@ -301,7 +301,7 @@ public class ProducerPerformanceTest { "--num-records", "5", "--throughput", "1.25", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; ArgumentParser parser = ProducerPerformance.argParser(); assertDoesNotThrow(() -> parser.parseArgs(args)); } @@ -354,7 +354,7 @@ public class ProducerPerformanceTest { SplittableRandom random = new SplittableRandom(0); IllegalArgumentException thrown = assertThrows(IllegalArgumentException.class, () -> ProducerPerformance.generateRandomPayload(recordSize, payloadByteList, payload, random, false, 0L)); - assertEquals("no payload File Path or record Size or payload-monotonic option provided", thrown.getMessage()); + assertEquals("No payload file, record size or payload-monotonic option provided.", thrown.getMessage()); } @Test @@ -380,14 +380,14 @@ public class ProducerPerformanceTest { @Test public void testStatsInitializationWithLargeNumRecords() { long numRecords = Long.MAX_VALUE; - assertDoesNotThrow(() -> new ProducerPerformance.Stats(numRecords, false)); + assertDoesNotThrow(() -> new ProducerPerformance.Stats(numRecords, 5000L, false)); } @Test public void testStatsCorrectness() throws Exception { ExecutorService singleThreaded = Executors.newSingleThreadExecutor(); final long numRecords = 1000000; - ProducerPerformance.Stats stats = new ProducerPerformance.Stats(numRecords, false); + ProducerPerformance.Stats stats = new ProducerPerformance.Stats(numRecords, 5000L, false); for (long i = 0; i < numRecords; i++) { final Callback callback = new ProducerPerformance.PerfCallback(0, 100, stats, null); CompletableFuture.runAsync(() -> callback.onCompletion(null, null), singleThreaded); @@ -412,11 +412,12 @@ public class ProducerPerformanceTest { "--throughput", "100", "--record-size", "100", "--print-metrics", - "--producer-props", "bootstrap.servers=localhost:9000", + "--bootstrap-server", "localhost:9000", "--transactional-id", "foobar", "--transaction-duration-ms", "5000", }; ProducerPerformance.ConfigPostProcessor configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertEquals("localhost:9000", configs.bootstrapServers); assertEquals("Hello-Kafka", configs.topicName); assertEquals(5, configs.numRecords); assertEquals(100, configs.throughput); @@ -438,28 +439,28 @@ public class ProducerPerformanceTest { "--num-records", "5", "--throughput", "100", "--record-size", "100"}; - assertEquals("Either --producer-props or --producer.config must be specified.", - assertThrows(ArgumentParserException.class, - () -> new ProducerPerformance.ConfigPostProcessor(parser, invalidProducerProps)).getMessage()); + assertEquals("At least one of --bootstrap-server, --command-property, --producer-props, --producer.config or --command-config must be specified.", + assertThrows(ArgumentParserException.class, + () -> new ProducerPerformance.ConfigPostProcessor(parser, invalidProducerProps)).getMessage()); String[] invalidTransactionDurationMs = new String[]{ "--topic", "Hello-Kafka", "--num-records", "5", "--throughput", "100", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000", + "--bootstrap-server", "localhost:9000", "--transaction-duration-ms", "0"}; - assertEquals("--transaction-duration-ms should be greater than zero", - assertThrows(ArgumentParserException.class, - () -> new ProducerPerformance.ConfigPostProcessor(parser, invalidTransactionDurationMs)).getMessage()); + assertEquals("--transaction-duration-ms should be greater than zero.", + assertThrows(ArgumentParserException.class, + () -> new ProducerPerformance.ConfigPostProcessor(parser, invalidTransactionDurationMs)).getMessage()); String[] invalidNumRecords = new String[]{ "--topic", "Hello-Kafka", "--num-records", "-5", "--throughput", "100", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; - assertEquals("--num-records should be greater than zero", + "--bootstrap-server", "localhost:9000"}; + assertEquals("--num-records should be greater than zero.", assertThrows(ArgumentParserException.class, () -> new ProducerPerformance.ConfigPostProcessor(parser, invalidNumRecords)).getMessage()); @@ -468,10 +469,72 @@ public class ProducerPerformanceTest { "--num-records", "5", "--throughput", "100", "--record-size", "-100", - "--producer-props", "bootstrap.servers=localhost:9000"}; - assertEquals("--record-size should be greater than zero", + "--bootstrap-server", "localhost:9000"}; + assertEquals("--record-size should be greater than zero.", assertThrows(ArgumentParserException.class, () -> new ProducerPerformance.ConfigPostProcessor(parser, invalidRecordSize)).getMessage()); + + String[] invalidReportingInterval = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--reporting-interval", "0", + "--bootstrap-server", "localhost:9000"}; + assertEquals("--reporting-interval should be greater than zero.", + assertThrows(ArgumentParserException.class, + () -> new ProducerPerformance.ConfigPostProcessor(parser, invalidReportingInterval)).getMessage()); + } + + @Test + public void testBootstrapServer() throws IOException, ArgumentParserException { + ArgumentParser parser = ProducerPerformance.argParser(); + String[] args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--bootstrap-server", "localhost:9000"}; + ProducerPerformance.ConfigPostProcessor configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertEquals("localhost:9000", configs.producerProps.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + + args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--command-property", "bootstrap.servers=localhost:9001"}; + configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertEquals("localhost:9001", configs.producerProps.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + + args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--bootstrap-server", "localhost:9000", + "--command-property", "bootstrap.servers=localhost:9001"}; + configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertEquals("localhost:9000", configs.producerProps.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + + args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--producer-props", "bootstrap.servers=localhost:9001"}; + configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertEquals("localhost:9001", configs.producerProps.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + + args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--bootstrap-server", "localhost:9000", + "--producer-props", "bootstrap.servers=localhost:9001"}; + configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertEquals("localhost:9000", configs.producerProps.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); } @Test @@ -482,7 +545,7 @@ public class ProducerPerformanceTest { "--num-records", "5", "--throughput", "100", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; ProducerPerformance.ConfigPostProcessor configs = new ProducerPerformance.ConfigPostProcessor(parser, args); assertFalse(configs.transactionsEnabled); assertNull(configs.transactionDurationMs); @@ -490,14 +553,15 @@ public class ProducerPerformanceTest { } @Test - public void testEnableTransactionByProducerProps() throws IOException, ArgumentParserException { + public void testEnableTransactionByProducerProperty() throws IOException, ArgumentParserException { ArgumentParser parser = ProducerPerformance.argParser(); String[] args = new String[]{ "--topic", "Hello-Kafka", "--num-records", "5", "--throughput", "100", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000", "transactional.id=foobar"}; + "--bootstrap-server", "localhost:9000", + "--command-property", "transactional.id=foobar"}; ProducerPerformance.ConfigPostProcessor configs = new ProducerPerformance.ConfigPostProcessor(parser, args); assertTrue(configs.transactionsEnabled); assertEquals(ProducerPerformance.DEFAULT_TRANSACTION_DURATION_MS, configs.transactionDurationMs); @@ -513,8 +577,8 @@ public class ProducerPerformanceTest { "--num-records", "5", "--throughput", "100", "--record-size", "100", - "--producer.config", producerConfigFile.getAbsolutePath(), - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000", + "--command-config", producerConfigFile.getAbsolutePath()}; ProducerPerformance.ConfigPostProcessor configs = new ProducerPerformance.ConfigPostProcessor(parser, args); assertTrue(configs.transactionsEnabled); assertEquals(ProducerPerformance.DEFAULT_TRANSACTION_DURATION_MS, configs.transactionDurationMs); @@ -525,8 +589,9 @@ public class ProducerPerformanceTest { "--num-records", "5", "--throughput", "100", "--record-size", "100", - "--producer.config", producerConfigFile.getAbsolutePath(), - "--producer-props", "bootstrap.servers=localhost:9000", "transactional.id=hello_kafka"}; + "--bootstrap-server", "localhost:9000", + "--command-config", producerConfigFile.getAbsolutePath(), + "--command-property", "transactional.id=hello_kafka"}; configs = new ProducerPerformance.ConfigPostProcessor(parser, args); assertTrue(configs.transactionsEnabled); assertEquals(ProducerPerformance.DEFAULT_TRANSACTION_DURATION_MS, configs.transactionDurationMs); @@ -538,8 +603,9 @@ public class ProducerPerformanceTest { "--throughput", "100", "--record-size", "100", "--transactional-id", "kafka_hello", - "--producer.config", producerConfigFile.getAbsolutePath(), - "--producer-props", "bootstrap.servers=localhost:9000", "transactional.id=hello_kafka"}; + "--bootstrap-server", "localhost:9000", + "--command-config", producerConfigFile.getAbsolutePath(), + "--command-property", "transactional.id=hello_kafka"}; configs = new ProducerPerformance.ConfigPostProcessor(parser, args); assertTrue(configs.transactionsEnabled); assertEquals(ProducerPerformance.DEFAULT_TRANSACTION_DURATION_MS, configs.transactionDurationMs); @@ -548,6 +614,77 @@ public class ProducerPerformanceTest { Utils.delete(producerConfigFile); } + @Test + public void testEnableTransactionByTransactionIdDeprecated() throws IOException, ArgumentParserException { + File producerConfigFile = createTempFile("transactional.id=foobar"); + ArgumentParser parser = ProducerPerformance.argParser(); + String[] args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--bootstrap-server", "localhost:9000", + "--producer.config", producerConfigFile.getAbsolutePath()}; + ProducerPerformance.ConfigPostProcessor configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertTrue(configs.transactionsEnabled); + assertEquals(ProducerPerformance.DEFAULT_TRANSACTION_DURATION_MS, configs.transactionDurationMs); + assertEquals("foobar", configs.producerProps.get(ProducerConfig.TRANSACTIONAL_ID_CONFIG)); + + args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--bootstrap-server", "localhost:9000", + "--producer.config", producerConfigFile.getAbsolutePath(), + "--producer-props", "transactional.id=hello_kafka"}; + configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertTrue(configs.transactionsEnabled); + assertEquals(ProducerPerformance.DEFAULT_TRANSACTION_DURATION_MS, configs.transactionDurationMs); + assertEquals("hello_kafka", configs.producerProps.get(ProducerConfig.TRANSACTIONAL_ID_CONFIG)); + + args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--transactional-id", "kafka_hello", + "--bootstrap-server", "localhost:9000", + "--producer.config", producerConfigFile.getAbsolutePath(), + "--producer-props", "transactional.id=hello_kafka"}; + configs = new ProducerPerformance.ConfigPostProcessor(parser, args); + assertTrue(configs.transactionsEnabled); + assertEquals(ProducerPerformance.DEFAULT_TRANSACTION_DURATION_MS, configs.transactionDurationMs); + assertEquals("kafka_hello", configs.producerProps.get(ProducerConfig.TRANSACTIONAL_ID_CONFIG)); + + Utils.delete(producerConfigFile); + } + + @Test + public void testEnsureDeprecatedAndModernArgumentsNotBothSpecified() throws IOException { + File producerConfigFile = createTempFile("bootstrap.servers=localhost:9000"); + String[] args = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--producer.config", producerConfigFile.getAbsolutePath(), + "--command-config", producerConfigFile.getAbsolutePath()}; + ArgumentParser parser = ProducerPerformance.argParser(); + assertThrows(ArgumentParserException.class, () -> new ProducerPerformance.ConfigPostProcessor(parser, args)); + + String[] args2 = new String[]{ + "--topic", "Hello-Kafka", + "--num-records", "5", + "--throughput", "100", + "--record-size", "100", + "--producer-props", "bootstrap.servers=localhost:9090", + "--command-property", "bootstrap.servers=localhost:9090"}; + assertThrows(ArgumentParserException.class, () -> new ProducerPerformance.ConfigPostProcessor(parser, args2)); + + Utils.delete(producerConfigFile); + } + @Test public void testEnableTransactionByTransactionDurationMs() throws IOException, ArgumentParserException { ArgumentParser parser = ProducerPerformance.argParser(); @@ -557,7 +694,7 @@ public class ProducerPerformanceTest { "--throughput", "100", "--record-size", "100", "--transaction-duration-ms", "5000", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; ProducerPerformance.ConfigPostProcessor configs = new ProducerPerformance.ConfigPostProcessor(parser, args); assertTrue(configs.transactionsEnabled); assertEquals(5000, configs.transactionDurationMs); @@ -566,28 +703,28 @@ public class ProducerPerformanceTest { } @Test - public void testWarmupRecordsFractionalValue() throws Exception { + public void testWarmupRecordsFractionalValue() { String[] args = new String[] { "--topic", "Hello-Kafka", "--num-records", "10", "--warmup-records", "1.5", "--throughput", "100", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; ArgumentParser parser = ProducerPerformance.argParser(); ArgumentParserException thrown = assertThrows(ArgumentParserException.class, () -> parser.parseArgs(args)); thrown.printStackTrace(); } @Test - public void testWarmupRecordsString() throws Exception { + public void testWarmupRecordsString() { String[] args = new String[] { "--topic", "Hello-Kafka", "--num-records", "10", "--warmup-records", "foo", "--throughput", "100", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; ArgumentParser parser = ProducerPerformance.argParser(); ArgumentParserException thrown = assertThrows(ArgumentParserException.class, () -> parser.parseArgs(args)); thrown.printStackTrace(); @@ -607,7 +744,7 @@ public class ProducerPerformanceTest { "--warmup-records", "2", "--throughput", "1", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; producerPerformanceSpy.start(args); verify(producerMock, times(10)).send(any(), any()); @@ -630,7 +767,7 @@ public class ProducerPerformanceTest { "--warmup-records", "-1", "--throughput", "1", "--record-size", "100", - "--producer-props", "bootstrap.servers=localhost:9000"}; + "--bootstrap-server", "localhost:9000"}; producerPerformanceSpy.start(args); verify(producerMock, times(10)).send(any(), any()); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java index b14d66c652a..9333bbbb65e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java @@ -1373,7 +1373,7 @@ public class ShareGroupCommandTest { } @Test - public void testAlterShareGroupFailureFailureWithNonExistentTopic() { + public void testAlterShareGroupFailureWithNonExistentTopic() { String group = "share-group"; String topic = "none"; String bootstrapServer = "localhost:9092"; diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java index 949b595a115..22e9011a2c2 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java @@ -71,6 +71,7 @@ import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.generate import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.getBrokerMetadata; import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.getReplicaAssignmentForPartitions; import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.getReplicaAssignmentForTopics; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.getReplicaToLogDir; import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.modifyInterBrokerThrottle; import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.modifyLogDirThrottle; import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.modifyTopicThrottles; @@ -436,29 +437,50 @@ public class ReassignPartitionsUnitTest { @Test public void testCurrentPartitionReplicaAssignmentToString() throws Exception { - Map> proposedParts = new HashMap<>(); + try (MockAdminClient adminClient = new MockAdminClient.Builder() + .numBrokers(6) + .brokerLogDirs(List.of( + List.of("/tmp/broker0/logs"), + List.of("/tmp/broker1/logs"), + List.of("/tmp/broker2/logs"), + List.of("/tmp/broker3/logs"), + List.of("/tmp/broker4/logs"), + List.of("/tmp/broker5/logs") + )) + .build() + ) { - proposedParts.put(new TopicPartition("foo", 1), List.of(1, 2, 3)); - proposedParts.put(new TopicPartition("bar", 0), List.of(7, 8, 9)); + List brokers = adminClient.brokers(); + adminClient.addTopic(false, "foo", List.of( + new TopicPartitionInfo(1, brokers.get(1), + List.of(brokers.get(1), brokers.get(2), brokers.get(3)), + List.of(brokers.get(1), brokers.get(2), brokers.get(3))) + ), Map.of()); - Map> currentParts = new HashMap<>(); + adminClient.addTopic(false, "bar", List.of( + new TopicPartitionInfo(0, brokers.get(4), + List.of(brokers.get(4), brokers.get(5)), + List.of(brokers.get(4), brokers.get(5))) + ), Map.of()); - currentParts.put(new TopicPartition("foo", 0), List.of(1, 2, 3)); - currentParts.put(new TopicPartition("foo", 1), List.of(4, 5, 6)); - currentParts.put(new TopicPartition("bar", 0), List.of(7, 8)); - currentParts.put(new TopicPartition("baz", 0), List.of(10, 11, 12)); + Map> proposedParts = new HashMap<>(); + proposedParts.put(new TopicPartition("foo", 1), List.of(0, 1, 2)); + proposedParts.put(new TopicPartition("bar", 0), List.of(3, 4, 5)); - assertEquals(String.join(System.lineSeparator(), - "Current partition replica assignment", - "", - "{\"version\":1,\"partitions\":" + - "[{\"topic\":\"bar\",\"partition\":0,\"replicas\":[7,8],\"log_dirs\":[\"any\",\"any\"]}," + - "{\"topic\":\"foo\",\"partition\":1,\"replicas\":[4,5,6],\"log_dirs\":[\"any\",\"any\",\"any\"]}]" + - "}", - "", - "Save this to use as the --reassignment-json-file option during rollback"), - currentPartitionReplicaAssignmentToString(proposedParts, currentParts) - ); + Map> currentParts = new HashMap<>(); + currentParts.put(new TopicPartition("foo", 1), List.of(1, 2, 3)); + currentParts.put(new TopicPartition("bar", 0), List.of(4, 5)); + + assertEquals(String.join(System.lineSeparator(), + "Current partition replica assignment", + "", + "{\"version\":1,\"partitions\":[{\"topic\":\"bar\",\"partition\":0,\"replicas\":[4,5],\"log_dirs\":[\"/tmp/broker4/logs\",\"/tmp/broker4/logs\"]}," + + "{\"topic\":\"foo\",\"partition\":1,\"replicas\":[1,2,3],\"log_dirs\":[\"any\",\"any\",\"any\"]}]}", + "", + "Save this to use as the --reassignment-json-file option during rollback"), + currentPartitionReplicaAssignmentToString(adminClient, proposedParts, currentParts) + ); + } } @Test @@ -765,4 +787,38 @@ public class ReassignPartitionsUnitTest { assertThrows(AdminOperationException.class, () -> executeAssignment(adminClient, false, "{invalid_json", -1L, -1L, 10000L, Time.SYSTEM, false)).getMessage()); } } + + @Test + public void testGetReplicaToLogDir() throws Exception { + try (MockAdminClient adminClient = new MockAdminClient.Builder() + .numBrokers(4) + .brokerLogDirs(List.of( + List.of("/tmp/broker0/logs0"), + List.of("/tmp/broker1/logs0"), + List.of("/tmp/broker2/logs0"), + List.of("/tmp/broker3/logs0") + )).build() + ) { + addTopics(adminClient); + + Map> topicPartitionToReplicas = Map.of( + new TopicPartition("foo", 0), List.of(0, 1, 2), + new TopicPartition("foo", 1), List.of(1, 2, 3), + new TopicPartition("bar", 0), List.of(2, 3, 0) + ); + + Map result = getReplicaToLogDir(adminClient, topicPartitionToReplicas); + + assertFalse(result.isEmpty()); + assertEquals("/tmp/broker0/logs0", result.get(new TopicPartitionReplica("foo", 0, 0))); + assertEquals("/tmp/broker0/logs0", result.get(new TopicPartitionReplica("foo", 0, 1))); + assertEquals("/tmp/broker0/logs0", result.get(new TopicPartitionReplica("foo", 0, 2))); + assertEquals("/tmp/broker1/logs0", result.get(new TopicPartitionReplica("foo", 1, 1))); + assertEquals("/tmp/broker1/logs0", result.get(new TopicPartitionReplica("foo", 1, 2))); + assertEquals("/tmp/broker1/logs0", result.get(new TopicPartitionReplica("foo", 1, 3))); + assertEquals("/tmp/broker2/logs0", result.get(new TopicPartitionReplica("bar", 0, 0))); + assertEquals("/tmp/broker2/logs0", result.get(new TopicPartitionReplica("bar", 0, 2))); + assertEquals("/tmp/broker2/logs0", result.get(new TopicPartitionReplica("bar", 0, 3))); + } + } } diff --git a/tools/src/test/java/org/apache/kafka/tools/streams/StreamsGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/streams/StreamsGroupCommandTest.java index 6f38c47f15a..4f1e116437e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/streams/StreamsGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/streams/StreamsGroupCommandTest.java @@ -43,6 +43,7 @@ import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.test.TestUtils; @@ -65,6 +66,7 @@ import java.util.stream.IntStream; import joptsimple.OptionException; +import static org.apache.kafka.common.KafkaFuture.completedFuture; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -72,6 +74,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -293,21 +296,30 @@ public class StreamsGroupCommandTest { @Test public void testAdminRequestsForResetOffsets() { Admin adminClient = mock(KafkaAdminClient.class); + String topic = "topic1"; String groupId = "foo-group"; List args = List.of("--bootstrap-server", "localhost:9092", "--group", groupId, "--reset-offsets", "--input-topic", "topic1", "--to-latest"); - List topics = List.of("topic1"); + List topics = List.of(topic); + DescribeTopicsResult describeTopicsResult = mock(DescribeTopicsResult.class); when(adminClient.describeStreamsGroups(List.of(groupId))) .thenReturn(describeStreamsResult(groupId, GroupState.DEAD)); + Map descriptions = Map.of( + topic, new TopicDescription(topic, false, List.of( + new TopicPartitionInfo(0, Node.noNode(), List.of(), List.of())) + )); + when(adminClient.describeTopics(anyCollection())) + .thenReturn(describeTopicsResult); when(adminClient.describeTopics(eq(topics), any(DescribeTopicsOptions.class))) - .thenReturn(describeTopicsResult(topics, 1)); + .thenReturn(describeTopicsResult); + when(describeTopicsResult.allTopicNames()).thenReturn(completedFuture(descriptions)); when(adminClient.listOffsets(any(), any())) .thenReturn(listOffsetsResult()); ListGroupsResult listGroupsResult = listGroupResult(groupId); when(adminClient.listGroups(any(ListGroupsOptions.class))).thenReturn(listGroupsResult); ListStreamsGroupOffsetsResult result = mock(ListStreamsGroupOffsetsResult.class); Map committedOffsetsMap = new HashMap<>(); - committedOffsetsMap.put(new TopicPartition("topic1", 0), mock(OffsetAndMetadata.class)); + committedOffsetsMap.put(new TopicPartition(topic, 0), mock(OffsetAndMetadata.class)); when(adminClient.listStreamsGroupOffsets(ArgumentMatchers.anyMap())).thenReturn(result); when(result.partitionsToOffsetAndMetadata(ArgumentMatchers.anyString())).thenReturn(KafkaFuture.completedFuture(committedOffsetsMap)); @@ -427,6 +439,43 @@ public class StreamsGroupCommandTest { service.close(); } + + @Test + public void testResetOffsetsWithPartitionNotExist() { + Admin adminClient = mock(KafkaAdminClient.class); + String groupId = "foo-group"; + String topic = "topic"; + List args = new ArrayList<>(Arrays.asList("--bootstrap-server", "localhost:9092", "--group", groupId, "--reset-offsets", "--input-topic", "topic:3", "--to-latest")); + + when(adminClient.describeStreamsGroups(List.of(groupId))) + .thenReturn(describeStreamsResult(groupId, GroupState.DEAD)); + DescribeTopicsResult describeTopicsResult = mock(DescribeTopicsResult.class); + + Map descriptions = Map.of( + topic, new TopicDescription(topic, false, List.of( + new TopicPartitionInfo(0, Node.noNode(), List.of(), List.of())) + )); + when(adminClient.describeTopics(anyCollection())) + .thenReturn(describeTopicsResult); + when(adminClient.describeTopics(eq(List.of(topic)), any(DescribeTopicsOptions.class))) + .thenReturn(describeTopicsResult); + when(describeTopicsResult.allTopicNames()).thenReturn(completedFuture(descriptions)); + when(adminClient.listOffsets(any(), any())) + .thenReturn(listOffsetsResult()); + ListStreamsGroupOffsetsResult result = mock(ListStreamsGroupOffsetsResult.class); + Map committedOffsetsMap = Map.of( + new TopicPartition(topic, 0), + new OffsetAndMetadata(12, Optional.of(0), ""), + new TopicPartition(topic, 1), + new OffsetAndMetadata(12, Optional.of(0), "") + ); + + when(adminClient.listStreamsGroupOffsets(ArgumentMatchers.anyMap())).thenReturn(result); + when(result.partitionsToOffsetAndMetadata(ArgumentMatchers.anyString())).thenReturn(KafkaFuture.completedFuture(committedOffsetsMap)); + StreamsGroupCommand.StreamsGroupService service = getStreamsGroupService(args.toArray(new String[0]), adminClient); + assertThrows(UnknownTopicOrPartitionException.class, () -> service.resetOffsets()); + service.close(); + } private ListGroupsResult listGroupResult(String groupId) { ListGroupsResult listGroupsResult = mock(ListGroupsResult.class); diff --git a/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionMetadata.java b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionMetadata.java index 96a92dd01c9..8efeedc3ec4 100644 --- a/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionMetadata.java +++ b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionMetadata.java @@ -117,6 +117,7 @@ public class TransactionMetadata { } } + // VisibleForTesting public void addPartitions(Collection partitions) { topicPartitions.addAll(partitions); } @@ -500,6 +501,7 @@ public class TransactionMetadata { return transactionalId; } + // VisibleForTesting public void setProducerId(long producerId) { this.producerId = producerId; } @@ -507,6 +509,7 @@ public class TransactionMetadata { return producerId; } + // VisibleForTesting public void setPrevProducerId(long prevProducerId) { this.prevProducerId = prevProducerId; } @@ -534,6 +537,7 @@ public class TransactionMetadata { return txnTimeoutMs; } + // VisibleForTesting public void state(TransactionState state) { this.state = state; } @@ -550,6 +554,7 @@ public class TransactionMetadata { return txnStartTimestamp; } + // VisibleForTesting public void txnLastUpdateTimestamp(long txnLastUpdateTimestamp) { this.txnLastUpdateTimestamp = txnLastUpdateTimestamp; } diff --git a/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TxnTransitMetadata.java b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TxnTransitMetadata.java index 452c168687e..b2e78d45da3 100644 --- a/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TxnTransitMetadata.java +++ b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TxnTransitMetadata.java @@ -22,7 +22,7 @@ import org.apache.kafka.server.common.TransactionVersion; import java.util.HashSet; /** - * Immutable object representing the target transition of the transaction metadata + * Represent the target transition of the transaction metadata. The topicPartitions field is mutable. */ public record TxnTransitMetadata( long producerId,