mirror of https://github.com/apache/kafka.git
Merge branch 'apache:trunk' into trunk
This commit is contained in:
commit
9e997f395e
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -576,7 +576,7 @@ public class PlaintextConsumerSubscriptionTest {
|
|||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testAsyncConsumerClassicConsumerSubscribeInvalidTopicCanUnsubscribe() throws InterruptedException {
|
||||
public void testAsyncConsumerSubscribeInvalidTopicCanUnsubscribe() throws InterruptedException {
|
||||
testSubscribeInvalidTopicCanUnsubscribe(GroupProtocol.CONSUMER);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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()) {
|
||||
/**
|
||||
* 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.
|
||||
return acceptedCompressionTypes.get(0);
|
||||
}
|
||||
return CompressionType.NONE;
|
||||
// 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 {
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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": [
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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,8 +708,26 @@ 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);
|
||||
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.
|
||||
|
@ -708,7 +735,6 @@ public class SharePartition {
|
|||
// 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) {
|
||||
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;
|
||||
}
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -239,7 +239,7 @@ object DynamicBrokerConfig {
|
|||
}
|
||||
}
|
||||
val configHandler = new BrokerConfigHandler(config, quotaManagers)
|
||||
configHandler.processConfigChanges("", dynamicPerBrokerConfigs)
|
||||
configHandler.processConfigChanges("", dynamicDefaultConfigs)
|
||||
configHandler.processConfigChanges(config.brokerId.toString, dynamicPerBrokerConfigs)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 }
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
@ -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])
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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
|
||||
----------------------------
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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; \
|
||||
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; \
|
||||
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
|
||||
gpg --batch --verify kafka.tgz.asc kafka.tgz; \
|
||||
fi; \
|
||||
mkdir opt/kafka; \
|
||||
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"; \
|
||||
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
|
||||
|
|
|
@ -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; \
|
||||
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"; \
|
||||
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 ; \
|
||||
fi; \
|
||||
mkdir $KAFKA_DIR; \
|
||||
tar xfz kafka.tgz -C $KAFKA_DIR --strip-components 1; \
|
||||
# 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
|
||||
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
@ -8436,6 +8458,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.
|
||||
*
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
@ -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();
|
||||
|
|
|
@ -25,23 +25,17 @@ 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;
|
||||
|
||||
public ClusterImage(
|
||||
Map<Integer, BrokerRegistration> brokers,
|
||||
Map<Integer, ControllerRegistration> controllers
|
||||
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
@ -59,27 +57,10 @@ public final class DelegationTokenImage {
|
|||
}
|
||||
}
|
||||
|
||||
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();
|
||||
|
|
|
@ -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) {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,30 +21,16 @@ 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().
|
||||
|
@ -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();
|
||||
|
|
|
@ -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,13 +103,13 @@ 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())
|
||||
|
@ -134,27 +132,10 @@ public final class ScramImage {
|
|||
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();
|
||||
|
|
|
@ -26,27 +26,15 @@ 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 record TopicsImage(ImmutableMap<Uuid, TopicImage> topicsById, ImmutableMap<String, TopicImage> topicsByName) {
|
||||
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 TopicsImage including(TopicImage topic) {
|
||||
return new TopicsImage(
|
||||
this.topicsById.updated(topic.id(), topic),
|
||||
|
@ -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() {
|
||||
|
|
|
@ -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 +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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.
|
||||
/**
|
||||
* @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.
|
||||
*/
|
||||
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,
|
||||
public record 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 +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,15 +26,10 @@ 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
|
||||
) {
|
||||
|
@ -45,34 +40,6 @@ public final class ScramCredentialData {
|
|||
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;
|
||||
}
|
||||
|
||||
public UserScramCredentialRecord toRecord(
|
||||
String userName,
|
||||
ScramMechanism mechanism
|
||||
|
|
|
@ -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 +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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) {
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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> {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
||||
```
|
||||
|
|
|
@ -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}")
|
||||
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
||||
|
|
|
@ -89,5 +89,3 @@ def as_json():
|
|||
Export all saved preferences in JSON format.
|
||||
"""
|
||||
json.dumps(prefs, indent=2)
|
||||
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
||||
|
|
|
@ -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("")
|
||||
|
||||
|
||||
|
|
|
@ -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}")
|
||||
|
|
|
@ -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.
|
||||
"""
|
||||
|
||||
|
||||
|
|
|
@ -71,5 +71,3 @@ def replace(path, pattern, replacement, **kwargs):
|
|||
with open(path, "w") as f:
|
||||
for line in updated:
|
||||
f.write(line)
|
||||
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,9 +2046,7 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader {
|
|||
followerThreadPool.close();
|
||||
try {
|
||||
shutdownAndAwaitTermination(remoteStorageReaderThreadPool, "RemoteStorageReaderThreadPool", 10, TimeUnit.SECONDS);
|
||||
} finally {
|
||||
removeMetrics();
|
||||
}
|
||||
|
||||
leaderCopyRLMTasks.clear();
|
||||
leaderExpirationRLMTasks.clear();
|
||||
followerRLMTasks.clear();
|
||||
|
@ -2055,6 +2055,9 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader {
|
|||
Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin");
|
||||
Utils.closeQuietly(remoteStorageManagerPlugin, "remoteStorageManagerPlugin");
|
||||
closed = true;
|
||||
} finally {
|
||||
removeMetrics();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
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);
|
||||
}
|
||||
});
|
||||
|
||||
kafkaStreams.start();
|
||||
|
||||
assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@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,7 +917,8 @@ public class RestoreIntegrationTest {
|
|||
}
|
||||
}
|
||||
|
||||
private void setCommittedOffset(final String topic, final int limitDelta) {
|
||||
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);
|
||||
|
@ -822,7 +926,7 @@ public class RestoreIntegrationTest {
|
|||
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);
|
||||
try (final Consumer<Integer, Integer> consumer = new KafkaConsumer<>(consumerConfig)) {
|
||||
final List<TopicPartition> partitions = asList(
|
||||
new TopicPartition(topic, 0),
|
||||
new TopicPartition(topic, 1));
|
||||
|
@ -836,7 +940,31 @@ public class RestoreIntegrationTest {
|
|||
}
|
||||
|
||||
consumer.commitSync();
|
||||
consumer.close();
|
||||
}
|
||||
} 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void waitForTransitionTo(final Set<KafkaStreams.State> observed, final KafkaStreams.State state, final Duration timeout) throws Exception {
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue