Merge branch 'apache:trunk' into trunk

This commit is contained in:
Abhi Tiwari 2025-08-27 15:24:04 +05:30 committed by GitHub
commit 9e997f395e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
121 changed files with 4052 additions and 2214 deletions

View File

@ -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

View File

@ -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.

View File

@ -576,7 +576,7 @@ public class PlaintextConsumerSubscriptionTest {
}
@ClusterTest
public void testAsyncConsumerClassicConsumerSubscribeInvalidTopicCanUnsubscribe() throws InterruptedException {
public void testAsyncConsumerSubscribeInvalidTopicCanUnsubscribe() throws InterruptedException {
testSubscribeInvalidTopicCanUnsubscribe(GroupProtocol.CONSUMER);
}

View File

@ -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);
}
/**

View File

@ -26,6 +26,20 @@ import java.util.stream.Collectors;
/**
* A class representing an alter configuration entry containing name, value and operation type.
* <p>
* <b>Note for Broker Logger Configuration:</b><br>
* 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.
* <p>
* Example:
* <pre>
* Recommended approach:
* new AlterConfigOp(new ConfigEntry(loggerName, LogLevelConfig.DEBUG_LOG_LEVEL), OpType.SET)
*
* Avoid this:
* new AlterConfigOp(new ConfigEntry(loggerName, "DEBUG"), OpType.SET)
* </pre>
*/
public class AlterConfigOp {

View File

@ -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;
}

View File

@ -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.
* <br/>
* Note: When tiered storage is not enabled, we will return unknown offset.
*/
public static OffsetSpec earliestPendingUpload() {
return new EarliestPendingUploadSpec();
}
}

View File

@ -103,12 +103,17 @@ public final class ListOffsetsHandler extends Batched<TopicPartition, ListOffset
.stream()
.anyMatch(key -> 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);
}

View File

@ -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;

View File

@ -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<CompressionType> 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;
}

View File

@ -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<CompressionType> 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<CompressionType> acceptedCompressionTypes, Set<CompressionType> 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 {

View File

@ -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",

View File

@ -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": [

View File

@ -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,

View File

@ -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<Node> nodes = Collections.singletonList(node);
List<PartitionInfo> 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<String, FeatureUpdate> 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<String, KafkaFuture<TopicDescription>> topicDescriptionMap = result.topicNameValues();
KafkaFuture<TopicDescription> 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);
}
}
}

View File

@ -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());
}
}

View File

@ -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<ClientTelemetryUtils> 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<AbstractRequest.Builder<?>> 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<ClientTelemetryUtils> 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<AbstractRequest.Builder<?>> 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<AbstractRequest.Builder<?>> 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();

View File

@ -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<? super MetricKeyable> selector = ClientTelemetryUtils.getSelectorFromRequestedMetrics(Arrays.asList("metric1", "metric2"));
Predicate<? super MetricKeyable> 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<Byte> 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<Metric> 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();

View File

@ -255,7 +255,7 @@ public abstract class Loggers {
LoggerContext context = (LoggerContext) LogManager.getContext(false);
var results = new HashMap<String, org.apache.logging.log4j.core.Logger>();
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;
}

View File

@ -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<Long, InFlightBatch> 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<Long, InFlightBatch> 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.
* <p>
* 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.
* <p>
* 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<Long, InFlightBatch> 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;
}

View File

@ -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")
}
}

View File

@ -239,7 +239,7 @@ object DynamicBrokerConfig {
}
}
val configHandler = new BrokerConfigHandler(config, quotaManagers)
configHandler.processConfigChanges("", dynamicPerBrokerConfigs)
configHandler.processConfigChanges("", dynamicDefaultConfigs)
configHandler.processConfigChanges(config.brokerId.toString, dynamicPerBrokerConfigs)
}
}

View File

@ -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 }

View File

@ -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

View File

@ -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<KafkaConfig> generateConfigs() {
List<Properties> 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.<SecurityProtocol>empty(),
scala.Option.<File>empty(),
scala.Option.<Properties>empty(),
true,
false,
false,
false,
new HashMap<>(),
1,
false,
1,
(short) 4,
0,
false
)));
List<KafkaConfig> 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<AlterConfigOp> ops = new ArrayList<>();
ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET));
Map<ConfigResource, Collection<AlterConfigOp>> 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<Node> 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<AlterConfigOp> ops = new ArrayList<>();
ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET));
Map<ConfigResource, Collection<AlterConfigOp>> 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<Node> 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<AlterConfigOp> ops = new ArrayList<>();
ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET));
Map<ConfigResource, Collection<AlterConfigOp>> 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<Node> 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<File> 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<AlterConfigOp> ops = new ArrayList<>();
ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET));
Map<ConfigResource, Collection<AlterConfigOp>> 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<Node> 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<Integer> initialReplicaSet = initialReplicas.stream().map(node -> node.id()).collect(Collectors.toSet());
brokers().foreach(broker -> {
if (initialReplicaSet.contains(broker.config().brokerId())) {
Seq<File> 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<Integer, Integer, Boolean> 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);
}
}

File diff suppressed because it is too large Load Diff

View File

@ -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])

View File

@ -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 {

View File

@ -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)
}
}

View File

@ -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=<jvm/native> --kafka-archive=</absolute/path/to/core/build/distributions/kafka_2.13-4.1.0-SNAPSHOT.tgz>
```
Creating a Release Candidate
----------------------------

View File

@ -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:

View File

@ -25,9 +25,11 @@ Usage:
Example command:-
docker_build_test.py <image_name> --image-tag <image_tag> --image-type <image_type> --kafka-url <kafka_url>
docker_build_test.py <image_name> --image-tag <image_tag> --image-type <image_type> --kafka-archive <kafka_archive>
This command will build an image with <image_name> as image name, <image_tag> as image_tag (it will be latest by default),
<image_type> as image type (jvm by default), <kafka_url> for the kafka inside the image and run tests on the image.
<kafka_archive> can be passed as an alternative to <kafka_url> 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)

View File

@ -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

View File

@ -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

View File

@ -4116,11 +4116,9 @@ Feature: metadata.version SupportedMinVersion: 3.3-IV3 SupportedMaxVers
<pre><code class="language-bash">
$ 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.
</code></pre><p>
Note: Currently it is <b>not</b> 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 <a href="#kraft_upgrade">Upgrade</a> section.
<h5 class="anchor-heading"><a id="kraft_reconfig_add" class="anchor-link"></a><a href="#kraft_reconfig_add">Add New Controller</a></h5>
If a dynamic controller cluster already exists, it can be expanded by first provisioning a new controller using the <a href="#kraft_nodes_observers">kafka-storage.sh tool</a> and starting the controller.
@ -4318,7 +4316,7 @@ $ bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:
<p>Try to send messages to the `tieredTopic` topic to roll the log segment:</p>
<pre><code class="language-bash">$ bin/kafka-producer-perf-test.sh --topic tieredTopic --num-records 1200 --record-size 1024 --throughput -1 --producer-props bootstrap.servers=localhost:9092</code></pre>
<pre><code class="language-bash">$ bin/kafka-producer-perf-test.sh --bootstrap-server localhost:9092 --topic tieredTopic --num-records 1200 --record-size 1024 --throughput -1</code></pre>
<p>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:

View File

@ -40,6 +40,10 @@
<li>
The <code>PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG</code> in <code>ProducerConfig</code> was deprecated and will be removed in Kafka 5.0. Please use the <code>PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG</code> instead.
</li>
<li>
The <code>ConsumerPerformance</code> command line tool has a new <code>include</code> option that is alternative to the <code>topic</code> 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.
</li>
</ul>
<h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4>

View File

@ -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",

View File

@ -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<CoordinatorRecord> 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<CoordinatorRecord> 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.

View File

@ -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<StreamsGroupHeartbeatResult, CoordinatorRecord> 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<CoordinatorRecord> 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";

View File

@ -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<String, StreamsGroupMember> 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<String, AssignmentMemberSpec> 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(

View File

@ -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

View File

@ -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) {
}

View File

@ -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) {
}

View File

@ -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();
}

View File

@ -129,14 +129,7 @@ class PartitionReassignmentReplicas {
);
}
static class CompletedReassignment {
final List<Integer> replicas;
final List<Integer> isr;
public CompletedReassignment(List<Integer> replicas, List<Integer> isr) {
this.replicas = replicas;
this.isr = isr;
}
record CompletedReassignment(List<Integer> replicas, List<Integer> isr) {
}
List<Integer> originalReplicas() {

View File

@ -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() {

View File

@ -30,14 +30,12 @@ import java.util.Map.Entry;
/**
* Represents the ACLs in the metadata image.
*
* <p>
* This class is thread-safe.
*/
public final class AclsImage {
public record AclsImage(Map<Uuid, StandardAcl> acls) {
public static final AclsImage EMPTY = new AclsImage(Map.of());
private final Map<Uuid, StandardAcl> acls;
public AclsImage(Map<Uuid, StandardAcl> acls) {
this.acls = Collections.unmodifiableMap(acls);
}
@ -46,10 +44,6 @@ public final class AclsImage {
return acls.isEmpty();
}
public Map<Uuid, StandardAcl> acls() {
return acls;
}
public void write(ImageWriter writer) {
for (Entry<Uuid, StandardAcl> 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();

View File

@ -45,14 +45,12 @@ import static org.apache.kafka.common.requests.DescribeClientQuotasRequest.MATCH
/**
* Represents the client quotas in the metadata image.
*
* <p>
* This class is thread-safe.
*/
public final class ClientQuotasImage {
public record ClientQuotasImage(Map<ClientQuotaEntity, ClientQuotaImage> entities) {
public static final ClientQuotasImage EMPTY = new ClientQuotasImage(Map.of());
private final Map<ClientQuotaEntity, ClientQuotaImage> entities;
public ClientQuotasImage(Map<ClientQuotaEntity, ClientQuotaImage> entities) {
this.entities = Collections.unmodifiableMap(entities);
}
@ -61,11 +59,6 @@ public final class ClientQuotasImage {
return entities.isEmpty();
}
// Visible for testing
public Map<ClientQuotaEntity, ClientQuotaImage> entities() {
return entities;
}
public void write(ImageWriter writer) {
for (Entry<ClientQuotaEntity, ClientQuotaImage> 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();

View File

@ -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.
*
* <p>
* This class is thread-safe.
*/
public final class ClusterImage {
public record ClusterImage(Map<Integer, BrokerRegistration> brokers, Map<Integer, ControllerRegistration> controllers) {
public static final ClusterImage EMPTY = new ClusterImage(
Map.of(),
Map.of());
private final Map<Integer, BrokerRegistration> brokers;
private final Map<Integer, ControllerRegistration> controllers;
Map.of(),
Map.of());
public ClusterImage(
Map<Integer, BrokerRegistration> brokers,
@ -54,18 +48,10 @@ public final class ClusterImage {
return brokers.isEmpty();
}
public Map<Integer, BrokerRegistration> brokers() {
return brokers;
}
public BrokerRegistration broker(int nodeId) {
return brokers.get(nodeId);
}
public Map<Integer, ControllerRegistration> 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();

View File

@ -30,29 +30,10 @@ import java.util.Properties;
/**
* Represents the configuration of a resource.
*
* <p>
* This class is thread-safe.
*/
public final class ConfigurationImage {
private final ConfigResource resource;
private final Map<String, String> data;
public ConfigurationImage(
ConfigResource resource,
Map<String, String> data
) {
this.resource = resource;
this.data = data;
}
public ConfigResource resource() {
return resource;
}
public Map<String, String> data() {
return data;
}
public record ConfigurationImage(ConfigResource resource, Map<String, String> data) {
public boolean isEmpty() {
return data.isEmpty();

View File

@ -31,16 +31,14 @@ import java.util.Map.Entry;
/**
* Represents the DelegationToken credentials in the metadata image.
*
* <p>
* 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<String, DelegationTokenData> 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<String, DelegationTokenData> tokens;
public DelegationTokenImage(Map<String, DelegationTokenData> tokens) {
this.tokens = Collections.unmodifiableMap(tokens);
}
@ -55,31 +53,14 @@ public final class DelegationTokenImage {
List<String> tokenIds = new ArrayList<>(tokens.keySet());
String delegationTokenImageString = "DelegationTokenImage(" + String.join(", ", tokenIds) + ")";
options.handleLoss(delegationTokenImageString);
}
}
}
}
public Map<String, DelegationTokenData> 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();

View File

@ -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) {
}
}

View File

@ -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.
*
* <p>
* 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();

View File

@ -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 +
")";
}
}

View File

@ -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 +
")";
}
}

View File

@ -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.
*
* <p>
* 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();

View File

@ -39,29 +39,27 @@ import java.util.Map.Entry;
/**
* Represents the SCRAM credentials in the metadata image.
*
* <p>
* This class is thread-safe.
*/
public final class ScramImage {
public record ScramImage(Map<ScramMechanism, Map<String, ScramCredentialData>> mechanisms) {
public static final ScramImage EMPTY = new ScramImage(Map.of());
private final Map<ScramMechanism, Map<String, ScramCredentialData>> mechanisms;
public ScramImage(Map<ScramMechanism, Map<String, ScramCredentialData>> 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<ScramMechanism, Map<String, ScramCredentialData>> mechanismEntry : mechanisms.entrySet()) {
for (Entry<String, ScramCredentialData> 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<ScramMechanism, Map<String, ScramCredentialData>> mechanismEntry : mechanisms.entrySet()) {
for (var mechanismEntry : mechanisms.entrySet()) {
if (!mechanismEntry.getValue().isEmpty()) {
scramImageString.append(mechanismEntry.getKey()).append(":");
List<String> 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<String, ScramCredentialData> 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<String, Boolean> user : uniqueUsers.entrySet()) {
for (Entry<String, Boolean> user : uniqueUsers.entrySet()) {
DescribeUserScramCredentialsResult result = new DescribeUserScramCredentialsResult().setUser(user.getKey());
if (!user.getValue()) {
boolean dataFound = false;
List<CredentialInfo> credentialInfos = new ArrayList<>();
for (Map.Entry<ScramMechanism, Map<String, ScramCredentialData>> mechanismsEntry : mechanisms.entrySet()) {
for (var mechanismsEntry : mechanisms.entrySet()) {
Map<String, ScramCredentialData> 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<ScramMechanism, Map<String, ScramCredentialData>> 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();

View File

@ -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.
*
* <p>
* This class is thread-safe.
*/
public final class TopicsImage {
public static final TopicsImage EMPTY = new TopicsImage(ImmutableMap.empty(), ImmutableMap.empty());
private final ImmutableMap<Uuid, TopicImage> topicsById;
private final ImmutableMap<String, TopicImage> topicsByName;
public TopicsImage(
ImmutableMap<Uuid, TopicImage> topicsById,
ImmutableMap<String, TopicImage> topicsByName
) {
this.topicsById = topicsById;
this.topicsByName = topicsByName;
}
public record TopicsImage(ImmutableMap<Uuid, TopicImage> topicsById, ImmutableMap<String, TopicImage> 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<Uuid, TopicImage> topicsById() {
return topicsById;
}
public ImmutableMap<String, TopicImage> 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.
*
* <p>
* Like TopicsImage itself, this map is immutable.
*/
public Map<String, Uuid> topicNameToIdView() {
@ -108,7 +76,7 @@ public final class TopicsImage {
/**
* Expose a view of this TopicsImage as a map from IDs to names.
*
* <p>
* Like TopicsImage itself, this map is immutable.
*/
public Map<Uuid, String> topicIdToNameView() {

View File

@ -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 +
")";
}
}

View File

@ -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<String, Function<MetadataImage, MetadataNode>> 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<String> childNames() {
return CHILDREN.keySet();

View File

@ -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) {
}

View File

@ -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) {
}

View File

@ -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.
*
* <p>
* 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<KafkaPrincipal> 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.

View File

@ -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<String, Short> featureMap;
private final long epoch;
public record FinalizedControllerFeatures(Map<String, Short> featureMap, long epoch) {
public FinalizedControllerFeatures(Map<String, Short> featureMap, long epoch) {
this.featureMap = Collections.unmodifiableMap(featureMap);
this.epoch = epoch;
@ -47,31 +42,4 @@ public class FinalizedControllerFeatures {
public Set<String> featureNames() {
return featureMap.keySet();
}
public Map<String, Short> 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 +
")";
}
}

View File

@ -26,51 +26,18 @@ import java.util.Objects;
/**
* Represents the ACLs in the metadata image.
*
* <p>
* 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

View File

@ -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<StandardAcl> {
public record StandardAcl(ResourceType resourceType, String resourceName, PatternType patternType, String principal,
String host, AclOperation operation,
AclPermissionType permissionType) implements Comparable<StandardAcl> {
public static StandardAcl fromRecord(AccessControlEntryRecord record) {
return new StandardAcl(
ResourceType.fromCode(record.resourceType()),
@ -56,47 +56,6 @@ public final class StandardAcl implements Comparable<StandardAcl> {
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<StandardAcl> {
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<StandardAcl> {
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<StandardAcl> {
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 +
")";
}
}

View File

@ -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 +
")";
}
}

View File

@ -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() {

View File

@ -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) {
}

View File

@ -18,42 +18,14 @@
package org.apache.kafka.metadata.placement;
import java.util.List;
import java.util.Objects;
/**
* The topic assignment.
*
* <p>
* This class is immutable. It's internal state does not change.
*/
public class TopicAssignment {
private final List<PartitionAssignment> assignments;
public TopicAssignment(List<PartitionAssignment> 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<PartitionAssignment> 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<PartitionAssignment> assignments) {
public TopicAssignment {
assignments = List.copyOf(assignments);
}
}

View File

@ -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<String> rack;
private final boolean fenced;
public UsableBroker(int id, Optional<String> rack, boolean fenced) {
this.id = id;
this.rack = rack;
this.fenced = fenced;
}
public int id() {
return id;
}
public Optional<String> 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<String> rack, boolean fenced) {
}

View File

@ -69,22 +69,7 @@ public final class BatchFileReader implements Iterator<BatchFileReader.BatchAndT
}
}
public static class BatchAndType {
private final Batch<ApiMessageAndVersion> batch;
private final boolean isControl;
public BatchAndType(Batch<ApiMessageAndVersion> batch, boolean isControl) {
this.batch = batch;
this.isControl = isControl;
}
public Batch<ApiMessageAndVersion> batch() {
return batch;
}
public boolean isControl() {
return isControl;
}
public record BatchAndType(Batch<ApiMessageAndVersion> batch, boolean isControl) {
}
private final FileRecords fileRecords;

View File

@ -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

View File

@ -64,20 +64,7 @@ public class PeriodicTaskControlManagerTest {
}
}
static class TrackedTask {
final String tag;
final long deadlineNs;
final Supplier<ControllerResult<Void>> op;
TrackedTask(
String tag,
long deadlineNs,
Supplier<ControllerResult<Void>> op
) {
this.tag = tag;
this.deadlineNs = deadlineNs;
this.op = op;
}
record TrackedTask(String tag, long deadlineNs, Supplier<ControllerResult<Void>> op) {
}
static class PeriodicTaskControlManagerTestEnv implements PeriodicTaskControlManager.QueueAccessor {

View File

@ -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));
}
}

View File

@ -106,22 +106,7 @@ public class RecordTestUtils {
}
}
public static class ImageDeltaPair<I, D> {
private final Supplier<I> imageSupplier;
private final Function<I, D> deltaCreator;
public ImageDeltaPair(Supplier<I> imageSupplier, Function<I, D> deltaCreator) {
this.imageSupplier = imageSupplier;
this.deltaCreator = deltaCreator;
}
public Supplier<I> imageSupplier() {
return imageSupplier;
}
public Function<I, D> deltaCreator() {
return deltaCreator;
}
public record ImageDeltaPair<I, D>(Supplier<I> imageSupplier, Function<I, D> deltaCreator) {
}
public static class TestThroughAllIntermediateImagesLeadingToFinalImageHelper<D, I> {

View File

@ -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);

View File

@ -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());
}
}

View File

@ -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.
```

View File

@ -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}")

View File

@ -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()

View File

@ -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)

View File

@ -89,5 +89,3 @@ def as_json():
Export all saved preferences in JSON format.
"""
json.dumps(prefs, indent=2)

View File

@ -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))

View File

@ -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("")

View File

@ -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}")

View File

@ -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 <ORDINAL> candidate for release of Apache Kafka {release_version}.
<DESCRIPTION OF MAJOR CHANGES, INCLUDE INDICATION OF MAJOR/MINOR RELEASE>
@ -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}/<BUILD NUMBER>/
-- 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}/<BUILD_NUMBER>/report.html
Unit/integration tests: https://github.com/apache/kafka/actions/runs/<RUN_NUMBER>
System tests:
<Confluent engineers can access the semaphore build to provide the build number
https://confluent-open-source-kafka-system-test-results.s3-us-west-2.amazonaws.com/{dev_branch}/<BUILD_NUMBER>/report.html>
<USE docker/README.md FOR STEPS TO RUN DOCKER BUILD TEST GITHUB ACTIONS>
* Successful Docker Image Github Actions Pipeline for {dev_branch} branch:
Docker Build Test Pipeline (JVM): https://github.com/apache/kafka/actions/runs/<RUN_NUMBER>
Docker Build Test Pipeline (Native): https://github.com/apache/kafka/actions/runs/<RUN_NUMBER>
/**************************************
Thanks,
<YOU>
"""
@ -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.
"""

View File

@ -71,5 +71,3 @@ def replace(path, pattern, replacement, **kwargs):
with open(path, "w") as f:
for line in updated:
f.write(line)

View File

@ -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;

View File

@ -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 {

View File

@ -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);

View File

@ -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<String> createMockOptionSpec(String name) {
OptionSpec<String> 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<String> opt1 = createMockOptionSpec("--first-option");
OptionSpec<String> opt2 = createMockOptionSpec("--second-option");
OptionSpec<String> 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<String> opt1 = createMockOptionSpec("--first-option");
OptionSpec<String> opt2 = createMockOptionSpec("--second-option");
OptionSpec<String> 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<String> opt1 = createMockOptionSpec("--first-option");
OptionSpec<String> opt2 = createMockOptionSpec("--second-option");
OptionSpec<String> 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();
}
}
}

View File

@ -24,6 +24,9 @@ import java.util.List;
* The class abstracts the acknowledgement request for <code>SharePartition</code> class constructed
* from {@link org.apache.kafka.common.message.ShareFetchRequestData.AcknowledgementBatch} and
* {@link org.apache.kafka.common.message.ShareAcknowledgeRequestData.AcknowledgementBatch} classes.
* <p>
* 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,

View File

@ -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<AlterConfigOp> ops = new ArrayList<>();
ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET));
Map<ConfigResource, Collection<AlterConfigOp>> 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<Node> 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<AlterConfigOp> ops = new ArrayList<>();
ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET));
Map<ConfigResource, Collection<AlterConfigOp>> 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<Node> 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<AlterConfigOp> ops = new ArrayList<>();
ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET));
Map<ConfigResource, Collection<AlterConfigOp>> 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<Node> 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<File> 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<AlterConfigOp> ops = new ArrayList<>();
ops.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), AlterConfigOp.OpType.SET));
Map<ConfigResource, Collection<AlterConfigOp>> 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<Node> 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<Integer> initialReplicaSet = initialReplicas.stream().map(node -> node.id()).collect(Collectors.toSet());
clusterInstance.brokers().forEach((id, broker) -> {
if (initialReplicaSet.contains(id)) {
List<File> 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<Integer, Integer, Boolean> 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)
);
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}
}

View File

@ -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<LogSegment> segments = logSegments();
@ -1709,6 +1711,31 @@ public class UnifiedLog implements AutoCloseable {
});
}
private OffsetResultHolder fetchEarliestPendingUploadOffset(Optional<AsyncOffsetReader> 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<Integer> 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.

View File

@ -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));
}
}

View File

@ -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<Integer, Integer> stream = builder.stream(inputStream);
@ -414,23 +487,26 @@ public class RestoreIntegrationTest {
Integer::sum,
Materialized.<Integer, Integer, KeyValueStore<Bytes, byte[]>>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<String, Object> extraConfiguration) {
final Map<String, Object> 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<Integer, Integer> consumer = new KafkaConsumer<>(consumerConfig);
final List<TopicPartition> partitions = asList(
new TopicPartition(topic, 0),
new TopicPartition(topic, 1));
try (final Consumer<Integer, Integer> consumer = new KafkaConsumer<>(consumerConfig)) {
final List<TopicPartition> 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<TopicPartition> partitions = asList(
new TopicPartition(topic, 0),
new TopicPartition(topic, 1));
final Map<TopicPartition, OffsetSpec> offsetSpecs = partitions.stream()
.collect(Collectors.toMap(tp -> tp, tp -> OffsetSpec.latest()));
final Map<TopicPartition, ListOffsetsResult.ListOffsetsResultInfo> endOffsets =
admin.listOffsets(offsetSpecs).all().get();
final Map<TopicPartition, OffsetAndMetadata> 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<KafkaStreams.State> observed, final KafkaStreams.State state, final Duration timeout) throws Exception {

View File

@ -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:

Some files were not shown because too many files have changed in this diff Show More