mirror of https://github.com/apache/kafka.git
Merge branch 'apache:trunk' into KAFKA-16768
This commit is contained in:
commit
86dda289c9
12
.asf.yaml
12
.asf.yaml
|
@ -29,16 +29,16 @@ notifications:
|
|||
# Read more here: https://github.com/apache/infrastructure-asfyaml
|
||||
github:
|
||||
collaborators:
|
||||
- brandboat
|
||||
- FrankYang0529
|
||||
- gongxuanzhang
|
||||
- m1a2st
|
||||
- mingyen066
|
||||
- ShivsundarR
|
||||
- smjn
|
||||
- TaiJuWu
|
||||
- xijiu
|
||||
- brandboat
|
||||
- Yunyung
|
||||
- xijiu
|
||||
- chirag-wadhwa5
|
||||
- mingyen066
|
||||
- ShivsundarR
|
||||
- Rancho-7
|
||||
enabled_merge_buttons:
|
||||
squash: true
|
||||
squash_commit_message: PR_TITLE_AND_DESC
|
||||
|
|
|
@ -83,6 +83,9 @@ runs:
|
|||
RUN_FLAKY_TESTS: ${{ inputs.run-flaky-tests }}
|
||||
TEST_XML_OUTPUT_DIR: ${{ inputs.test-xml-output }}
|
||||
TEST_VERBOSE: ${{ inputs.test-verbose }}
|
||||
# This build step is invoked by build.yml to run junit tests only,
|
||||
# Spotbugs is being run by that workflow via the "check" task and does not need to also be run here,
|
||||
# since that is redundant.
|
||||
run: |
|
||||
set +e
|
||||
./.github/scripts/thread-dump.sh &
|
||||
|
@ -97,6 +100,8 @@ runs:
|
|||
-Pkafka.cluster.test.repeat=$TEST_REPEAT \
|
||||
-Pkafka.test.verbose=$TEST_VERBOSE \
|
||||
-PcommitId=xxxxxxxxxxxxxxxx \
|
||||
-x spotbugsMain \
|
||||
-x spotbugsTest \
|
||||
$TEST_TASK
|
||||
exitcode="$?"
|
||||
echo "exitcode=$exitcode" >> $GITHUB_OUTPUT
|
||||
|
|
|
@ -127,7 +127,7 @@ jobs:
|
|||
- name: Setup Gradle
|
||||
uses: ./.github/actions/setup-gradle
|
||||
with:
|
||||
java-version: 23
|
||||
java-version: 24
|
||||
gradle-cache-read-only: ${{ !inputs.is-trunk }}
|
||||
gradle-cache-write-only: ${{ inputs.is-trunk }}
|
||||
develocity-access-key: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
|
||||
|
@ -181,7 +181,7 @@ jobs:
|
|||
fail-fast: false
|
||||
matrix:
|
||||
# If we change these, make sure to adjust ci-complete.yml
|
||||
java: [ 23, 17 ]
|
||||
java: [ 24, 17 ]
|
||||
run-flaky: [ true, false ]
|
||||
run-new: [ true, false ]
|
||||
exclude:
|
||||
|
@ -270,7 +270,7 @@ jobs:
|
|||
python .github/scripts/junit.py \
|
||||
--path build/junit-xml >> $GITHUB_STEP_SUMMARY
|
||||
|
||||
# This job downloads all the JUnit XML files and thread dumps from the JDK 23 test runs.
|
||||
# This job downloads all the JUnit XML files and thread dumps from the JDK 24 test runs.
|
||||
# If any test job fails, we will not run this job. Also, if any thread dump artifacts
|
||||
# are present, this means there was a timeout in the tests and so we will not proceed
|
||||
# with catalog creation.
|
||||
|
@ -288,7 +288,7 @@ jobs:
|
|||
- name: Download Thread Dumps
|
||||
uses: actions/download-artifact@v4
|
||||
with:
|
||||
pattern: junit-thread-dumps-23-*
|
||||
pattern: junit-thread-dumps-24-*
|
||||
path: thread-dumps
|
||||
merge-multiple: true
|
||||
- name: Check For Thread Dump
|
||||
|
@ -302,7 +302,7 @@ jobs:
|
|||
- name: Download JUnit XMLs
|
||||
uses: actions/download-artifact@v4
|
||||
with:
|
||||
pattern: junit-xml-23-* # Only look at JDK 23 tests for the test catalog
|
||||
pattern: junit-xml-24-* # Only look at JDK 24 tests for the test catalog
|
||||
path: junit-xml
|
||||
merge-multiple: true
|
||||
- name: Collate Test Catalog
|
||||
|
|
|
@ -44,7 +44,7 @@ jobs:
|
|||
fail-fast: false
|
||||
matrix:
|
||||
# Make sure these match build.yml
|
||||
java: [ 23, 17 ]
|
||||
java: [ 24, 17 ]
|
||||
run-flaky: [ true, false ]
|
||||
run-new: [ true, false ]
|
||||
exclude:
|
||||
|
|
|
@ -13,7 +13,7 @@
|
|||
|
||||
You need to have [Java](http://www.oracle.com/technetwork/java/javase/downloads/index.html) installed.
|
||||
|
||||
We build and test Apache Kafka with 17 and 23. The `release` parameter in javac is set to `11` for the clients
|
||||
We build and test Apache Kafka with 17 and 24. The `release` parameter in javac is set to `11` for the clients
|
||||
and streams modules, and `17` for the rest, ensuring compatibility with their respective
|
||||
minimum Java versions. Similarly, the `release` parameter in scalac is set to `11` for the streams modules and `17`
|
||||
for the rest.
|
||||
|
|
|
@ -37,7 +37,7 @@ plugins {
|
|||
id 'org.nosphere.apache.rat' version "0.8.1"
|
||||
id "io.swagger.core.v3.swagger-gradle-plugin" version "${swaggerVersion}"
|
||||
|
||||
id "com.github.spotbugs" version '6.0.25' apply false
|
||||
id "com.github.spotbugs" version '6.2.3' apply false
|
||||
id 'org.scoverage' version '8.0.3' apply false
|
||||
id 'com.gradleup.shadow' version '8.3.6' apply false
|
||||
id 'com.diffplug.spotless' version "6.25.0"
|
||||
|
|
|
@ -928,6 +928,9 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
|
|||
}
|
||||
|
||||
private void stopFindCoordinatorOnClose() {
|
||||
if (applicationEventHandler == null) {
|
||||
return;
|
||||
}
|
||||
log.debug("Stop finding coordinator during consumer close");
|
||||
applicationEventHandler.add(new StopFindCoordinatorOnCloseEvent());
|
||||
}
|
||||
|
@ -944,6 +947,10 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
|
|||
* 2. leave the group
|
||||
*/
|
||||
private void sendAcknowledgementsAndLeaveGroup(final Timer timer, final AtomicReference<Throwable> firstException) {
|
||||
if (applicationEventHandler == null || backgroundEventProcessor == null ||
|
||||
backgroundEventReaper == null || backgroundEventQueue == null) {
|
||||
return;
|
||||
}
|
||||
completeQuietly(
|
||||
() -> applicationEventHandler.addAndGet(new ShareAcknowledgeOnCloseEvent(acknowledgementsToSend(), calculateDeadlineMs(timer))),
|
||||
"Failed to send pending acknowledgements with a timeout(ms)=" + timer.timeoutMs(), firstException);
|
||||
|
@ -1035,6 +1042,9 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
|
|||
* If the acknowledgement commit callback throws an exception, this method will throw an exception.
|
||||
*/
|
||||
private void handleCompletedAcknowledgements(boolean onClose) {
|
||||
if (backgroundEventQueue == null || backgroundEventReaper == null || backgroundEventProcessor == null) {
|
||||
return;
|
||||
}
|
||||
// If the user gets any fatal errors, they will get these exceptions in the background queue.
|
||||
// While closing, we ignore these exceptions so that the consumers close successfully.
|
||||
processBackgroundEvents(onClose ? e -> (e instanceof GroupAuthorizationException
|
||||
|
|
|
@ -418,7 +418,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
|
|||
this.transactionManager = configureTransactionState(config, logContext);
|
||||
// There is no need to do work required for adaptive partitioning, if we use a custom partitioner.
|
||||
boolean enableAdaptivePartitioning = partitionerPlugin.get() == null &&
|
||||
config.getBoolean(ProducerConfig.PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG);
|
||||
config.getBoolean(ProducerConfig.PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG);
|
||||
RecordAccumulator.PartitionerConfig partitionerConfig = new RecordAccumulator.PartitionerConfig(
|
||||
enableAdaptivePartitioning,
|
||||
config.getLong(ProducerConfig.PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG)
|
||||
|
|
|
@ -101,8 +101,10 @@ public class ProducerConfig extends AbstractConfig {
|
|||
+ "similar or lower producer latency despite the increased linger.";
|
||||
|
||||
/** <code>partitioner.adaptive.partitioning.enable</code> */
|
||||
public static final String PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG = "partitioner.adaptive.partitioning.enable";
|
||||
private static final String PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC =
|
||||
public static final String PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG = "partitioner.adaptive.partitioning.enable";
|
||||
@Deprecated
|
||||
public static final String PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG = PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG;
|
||||
private static final String PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_DOC =
|
||||
"When set to 'true', the producer will try to adapt to broker performance and produce more messages to partitions hosted on faster brokers. "
|
||||
+ "If 'false', the producer will try to distribute messages uniformly. Note: this setting has no effect if a custom partitioner is used.";
|
||||
|
||||
|
@ -111,7 +113,7 @@ public class ProducerConfig extends AbstractConfig {
|
|||
private static final String PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC =
|
||||
"If a broker cannot process produce requests from a partition for <code>" + PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG + "</code> time, "
|
||||
+ "the partitioner treats that partition as not available. If the value is 0, this logic is disabled. "
|
||||
+ "Note: this setting has no effect if a custom partitioner is used or <code>" + PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG
|
||||
+ "Note: this setting has no effect if a custom partitioner is used or <code>" + PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG
|
||||
+ "</code> is set to 'false'.";
|
||||
|
||||
/** <code>partitioner.ignore.keys</code> */
|
||||
|
@ -392,7 +394,7 @@ public class ProducerConfig extends AbstractConfig {
|
|||
.define(COMPRESSION_LZ4_LEVEL_CONFIG, Type.INT, CompressionType.LZ4.defaultLevel(), CompressionType.LZ4.levelValidator(), Importance.MEDIUM, COMPRESSION_LZ4_LEVEL_DOC)
|
||||
.define(COMPRESSION_ZSTD_LEVEL_CONFIG, Type.INT, CompressionType.ZSTD.defaultLevel(), CompressionType.ZSTD.levelValidator(), Importance.MEDIUM, COMPRESSION_ZSTD_LEVEL_DOC)
|
||||
.define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC)
|
||||
.define(PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG, Type.BOOLEAN, true, Importance.LOW, PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC)
|
||||
.define(PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG, Type.BOOLEAN, true, Importance.LOW, PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_DOC)
|
||||
.define(PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG, Type.LONG, 0, atLeast(0), Importance.LOW, PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC)
|
||||
.define(PARTITIONER_IGNORE_KEYS_CONFIG, Type.BOOLEAN, false, Importance.MEDIUM, PARTITIONER_IGNORE_KEYS_DOC)
|
||||
.define(LINGER_MS_CONFIG, Type.LONG, 5, atLeast(0), Importance.MEDIUM, LINGER_MS_DOC)
|
||||
|
|
|
@ -181,7 +181,8 @@ public class TopicConfig {
|
|||
"When used together, <code>min.insync.replicas</code> and <code>acks</code> allow you to enforce greater durability guarantees. " +
|
||||
"A typical scenario would be to create a topic with a replication factor of 3, " +
|
||||
"set <code>min.insync.replicas</code> to 2, and produce with <code>acks</code> of \"all\". " +
|
||||
"This will ensure that a majority of replicas must persist a write before it's considered successful by the producer and it's visible to consumers.";
|
||||
"This ensures that a majority of replicas must persist a write before it's considered successful by the producer and it's visible to consumers." +
|
||||
"<p>Note that when the Eligible Leader Replicas feature is enabled, the semantics of this config changes. Please refer to <a href=\"#eligible_leader_replicas\">the ELR section</a> for more info.</p>";
|
||||
|
||||
public static final String COMPRESSION_TYPE_CONFIG = "compression.type";
|
||||
public static final String COMPRESSION_TYPE_DOC = "Specify the final compression type for a given topic. " +
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.kafka.common.errors.WakeupException;
|
|||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||
import org.apache.kafka.common.utils.LogCaptureAppender;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
|
@ -77,6 +78,7 @@ import static java.util.Collections.singleton;
|
|||
import static java.util.Collections.singletonList;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
@ -207,11 +209,19 @@ public class ShareConsumerImplTest {
|
|||
props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id");
|
||||
props.put(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, "an.invalid.class");
|
||||
final ConsumerConfig config = new ConsumerConfig(props);
|
||||
|
||||
LogCaptureAppender appender = LogCaptureAppender.createAndRegister();
|
||||
KafkaException ce = assertThrows(
|
||||
KafkaException.class,
|
||||
() -> newConsumer(config));
|
||||
assertTrue(ce.getMessage().contains("Failed to construct Kafka share consumer"), "Unexpected exception message: " + ce.getMessage());
|
||||
assertTrue(ce.getCause().getMessage().contains("Class an.invalid.class cannot be found"), "Unexpected cause: " + ce.getCause());
|
||||
|
||||
boolean npeLogged = appender.getEvents().stream()
|
||||
.flatMap(event -> event.getThrowableInfo().stream())
|
||||
.anyMatch(str -> str.contains("NullPointerException"));
|
||||
|
||||
assertFalse(npeLogged, "Unexpected NullPointerException during consumer construction");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -75,8 +75,8 @@ log.dirs=/tmp/kraft-broker-logs
|
|||
num.partitions=1
|
||||
|
||||
# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
|
||||
# This value is recommended to be increased for installations with data dirs located in RAID array.
|
||||
num.recovery.threads.per.data.dir=1
|
||||
# This value is recommended to be increased based on the installation resources.
|
||||
num.recovery.threads.per.data.dir=2
|
||||
|
||||
############################# Internal Topic Settings #############################
|
||||
# The replication factor for the group metadata internal topics "__consumer_offsets" and "__transaction_state"
|
||||
|
|
|
@ -75,8 +75,8 @@ log.dirs=/tmp/kraft-controller-logs
|
|||
num.partitions=1
|
||||
|
||||
# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
|
||||
# This value is recommended to be increased for installations with data dirs located in RAID array.
|
||||
num.recovery.threads.per.data.dir=1
|
||||
# This value is recommended to be increased based on the installation resources.
|
||||
num.recovery.threads.per.data.dir=2
|
||||
|
||||
############################# Internal Topic Settings #############################
|
||||
# The replication factor for the group metadata internal topics "__consumer_offsets" and "__transaction_state"
|
||||
|
|
|
@ -44,7 +44,7 @@ Configuration:
|
|||
# State Change appender
|
||||
- name: StateChangeAppender
|
||||
fileName: "${sys:kafka.logs.dir}/state-change.log"
|
||||
filePattern: "${sys:kafka.logs.dir}/stage-change.log.%d{yyyy-MM-dd-HH}"
|
||||
filePattern: "${sys:kafka.logs.dir}/state-change.log.%d{yyyy-MM-dd-HH}"
|
||||
PatternLayout:
|
||||
pattern: "${logPattern}"
|
||||
TimeBasedTriggeringPolicy:
|
||||
|
|
|
@ -78,8 +78,8 @@ log.dirs=/tmp/kraft-combined-logs
|
|||
num.partitions=1
|
||||
|
||||
# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
|
||||
# This value is recommended to be increased for installations with data dirs located in RAID array.
|
||||
num.recovery.threads.per.data.dir=1
|
||||
# This value is recommended to be increased based on the installation resources.
|
||||
num.recovery.threads.per.data.dir=2
|
||||
|
||||
############################# Internal Topic Settings #############################
|
||||
# The replication factor for the group metadata internal topics "__consumer_offsets", "__share_group_state" and "__transaction_state"
|
||||
|
|
|
@ -224,6 +224,7 @@ public class JsonConverter implements Converter, HeaderConverter, Versioned {
|
|||
private JsonConverterConfig config;
|
||||
private Cache<Schema, ObjectNode> fromConnectSchemaCache;
|
||||
private Cache<JsonNode, Schema> toConnectSchemaCache;
|
||||
private Schema schema = null; // if a schema is provided in config, this schema will be used for all messages
|
||||
|
||||
private final JsonSerializer serializer;
|
||||
private final JsonDeserializer deserializer;
|
||||
|
@ -286,6 +287,16 @@ public class JsonConverter implements Converter, HeaderConverter, Versioned {
|
|||
|
||||
fromConnectSchemaCache = new SynchronizedCache<>(new LRUCache<>(config.schemaCacheSize()));
|
||||
toConnectSchemaCache = new SynchronizedCache<>(new LRUCache<>(config.schemaCacheSize()));
|
||||
|
||||
try {
|
||||
final byte[] schemaContent = config.schemaContent();
|
||||
if (schemaContent != null) {
|
||||
final JsonNode schemaNode = deserializer.deserialize("", schemaContent);
|
||||
this.schema = asConnectSchema(schemaNode);
|
||||
}
|
||||
} catch (SerializationException e) {
|
||||
throw new DataException("Failed to parse schema in converter config due to serialization error: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -340,13 +351,16 @@ public class JsonConverter implements Converter, HeaderConverter, Versioned {
|
|||
throw new DataException("Converting byte[] to Kafka Connect data failed due to serialization error: ", e);
|
||||
}
|
||||
|
||||
if (config.schemasEnabled() && (!jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !jsonValue.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)))
|
||||
throw new DataException("JsonConverter with schemas.enable requires \"schema\" and \"payload\" fields and may not contain additional fields." +
|
||||
if (config.schemasEnabled()) {
|
||||
if (schema != null) {
|
||||
return new SchemaAndValue(schema, convertToConnect(schema, jsonValue, config));
|
||||
} else if (!jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !jsonValue.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)) {
|
||||
throw new DataException("JsonConverter with schemas.enable requires \"schema\" and \"payload\" fields and may not contain additional fields." +
|
||||
" If you are trying to deserialize plain JSON data, set schemas.enable=false in your converter configuration.");
|
||||
|
||||
// The deserialized data should either be an envelope object containing the schema and the payload or the schema
|
||||
// was stripped during serialization and we need to fill in an all-encompassing schema.
|
||||
if (!config.schemasEnabled()) {
|
||||
}
|
||||
} else {
|
||||
// The deserialized data should either be an envelope object containing the schema and the payload or the schema
|
||||
// was stripped during serialization and we need to fill in an all-encompassing schema.
|
||||
ObjectNode envelope = JSON_NODE_FACTORY.objectNode();
|
||||
envelope.set(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME, null);
|
||||
envelope.set(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME, jsonValue);
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.kafka.common.config.ConfigDef.Type;
|
|||
import org.apache.kafka.common.config.ConfigDef.Width;
|
||||
import org.apache.kafka.connect.storage.ConverterConfig;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -35,6 +36,11 @@ public final class JsonConverterConfig extends ConverterConfig {
|
|||
private static final String SCHEMAS_ENABLE_DOC = "Include schemas within each of the serialized values and keys.";
|
||||
private static final String SCHEMAS_ENABLE_DISPLAY = "Enable Schemas";
|
||||
|
||||
public static final String SCHEMA_CONTENT_CONFIG = "schema.content";
|
||||
public static final String SCHEMA_CONTENT_DEFAULT = null;
|
||||
private static final String SCHEMA_CONTENT_DOC = "When set, this is used as the schema for all messages. Otherwise, the schema will be included in the content of each message.";
|
||||
private static final String SCHEMA_CONTENT_DISPLAY = "Schema Content";
|
||||
|
||||
public static final String SCHEMAS_CACHE_SIZE_CONFIG = "schemas.cache.size";
|
||||
public static final int SCHEMAS_CACHE_SIZE_DEFAULT = 1000;
|
||||
private static final String SCHEMAS_CACHE_SIZE_DOC = "The maximum number of schemas that can be cached in this converter instance.";
|
||||
|
@ -61,6 +67,8 @@ public final class JsonConverterConfig extends ConverterConfig {
|
|||
orderInGroup++, Width.MEDIUM, SCHEMAS_ENABLE_DISPLAY);
|
||||
CONFIG.define(SCHEMAS_CACHE_SIZE_CONFIG, Type.INT, SCHEMAS_CACHE_SIZE_DEFAULT, Importance.HIGH, SCHEMAS_CACHE_SIZE_DOC, group,
|
||||
orderInGroup++, Width.MEDIUM, SCHEMAS_CACHE_SIZE_DISPLAY);
|
||||
CONFIG.define(SCHEMA_CONTENT_CONFIG, Type.STRING, SCHEMA_CONTENT_DEFAULT, Importance.HIGH, SCHEMA_CONTENT_DOC, group,
|
||||
orderInGroup++, Width.MEDIUM, SCHEMA_CONTENT_DISPLAY);
|
||||
|
||||
group = "Serialization";
|
||||
orderInGroup = 0;
|
||||
|
@ -86,6 +94,7 @@ public final class JsonConverterConfig extends ConverterConfig {
|
|||
private final int schemaCacheSize;
|
||||
private final DecimalFormat decimalFormat;
|
||||
private final boolean replaceNullWithDefault;
|
||||
private final byte[] schemaContent;
|
||||
|
||||
public JsonConverterConfig(Map<String, ?> props) {
|
||||
super(CONFIG, props);
|
||||
|
@ -93,6 +102,10 @@ public final class JsonConverterConfig extends ConverterConfig {
|
|||
this.schemaCacheSize = getInt(SCHEMAS_CACHE_SIZE_CONFIG);
|
||||
this.decimalFormat = DecimalFormat.valueOf(getString(DECIMAL_FORMAT_CONFIG).toUpperCase(Locale.ROOT));
|
||||
this.replaceNullWithDefault = getBoolean(REPLACE_NULL_WITH_DEFAULT_CONFIG);
|
||||
String schemaContentStr = getString(SCHEMA_CONTENT_CONFIG);
|
||||
this.schemaContent = (schemaContentStr == null || schemaContentStr.isEmpty())
|
||||
? null
|
||||
: schemaContentStr.getBytes(StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -130,4 +143,15 @@ public final class JsonConverterConfig extends ConverterConfig {
|
|||
return replaceNullWithDefault;
|
||||
}
|
||||
|
||||
/**
|
||||
* If a default schema is provided in the converter config, this will be
|
||||
* used for all messages.
|
||||
*
|
||||
* This is only relevant if schemas are enabled.
|
||||
*
|
||||
* @return Schema Contents, will return null if no value is provided
|
||||
*/
|
||||
public byte[] schemaContent() {
|
||||
return schemaContent;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,6 +36,8 @@ import com.fasterxml.jackson.databind.node.JsonNodeFactory;
|
|||
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -977,6 +979,58 @@ public class JsonConverterTest {
|
|||
assertEquals(AppInfoParser.getVersion(), converter.version());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaContentIsNull() {
|
||||
Map<String, Object> config = new HashMap<>();
|
||||
config.put(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, null);
|
||||
converter.configure(config, false);
|
||||
byte[] jsonBytes = "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes();
|
||||
SchemaAndValue result = converter.toConnectData(TOPIC, jsonBytes);
|
||||
assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), result);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaContentIsEmptyString() {
|
||||
converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, ""), false);
|
||||
assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaContentValidSchema() {
|
||||
converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"type\": \"string\" }"), false);
|
||||
assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectData(TOPIC, "\"foo-bar-baz\"".getBytes()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaContentInValidSchema() {
|
||||
assertThrows(
|
||||
DataException.class,
|
||||
() -> converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"string\" }"), false),
|
||||
" Provided schema is invalid , please recheck the schema you have provided");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaContentLooksLikeSchema() {
|
||||
converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"type\": \"struct\", \"fields\": [{\"field\": \"schema\", \"type\": \"struct\",\"fields\": [{\"field\": \"type\", \"type\": \"string\" }]}, {\"field\": \"payload\", \"type\": \"string\"}]}"), false);
|
||||
SchemaAndValue connectData = converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes());
|
||||
assertEquals("foo-bar-baz", ((Struct) connectData.value()).getString("payload"));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {
|
||||
"{ }",
|
||||
"{ \"wrong\": \"schema\" }",
|
||||
"{ \"schema\": { \"type\": \"string\" } }",
|
||||
"{ \"payload\": \"foo-bar-baz\" }",
|
||||
"{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\", \"extra\": \"field\" }",
|
||||
})
|
||||
public void testNullSchemaContentWithWrongConnectDataValue(String value) {
|
||||
converter.configure(Map.of(), false);
|
||||
assertThrows(
|
||||
DataException.class,
|
||||
() -> converter.toConnectData(TOPIC, value.getBytes()));
|
||||
}
|
||||
|
||||
private JsonNode parse(byte[] json) {
|
||||
try {
|
||||
return objectMapper.readTree(json);
|
||||
|
|
|
@ -51,61 +51,14 @@ public class QuotaFactory {
|
|||
}
|
||||
};
|
||||
|
||||
public static class QuotaManagers {
|
||||
private final ClientQuotaManager fetch;
|
||||
private final ClientQuotaManager produce;
|
||||
private final ClientRequestQuotaManager request;
|
||||
private final ControllerMutationQuotaManager controllerMutation;
|
||||
private final ReplicationQuotaManager leader;
|
||||
private final ReplicationQuotaManager follower;
|
||||
private final ReplicationQuotaManager alterLogDirs;
|
||||
private final Optional<Plugin<ClientQuotaCallback>> clientQuotaCallbackPlugin;
|
||||
|
||||
public QuotaManagers(ClientQuotaManager fetch, ClientQuotaManager produce, ClientRequestQuotaManager request,
|
||||
ControllerMutationQuotaManager controllerMutation, ReplicationQuotaManager leader,
|
||||
ReplicationQuotaManager follower, ReplicationQuotaManager alterLogDirs,
|
||||
Optional<Plugin<ClientQuotaCallback>> clientQuotaCallbackPlugin) {
|
||||
this.fetch = fetch;
|
||||
this.produce = produce;
|
||||
this.request = request;
|
||||
this.controllerMutation = controllerMutation;
|
||||
this.leader = leader;
|
||||
this.follower = follower;
|
||||
this.alterLogDirs = alterLogDirs;
|
||||
this.clientQuotaCallbackPlugin = clientQuotaCallbackPlugin;
|
||||
}
|
||||
|
||||
public ClientQuotaManager fetch() {
|
||||
return fetch;
|
||||
}
|
||||
|
||||
public ClientQuotaManager produce() {
|
||||
return produce;
|
||||
}
|
||||
|
||||
public ClientRequestQuotaManager request() {
|
||||
return request;
|
||||
}
|
||||
|
||||
public ControllerMutationQuotaManager controllerMutation() {
|
||||
return controllerMutation;
|
||||
}
|
||||
|
||||
public ReplicationQuotaManager leader() {
|
||||
return leader;
|
||||
}
|
||||
|
||||
public ReplicationQuotaManager follower() {
|
||||
return follower;
|
||||
}
|
||||
|
||||
public ReplicationQuotaManager alterLogDirs() {
|
||||
return alterLogDirs;
|
||||
}
|
||||
|
||||
public Optional<Plugin<ClientQuotaCallback>> clientQuotaCallbackPlugin() {
|
||||
return clientQuotaCallbackPlugin;
|
||||
}
|
||||
public record QuotaManagers(ClientQuotaManager fetch,
|
||||
ClientQuotaManager produce,
|
||||
ClientRequestQuotaManager request,
|
||||
ControllerMutationQuotaManager controllerMutation,
|
||||
ReplicationQuotaManager leader,
|
||||
ReplicationQuotaManager follower,
|
||||
ReplicationQuotaManager alterLogDirs,
|
||||
Optional<Plugin<ClientQuotaCallback>> clientQuotaCallbackPlugin) {
|
||||
|
||||
public void shutdown() {
|
||||
fetch.shutdown();
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.kafka.common.message.ShareFetchResponseData;
|
|||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.requests.FetchRequest;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.raft.errors.NotLeaderException;
|
||||
import org.apache.kafka.server.LogReadResult;
|
||||
import org.apache.kafka.server.metrics.KafkaMetricsGroup;
|
||||
import org.apache.kafka.server.purgatory.DelayedOperation;
|
||||
|
@ -368,6 +369,14 @@ public class DelayedShareFetch extends DelayedOperation {
|
|||
"topic partitions {}", shareFetch.groupId(), shareFetch.memberId(),
|
||||
sharePartitions.keySet());
|
||||
}
|
||||
// At this point, there could be delayed requests sitting in the purgatory which are waiting on
|
||||
// DelayedShareFetchPartitionKeys corresponding to partitions, whose leader has been changed to a different broker.
|
||||
// In that case, such partitions would not be able to get acquired, and the tryComplete will keep on returning false.
|
||||
// Eventually the operation will get timed out and completed, but it might not get removed from the purgatory.
|
||||
// This has been eventually left it like this because the purging mechanism will trigger whenever the number of completed
|
||||
// but still being watched operations is larger than the purge interval. This purge interval is defined by the config
|
||||
// share.fetch.purgatory.purge.interval.requests and is 1000 by default, thereby ensuring that such stale operations do not
|
||||
// grow indefinitely.
|
||||
return false;
|
||||
} catch (Exception e) {
|
||||
log.error("Error processing delayed share fetch request", e);
|
||||
|
@ -757,7 +766,8 @@ public class DelayedShareFetch extends DelayedOperation {
|
|||
* Case a: The partition is in an offline log directory on this broker
|
||||
* Case b: This broker does not know the partition it tries to fetch
|
||||
* Case c: This broker is no longer the leader of the partition it tries to fetch
|
||||
* Case d: All remote storage read requests completed
|
||||
* Case d: This broker is no longer the leader or follower of the partition it tries to fetch
|
||||
* Case e: All remote storage read requests completed
|
||||
* @return boolean representing whether the remote fetch is completed or not.
|
||||
*/
|
||||
private boolean maybeCompletePendingRemoteFetch() {
|
||||
|
@ -765,14 +775,20 @@ public class DelayedShareFetch extends DelayedOperation {
|
|||
|
||||
for (TopicIdPartition topicIdPartition : pendingRemoteFetchesOpt.get().fetchOffsetMetadataMap().keySet()) {
|
||||
try {
|
||||
replicaManager.getPartitionOrException(topicIdPartition.topicPartition());
|
||||
Partition partition = replicaManager.getPartitionOrException(topicIdPartition.topicPartition());
|
||||
if (!partition.isLeader()) {
|
||||
throw new NotLeaderException("Broker is no longer the leader of topicPartition: " + topicIdPartition);
|
||||
}
|
||||
} catch (KafkaStorageException e) { // Case a
|
||||
log.debug("TopicPartition {} is in an offline log directory, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams());
|
||||
canComplete = true;
|
||||
} catch (UnknownTopicOrPartitionException e) { // Case b
|
||||
log.debug("Broker no longer knows of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams());
|
||||
canComplete = true;
|
||||
} catch (NotLeaderOrFollowerException e) { // Case c
|
||||
} catch (NotLeaderException e) { // Case c
|
||||
log.debug("Broker is no longer the leader of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams());
|
||||
canComplete = true;
|
||||
} catch (NotLeaderOrFollowerException e) { // Case d
|
||||
log.debug("Broker is no longer the leader or follower of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams());
|
||||
canComplete = true;
|
||||
}
|
||||
|
@ -780,7 +796,7 @@ public class DelayedShareFetch extends DelayedOperation {
|
|||
break;
|
||||
}
|
||||
|
||||
if (canComplete || pendingRemoteFetchesOpt.get().isDone()) { // Case d
|
||||
if (canComplete || pendingRemoteFetchesOpt.get().isDone()) { // Case e
|
||||
return forceComplete();
|
||||
} else
|
||||
return false;
|
||||
|
|
|
@ -803,7 +803,7 @@ public class SharePartition {
|
|||
}
|
||||
|
||||
InFlightState updateResult = inFlightBatch.tryUpdateBatchState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE, maxDeliveryCount, memberId);
|
||||
if (updateResult == null) {
|
||||
if (updateResult == null || updateResult.state() != RecordState.ACQUIRED) {
|
||||
log.info("Unable to acquire records for the batch: {} in share partition: {}-{}",
|
||||
inFlightBatch, groupId, topicIdPartition);
|
||||
continue;
|
||||
|
@ -1009,12 +1009,7 @@ public class SharePartition {
|
|||
updatedStates.add(updateResult);
|
||||
stateBatches.add(new PersisterStateBatch(offsetState.getKey(), offsetState.getKey(),
|
||||
updateResult.state().id(), (short) updateResult.deliveryCount()));
|
||||
|
||||
// If the maxDeliveryCount limit has been exceeded, the record will be transitioned to ARCHIVED state.
|
||||
// This should not change the next fetch offset because the record is not available for acquisition
|
||||
if (updateResult.state() != RecordState.ARCHIVED) {
|
||||
updateFindNextFetchOffset(true);
|
||||
}
|
||||
// Do not update the next fetch offset as the offset has not completed the transition yet.
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
|
@ -1054,12 +1049,7 @@ public class SharePartition {
|
|||
updatedStates.add(updateResult);
|
||||
stateBatches.add(new PersisterStateBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset(),
|
||||
updateResult.state().id(), (short) updateResult.deliveryCount()));
|
||||
|
||||
// If the maxDeliveryCount limit has been exceeded, the record will be transitioned to ARCHIVED state.
|
||||
// This should not change the next fetch offset because the record is not available for acquisition
|
||||
if (updateResult.state() != RecordState.ARCHIVED) {
|
||||
updateFindNextFetchOffset(true);
|
||||
}
|
||||
// Do not update the next fetch offset as the batch has not completed the transition yet.
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
@ -1641,7 +1631,7 @@ public class SharePartition {
|
|||
|
||||
InFlightState updateResult = offsetState.getValue().tryUpdateState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE,
|
||||
maxDeliveryCount, memberId);
|
||||
if (updateResult == null) {
|
||||
if (updateResult == null || updateResult.state() != RecordState.ACQUIRED) {
|
||||
log.trace("Unable to acquire records for the offset: {} in batch: {}"
|
||||
+ " for the share partition: {}-{}", offsetState.getKey(), inFlightBatch,
|
||||
groupId, topicIdPartition);
|
||||
|
@ -1941,12 +1931,7 @@ public class SharePartition {
|
|||
updatedStates.add(updateResult);
|
||||
stateBatches.add(new PersisterStateBatch(offsetState.getKey(), offsetState.getKey(),
|
||||
updateResult.state().id(), (short) updateResult.deliveryCount()));
|
||||
// If the maxDeliveryCount limit has been exceeded, the record will be transitioned to ARCHIVED state.
|
||||
// This should not change the next fetch offset because the record is not available for acquisition
|
||||
if (recordState == RecordState.AVAILABLE
|
||||
&& updateResult.state() != RecordState.ARCHIVED) {
|
||||
updateFindNextFetchOffset(true);
|
||||
}
|
||||
// Do not update the nextFetchOffset as the offset has not completed the transition yet.
|
||||
}
|
||||
} finally {
|
||||
lock.writeLock().unlock();
|
||||
|
@ -1996,13 +1981,7 @@ public class SharePartition {
|
|||
stateBatches.add(
|
||||
new PersisterStateBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset(),
|
||||
updateResult.state().id(), (short) updateResult.deliveryCount()));
|
||||
|
||||
// If the maxDeliveryCount limit has been exceeded, the record will be transitioned to ARCHIVED state.
|
||||
// This should not change the nextFetchOffset because the record is not available for acquisition
|
||||
if (recordState == RecordState.AVAILABLE
|
||||
&& updateResult.state() != RecordState.ARCHIVED) {
|
||||
updateFindNextFetchOffset(true);
|
||||
}
|
||||
// Do not update the next fetch offset as the batch has not completed the transition yet.
|
||||
} finally {
|
||||
lock.writeLock().unlock();
|
||||
}
|
||||
|
@ -2445,22 +2424,22 @@ public class SharePartition {
|
|||
releaseAcquisitionLockOnTimeoutForPerOffsetBatch(inFlightBatch, stateBatches, memberId, firstOffset, lastOffset);
|
||||
}
|
||||
}
|
||||
|
||||
if (!stateBatches.isEmpty()) {
|
||||
writeShareGroupState(stateBatches).whenComplete((result, exception) -> {
|
||||
if (exception != null) {
|
||||
log.debug("Failed to write the share group state on acquisition lock timeout for share partition: {}-{} memberId: {}",
|
||||
groupId, topicIdPartition, memberId, exception);
|
||||
}
|
||||
// Even if write share group state RPC call fails, we will still go ahead with the state transition.
|
||||
// Update the cached state and start and end offsets after releasing the acquisition lock on timeout.
|
||||
maybeUpdateCachedStateAndOffsets();
|
||||
});
|
||||
}
|
||||
} finally {
|
||||
lock.writeLock().unlock();
|
||||
}
|
||||
|
||||
if (!stateBatches.isEmpty()) {
|
||||
writeShareGroupState(stateBatches).whenComplete((result, exception) -> {
|
||||
if (exception != null) {
|
||||
log.debug("Failed to write the share group state on acquisition lock timeout for share partition: {}-{} memberId: {}",
|
||||
groupId, topicIdPartition, memberId, exception);
|
||||
}
|
||||
// Even if write share group state RPC call fails, we will still go ahead with the state transition.
|
||||
// Update the cached state and start and end offsets after releasing the acquisition lock on timeout.
|
||||
maybeUpdateCachedStateAndOffsets();
|
||||
});
|
||||
}
|
||||
|
||||
// If we have an acquisition lock timeout for a share-partition, then we should check if
|
||||
// there is a pending share fetch request for the share-partition and complete it.
|
||||
// Skip null check for stateBatches, it should always be initialized if reached here.
|
||||
|
|
|
@ -774,6 +774,7 @@ public class SharePartitionManager implements AutoCloseable {
|
|||
if (sharePartition != null) {
|
||||
sharePartition.markFenced();
|
||||
replicaManager.removeListener(sharePartitionKey.topicIdPartition().topicPartition(), sharePartition.listener());
|
||||
replicaManager.completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(sharePartitionKey.groupId(), sharePartitionKey.topicIdPartition()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData
|
|||
import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicConfig, CreatableTopicConfigCollection}
|
||||
import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic
|
||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.requests.{CreateTopicsRequest, RequestContext, RequestHeader}
|
||||
import org.apache.kafka.common.requests.{CreateTopicsRequest, CreateTopicsResponse, RequestContext, RequestHeader}
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinator
|
||||
import org.apache.kafka.coordinator.share.ShareCoordinator
|
||||
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
|
||||
|
@ -138,6 +138,19 @@ class DefaultAutoTopicCreationManager(
|
|||
} else if (response.versionMismatch() != null) {
|
||||
warn(s"Auto topic creation failed for ${creatableTopics.keys} with invalid version exception")
|
||||
} else {
|
||||
if (response.hasResponse) {
|
||||
response.responseBody() match {
|
||||
case createTopicsResponse: CreateTopicsResponse =>
|
||||
createTopicsResponse.data().topics().forEach(topicResult => {
|
||||
val error = Errors.forCode(topicResult.errorCode)
|
||||
if (error != Errors.NONE) {
|
||||
warn(s"Auto topic creation failed for ${topicResult.name} with error '${error.name}': ${topicResult.errorMessage}")
|
||||
}
|
||||
})
|
||||
case other =>
|
||||
warn(s"Auto topic creation request received unexpected response type: ${other.getClass.getSimpleName}")
|
||||
}
|
||||
}
|
||||
debug(s"Auto topic creation completed for ${creatableTopics.keys} with response ${response.responseBody}.")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -570,6 +570,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
|
|||
s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} not found in ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG} (an explicit security mapping for each controller listener is required if ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG} is non-empty, or if there are security protocols other than PLAINTEXT in use)")
|
||||
}
|
||||
}
|
||||
// controller.quorum.auto.join.enable must be false for KRaft broker-only
|
||||
require(!quorumConfig.autoJoin,
|
||||
s"${QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG} is only supported when ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the 'controller' role.")
|
||||
// warn that only the first controller listener is used if there is more than one
|
||||
if (controllerListenerNames.size > 1) {
|
||||
warn(s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} has multiple entries; only the first will be used since ${KRaftConfigs.PROCESS_ROLES_CONFIG}=broker: ${controllerListenerNames}")
|
||||
|
|
|
@ -638,7 +638,7 @@ object DumpLogSegments {
|
|||
" Instead, the value-decoder-class option can be used if a custom RLMM implementation is configured.")
|
||||
private val shareStateOpt = parser.accepts("share-group-state-decoder", "If set, log data will be parsed as share group state data from the " +
|
||||
"__share_group_state topic.")
|
||||
private val skipRecordMetadataOpt = parser.accepts("skip-record-metadata", "Whether to skip printing metadata for each record.")
|
||||
private val skipRecordMetadataOpt = parser.accepts("skip-record-metadata", "Skip metadata when printing records. This flag also skips control records.")
|
||||
options = parser.parse(args : _*)
|
||||
|
||||
def messageParser: MessageParser[_, _] =
|
||||
|
|
|
@ -24,11 +24,14 @@ import org.apache.kafka.clients.admin.RaftVoterEndpoint;
|
|||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.test.KafkaClusterTestKit;
|
||||
import org.apache.kafka.common.test.TestKitNodes;
|
||||
import org.apache.kafka.common.test.api.TestKitDefaults;
|
||||
import org.apache.kafka.raft.QuorumConfig;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
@ -164,4 +167,69 @@ public class ReconfigurableQuorumIntegrationTest {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testControllersAutoJoinStandaloneVoter() throws Exception {
|
||||
final var nodes = new TestKitNodes.Builder().
|
||||
setNumBrokerNodes(1).
|
||||
setNumControllerNodes(3).
|
||||
setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()).
|
||||
build();
|
||||
try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder(nodes).
|
||||
setConfigProp(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, true).
|
||||
setStandalone(true).
|
||||
build()
|
||||
) {
|
||||
cluster.format();
|
||||
cluster.startup();
|
||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
||||
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
||||
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
||||
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
|
||||
for (int replicaId : new int[] {3000, 3001, 3002}) {
|
||||
assertEquals(nodes.controllerNodes().get(replicaId).metadataDirectoryId(), voters.get(replicaId));
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNewVoterAutoRemovesAndAdds() throws Exception {
|
||||
final var nodes = new TestKitNodes.Builder().
|
||||
setNumBrokerNodes(1).
|
||||
setNumControllerNodes(3).
|
||||
setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()).
|
||||
build();
|
||||
|
||||
// Configure the initial voters with one voter having a different directory ID.
|
||||
// This simulates the case where the controller failed and is brought back up with a different directory ID.
|
||||
final Map<Integer, Uuid> initialVoters = new HashMap<>();
|
||||
final var oldDirectoryId = Uuid.randomUuid();
|
||||
for (final var controllerNode : nodes.controllerNodes().values()) {
|
||||
initialVoters.put(
|
||||
controllerNode.id(),
|
||||
controllerNode.id() == TestKitDefaults.CONTROLLER_ID_OFFSET ?
|
||||
oldDirectoryId : controllerNode.metadataDirectoryId()
|
||||
);
|
||||
}
|
||||
|
||||
try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder(nodes).
|
||||
setConfigProp(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, true).
|
||||
setInitialVoterSet(initialVoters).
|
||||
build()
|
||||
) {
|
||||
cluster.format();
|
||||
cluster.startup();
|
||||
try (Admin admin = Admin.create(cluster.clientProperties())) {
|
||||
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
|
||||
Map<Integer, Uuid> voters = findVoterDirs(admin);
|
||||
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
|
||||
for (int replicaId : new int[] {3000, 3001, 3002}) {
|
||||
assertEquals(nodes.controllerNodes().get(replicaId).metadataDirectoryId(), voters.get(replicaId));
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -153,6 +153,16 @@ public class DelayedShareFetchTest {
|
|||
when(sp0.canAcquireRecords()).thenReturn(false);
|
||||
when(sp1.canAcquireRecords()).thenReturn(false);
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p1 = mock(Partition.class);
|
||||
when(p1.isLeader()).thenReturn(true);
|
||||
|
||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1);
|
||||
|
||||
ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(new MockTime());
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
|
@ -160,6 +170,7 @@ public class DelayedShareFetchTest {
|
|||
.withSharePartitions(sharePartitions)
|
||||
.withShareGroupMetrics(shareGroupMetrics)
|
||||
.withFetchId(fetchId)
|
||||
.withReplicaManager(replicaManager)
|
||||
.build());
|
||||
|
||||
when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true);
|
||||
|
@ -218,6 +229,15 @@ public class DelayedShareFetchTest {
|
|||
|
||||
PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Set.of(tp0));
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p1 = mock(Partition.class);
|
||||
when(p1.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1);
|
||||
|
||||
Time time = mock(Time.class);
|
||||
when(time.hiResClockMs()).thenReturn(100L).thenReturn(110L);
|
||||
ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time);
|
||||
|
@ -287,6 +307,15 @@ public class DelayedShareFetchTest {
|
|||
mockTopicIdPartitionFetchBytes(replicaManager, tp0, hwmOffsetMetadata);
|
||||
BiConsumer<SharePartitionKey, Throwable> exceptionHandler = mockExceptionHandler();
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p1 = mock(Partition.class);
|
||||
when(p1.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1);
|
||||
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
|
@ -580,6 +609,19 @@ public class DelayedShareFetchTest {
|
|||
List<DelayedOperationKey> delayedShareFetchWatchKeys = new ArrayList<>();
|
||||
topicIdPartitions1.forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition())));
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p1 = mock(Partition.class);
|
||||
when(p1.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p2 = mock(Partition.class);
|
||||
when(p2.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1);
|
||||
when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenReturn(p2);
|
||||
|
||||
Uuid fetchId1 = Uuid.randomUuid();
|
||||
DelayedShareFetch delayedShareFetch1 = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch1)
|
||||
|
@ -737,6 +779,12 @@ public class DelayedShareFetchTest {
|
|||
when(time.hiResClockMs()).thenReturn(100L).thenReturn(110L).thenReturn(170L);
|
||||
ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time);
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
.withSharePartitions(sharePartitions)
|
||||
|
@ -881,10 +929,18 @@ public class DelayedShareFetchTest {
|
|||
BROKER_TOPIC_STATS);
|
||||
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
|
||||
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
.withSharePartitions(sharePartitions)
|
||||
.withFetchId(fetchId)
|
||||
.withReplicaManager(replicaManager)
|
||||
.build();
|
||||
|
||||
when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true);
|
||||
|
@ -1263,6 +1319,19 @@ public class DelayedShareFetchTest {
|
|||
when(remoteLogManager.asyncRead(any(), any())).thenReturn(mock(Future.class));
|
||||
when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager));
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p1 = mock(Partition.class);
|
||||
when(p1.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p2 = mock(Partition.class);
|
||||
when(p2.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1);
|
||||
when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenReturn(p2);
|
||||
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
|
@ -1288,6 +1357,70 @@ public class DelayedShareFetchTest {
|
|||
delayedShareFetch.lock().unlock();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoteStorageFetchPartitionLeaderChanged() {
|
||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
||||
TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
|
||||
|
||||
SharePartition sp0 = mock(SharePartition.class);
|
||||
|
||||
when(sp0.canAcquireRecords()).thenReturn(true);
|
||||
|
||||
LinkedHashMap<TopicIdPartition, SharePartition> sharePartitions = new LinkedHashMap<>();
|
||||
sharePartitions.put(tp0, sp0);
|
||||
|
||||
ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(),
|
||||
new CompletableFuture<>(), List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS,
|
||||
BROKER_TOPIC_STATS);
|
||||
|
||||
when(sp0.nextFetchOffset()).thenReturn(10L);
|
||||
|
||||
// Fetch offset does not match with the cached entry for sp0, hence, a replica manager fetch will happen for sp0.
|
||||
when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty());
|
||||
|
||||
// Mocking remote storage read result for tp0.
|
||||
doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(), Set.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||
|
||||
// Remote fetch related mocks. Remote fetch object does not complete within tryComplete in this mock.
|
||||
RemoteLogManager remoteLogManager = mock(RemoteLogManager.class);
|
||||
when(remoteLogManager.asyncRead(any(), any())).thenReturn(mock(Future.class));
|
||||
when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager));
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(false);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
.withSharePartitions(sharePartitions)
|
||||
.withReplicaManager(replicaManager)
|
||||
.withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0)))
|
||||
.withFetchId(fetchId)
|
||||
.build());
|
||||
|
||||
// All the topic partitions are acquirable.
|
||||
when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true);
|
||||
|
||||
// Mock the behaviour of replica manager such that remote storage fetch completion timer task completes on adding it to the watch queue.
|
||||
doAnswer(invocationOnMock -> {
|
||||
TimerTask timerTask = invocationOnMock.getArgument(0);
|
||||
timerTask.run();
|
||||
return null;
|
||||
}).when(replicaManager).addShareFetchTimerRequest(any());
|
||||
|
||||
assertFalse(delayedShareFetch.isCompleted());
|
||||
assertTrue(delayedShareFetch.tryComplete());
|
||||
assertTrue(delayedShareFetch.isCompleted());
|
||||
// Remote fetch object gets created for delayed share fetch object.
|
||||
assertNotNull(delayedShareFetch.pendingRemoteFetches());
|
||||
// Verify the locks are released for local log read topic partitions tp0.
|
||||
Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0));
|
||||
assertTrue(delayedShareFetch.lock().tryLock());
|
||||
delayedShareFetch.lock().unlock();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoteStorageFetchTryCompleteThrowsException() {
|
||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
||||
|
@ -1516,6 +1649,16 @@ public class DelayedShareFetchTest {
|
|||
when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager));
|
||||
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p1 = mock(Partition.class);
|
||||
when(p1.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1);
|
||||
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
.withSharePartitions(sharePartitions)
|
||||
|
@ -1586,6 +1729,12 @@ public class DelayedShareFetchTest {
|
|||
when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager));
|
||||
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
.withSharePartitions(sharePartitions)
|
||||
|
@ -1679,6 +1828,19 @@ public class DelayedShareFetchTest {
|
|||
}).when(remoteLogManager).asyncRead(any(), any());
|
||||
when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager));
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p1 = mock(Partition.class);
|
||||
when(p1.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p2 = mock(Partition.class);
|
||||
when(p2.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1);
|
||||
when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenReturn(p2);
|
||||
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
|
@ -1761,6 +1923,16 @@ public class DelayedShareFetchTest {
|
|||
when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager));
|
||||
|
||||
Uuid fetchId = Uuid.randomUuid();
|
||||
|
||||
Partition p0 = mock(Partition.class);
|
||||
when(p0.isLeader()).thenReturn(true);
|
||||
|
||||
Partition p1 = mock(Partition.class);
|
||||
when(p1.isLeader()).thenReturn(true);
|
||||
|
||||
when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0);
|
||||
when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1);
|
||||
|
||||
DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder()
|
||||
.withShareFetchData(shareFetch)
|
||||
.withSharePartitions(sharePartitions)
|
||||
|
|
|
@ -2622,7 +2622,8 @@ public class SharePartitionManagerTest {
|
|||
assertEquals(Errors.FENCED_STATE_EPOCH.code(), partitionDataMap.get(tp2).errorCode());
|
||||
assertEquals("Fenced state epoch", partitionDataMap.get(tp2).errorMessage());
|
||||
|
||||
Mockito.verify(replicaManager, times(0)).completeDelayedShareFetchRequest(any());
|
||||
Mockito.verify(replicaManager, times(1)).completeDelayedShareFetchRequest(
|
||||
new DelayedShareFetchGroupKey(groupId, tp2));
|
||||
Mockito.verify(replicaManager, times(1)).readFromLog(
|
||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||
// Should have 1 fetch recorded and 1 failure as single topic has multiple partition fetch
|
||||
|
|
|
@ -7878,6 +7878,105 @@ public class SharePartitionTest {
|
|||
assertNull(sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedException {
|
||||
Persister persister = Mockito.mock(Persister.class);
|
||||
SharePartition sharePartition = SharePartitionBuilder.builder()
|
||||
.withState(SharePartitionState.ACTIVE)
|
||||
.withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS)
|
||||
.withMaxDeliveryCount(2)
|
||||
.withPersister(persister)
|
||||
.build();
|
||||
|
||||
fetchAcquiredRecords(sharePartition, memoryRecords(5, 2), 5);
|
||||
fetchAcquiredRecords(sharePartition, memoryRecords(5, 7), 5);
|
||||
|
||||
// Futures which will be completed later, so the batch state has ongoing transition.
|
||||
CompletableFuture<WriteShareGroupStateResult> future1 = new CompletableFuture<>();
|
||||
CompletableFuture<WriteShareGroupStateResult> future2 = new CompletableFuture<>();
|
||||
Mockito.when(persister.writeState(Mockito.any())).thenReturn(future1).thenReturn(future2);
|
||||
|
||||
// Acknowledge batches.
|
||||
sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(3, 3, List.of(AcknowledgeType.ACCEPT.id))));
|
||||
sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(7, 11, List.of(AcknowledgeType.ACCEPT.id))));
|
||||
|
||||
assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(2L).offsetState().get(3L).state());
|
||||
assertEquals(1, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount());
|
||||
assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState());
|
||||
assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount());
|
||||
|
||||
WriteShareGroupStateResult writeShareGroupStateResult = Mockito.mock(WriteShareGroupStateResult.class);
|
||||
Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of(
|
||||
new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of(
|
||||
PartitionFactory.newPartitionErrorData(0, Errors.GROUP_ID_NOT_FOUND.code(), Errors.GROUP_ID_NOT_FOUND.message())))));
|
||||
|
||||
future1.complete(writeShareGroupStateResult);
|
||||
assertEquals(12, sharePartition.nextFetchOffset());
|
||||
assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(2L).offsetState().get(3L).state());
|
||||
assertEquals(1, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount());
|
||||
assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState());
|
||||
assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount());
|
||||
|
||||
future2.complete(writeShareGroupStateResult);
|
||||
assertEquals(12L, sharePartition.nextFetchOffset());
|
||||
assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(2L).offsetState().get(3L).state());
|
||||
assertEquals(1, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount());
|
||||
assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(7L).batchState());
|
||||
assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount());
|
||||
|
||||
// Allowing acquisition lock to expire. This will also ensure that acquisition lock timeout task
|
||||
// is run successfully post write state RPC failure.
|
||||
mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS);
|
||||
TestUtils.waitForCondition(
|
||||
() -> sharePartition.cachedState().get(2L).offsetState().get(3L).state() == RecordState.AVAILABLE &&
|
||||
sharePartition.cachedState().get(7L).batchState() == RecordState.AVAILABLE &&
|
||||
sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount() == 1 &&
|
||||
sharePartition.cachedState().get(7L).batchDeliveryCount() == 1 &&
|
||||
sharePartition.timer().size() == 0,
|
||||
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
|
||||
() -> assertionFailedMessage(sharePartition, Map.of(2L, List.of(3L), 7L, List.of())));
|
||||
// Acquisition lock timeout task has run already and next fetch offset is moved to 2.
|
||||
assertEquals(2, sharePartition.nextFetchOffset());
|
||||
// Send the same batches again.
|
||||
fetchAcquiredRecords(sharePartition, memoryRecords(5, 2), 5);
|
||||
fetchAcquiredRecords(sharePartition, memoryRecords(5, 7), 5);
|
||||
|
||||
future1 = new CompletableFuture<>();
|
||||
future2 = new CompletableFuture<>();
|
||||
Mockito.when(persister.writeState(Mockito.any())).thenReturn(future1).thenReturn(future2);
|
||||
|
||||
sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(3, 3, List.of(AcknowledgeType.ACCEPT.id))));
|
||||
sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(7, 11, List.of(AcknowledgeType.ACCEPT.id))));
|
||||
|
||||
mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS);
|
||||
// Verify the timer tasks have run and the state is archived for the offsets which are not acknowledged,
|
||||
// but the acquisition lock timeout task should be just expired for acknowledged offsets, though
|
||||
// the state should not be archived.
|
||||
TestUtils.waitForCondition(
|
||||
() -> sharePartition.cachedState().get(2L).offsetState().get(2L).state() == RecordState.ARCHIVED &&
|
||||
sharePartition.cachedState().get(2L).offsetState().get(3L).state() == RecordState.ACKNOWLEDGED &&
|
||||
sharePartition.cachedState().get(2L).offsetState().get(3L).acquisitionLockTimeoutTask().hasExpired() &&
|
||||
sharePartition.cachedState().get(7L).batchState() == RecordState.ACKNOWLEDGED &&
|
||||
sharePartition.cachedState().get(7L).batchAcquisitionLockTimeoutTask().hasExpired(),
|
||||
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
|
||||
() -> assertionFailedMessage(sharePartition, Map.of(2L, List.of(3L), 7L, List.of())));
|
||||
|
||||
future1.complete(writeShareGroupStateResult);
|
||||
// Now the state should be archived for the offsets despite the write state RPC failure, as the
|
||||
// delivery count has reached the max delivery count and the acquisition lock timeout task
|
||||
// has already expired for the offsets which were acknowledged.
|
||||
assertEquals(12, sharePartition.nextFetchOffset());
|
||||
assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(2L).offsetState().get(3L).state());
|
||||
assertEquals(2, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount());
|
||||
assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState());
|
||||
assertEquals(2, sharePartition.cachedState().get(7L).batchDeliveryCount());
|
||||
|
||||
future2.complete(writeShareGroupStateResult);
|
||||
assertEquals(12L, sharePartition.nextFetchOffset());
|
||||
assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(7L).batchState());
|
||||
assertEquals(2, sharePartition.cachedState().get(7L).batchDeliveryCount());
|
||||
}
|
||||
|
||||
/**
|
||||
* This function produces transactional data of a given no. of records followed by a transactional marker (COMMIT/ABORT).
|
||||
*/
|
||||
|
|
|
@ -1489,6 +1489,18 @@ class KafkaConfigTest {
|
|||
assertEquals(expected, addresses)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testInvalidQuorumAutoJoinForKRaftBroker(): Unit = {
|
||||
val props = TestUtils.createBrokerConfig(0)
|
||||
props.setProperty(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, String.valueOf(true))
|
||||
assertEquals(
|
||||
"requirement failed: controller.quorum.auto.join.enable is only " +
|
||||
"supported when process.roles contains the 'controller' role.",
|
||||
assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage
|
||||
)
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAcceptsLargeId(): Unit = {
|
||||
val largeBrokerId = 2000
|
||||
|
|
|
@ -385,6 +385,84 @@ Found problem:
|
|||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatWithReleaseVersionAndFeatureOverride(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties, Seq(
|
||||
"--release-version", "3.7-IV0",
|
||||
"--feature", "share.version=1")))
|
||||
|
||||
// Verify that the feature override is applied by checking the bootstrap metadata
|
||||
val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString).read
|
||||
|
||||
// Verify that the share.version feature is set to 1 as specified
|
||||
assertEquals(1.toShort, bootstrapMetadata.featureLevel("share.version"),
|
||||
"share.version should be set to 1")
|
||||
|
||||
// Verify the command output contains the expected release version
|
||||
assertTrue(stream.toString().contains("3.7-IV0"),
|
||||
"Failed to find release version in output: " + stream.toString())
|
||||
|
||||
// Verify that the format command completed successfully with features
|
||||
assertTrue(stream.toString().contains("Formatting metadata directory"),
|
||||
"Failed to find formatting message in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatWithMultipleFeatures(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties, Seq(
|
||||
"--release-version", "3.8-IV0",
|
||||
"--feature", "share.version=1",
|
||||
"--feature", "transaction.version=2",
|
||||
"--feature", "group.version=1")))
|
||||
|
||||
// Verify that all features are properly bootstrapped by checking the bootstrap metadata
|
||||
val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString).read
|
||||
|
||||
// Verify that all specified features are set correctly
|
||||
assertEquals(1.toShort, bootstrapMetadata.featureLevel("share.version"),
|
||||
"share.version should be set to 1")
|
||||
assertEquals(2.toShort, bootstrapMetadata.featureLevel("transaction.version"),
|
||||
"transaction.version should be set to 2")
|
||||
assertEquals(1.toShort, bootstrapMetadata.featureLevel("group.version"),
|
||||
"group.version should be set to 1")
|
||||
|
||||
// Verify the command output contains the expected release version
|
||||
assertTrue(stream.toString().contains("3.8-IV0"),
|
||||
"Failed to find release version in output: " + stream.toString())
|
||||
|
||||
// Verify that the format command completed successfully with multiple features
|
||||
assertTrue(stream.toString().contains("Formatting metadata directory"),
|
||||
"Failed to find formatting message in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatWithInvalidFeatureThrowsError(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
|
||||
// Test with an invalid feature that doesn't exist
|
||||
val exception = assertThrows(classOf[FormatterException], () => {
|
||||
runFormatCommand(stream, properties, Seq(
|
||||
"--release-version", "3.7-IV0",
|
||||
"--feature", "stream.version=1"))
|
||||
})
|
||||
|
||||
assertTrue(exception.getMessage.contains("Unsupported feature: stream.version."))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatWithStandaloneFlagOnBrokerFails(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
|
|
|
@ -87,8 +87,8 @@ log.dirs=/tmp/kraft-combined-logs
|
|||
num.partitions=1
|
||||
|
||||
# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
|
||||
# This value is recommended to be increased for installations with data dirs located in RAID array.
|
||||
num.recovery.threads.per.data.dir=1
|
||||
# This value is recommended to be increased based on the installation resources.
|
||||
num.recovery.threads.per.data.dir=2
|
||||
|
||||
############################# Internal Topic Settings #############################
|
||||
# The replication factor for the group metadata internal topics "__consumer_offsets", "__share_group_state" and "__transaction_state"
|
||||
|
|
|
@ -329,8 +329,23 @@
|
|||
<h3 class="anchor-heading"><a id="tieredstorageconfigs" class="anchor-link"></a><a href="#tieredstorageconfigs">3.11 Tiered Storage Configs</a></h3>
|
||||
Below is the Tiered Storage configuration.
|
||||
<!--#include virtual="generated/remote_log_manager_config.html" -->
|
||||
|
||||
<h4 class="anchor-heading"><a id="rlmmconfigs" class="anchor-link"></a><a href="#rlmmconfigs">3.11.1 RLMM Configs</a></h4>
|
||||
<p>Below is the configuration for <code>TopicBasedRemoteLogMetadataManager</code>, which is the default implementation of <code>RemoteLogMetadataManager</code>.</p>
|
||||
<p>All configurations here should start with the prefix defined by <code>remote.log.metadata.manager.impl.prefix</code>, for example, <code>rlmm.config.remote.log.metadata.consume.wait.ms</code>.</p>
|
||||
<!--#include virtual="generated/remote_log_metadata_manager_config.html" -->
|
||||
|
||||
<p>The implementation of <code>TopicBasedRemoteLogMetadataManager</code> needs to create admin, producer, and consumer clients for the internal topic <code>__remote_log_metadata</code>.</p>
|
||||
<p>Additional configurations can be provided for different types of clients using the following configuration properties: </p>
|
||||
<pre><code class="language-text"># Configs for admin, producer, and consumer clients
|
||||
<rlmm.prefix>.remote.log.metadata.common.client.<kafka.property> = <value>
|
||||
|
||||
# Configs only for producer client
|
||||
<rlmm.prefix>.remote.log.metadata.producer.<kafka.property> = <value>
|
||||
|
||||
# Configs only for consumer client
|
||||
<rlmm.prefix>.remote.log.metadata.consumer.<kafka.property> = <value></code></pre>
|
||||
|
||||
<h3 class="anchor-heading">
|
||||
<a id="config_providers" class="anchor-link"></a>
|
||||
<a href="#config_providers">3.12 Configuration Providers</a>
|
||||
|
|
|
@ -762,7 +762,7 @@ public List<SourceRecord> poll() throws InterruptedException {
|
|||
<p>The <code>flush()</code> method is used during the offset commit process, which allows tasks to recover from failures and resume from a safe point such that no events will be missed. The method should push any outstanding data to the destination system and then block until the write has been acknowledged. The <code>offsets</code> parameter can often be ignored, but is useful in some cases where implementations want to store offset information in the destination store to provide exactly-once
|
||||
delivery. For example, an HDFS connector could do this and use atomic move operations to make sure the <code>flush()</code> operation atomically commits the data and offsets to a final location in HDFS.</p>
|
||||
|
||||
<h5><a id="connect_errantrecordreporter" href="connect_errantrecordreporter">Errant Record Reporter</a></h5>
|
||||
<h5><a id="connect_errantrecordreporter" href="#connect_errantrecordreporter">Errant Record Reporter</a></h5>
|
||||
|
||||
<p>When <a href="#connect_errorreporting">error reporting</a> is enabled for a connector, the connector can use an <code>ErrantRecordReporter</code> to report problems with individual records sent to a sink connector. The following example shows how a connector's <code>SinkTask</code> subclass might obtain and use the <code>ErrantRecordReporter</code>, safely handling a null reporter when the DLQ is not enabled or when the connector is installed in an older Connect runtime that doesn't have this reporter feature:</p>
|
||||
|
||||
|
|
|
@ -2107,6 +2107,16 @@ The following set of metrics are available for monitoring the group coordinator:
|
|||
<td>The max time in millis remote copies was throttled by a broker</td>
|
||||
<td>kafka.server:type=RemoteLogManager, name=remote-copy-throttle-time-max</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>RemoteLogReader Fetch Rate And Time</td>
|
||||
<td>The time to read data from remote storage by a broker</td>
|
||||
<td>kafka.log.remote:type=RemoteLogManager,name=RemoteLogReaderFetchRateAndTimeMs</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Delayed Remote List Offsets Expires Per Sec</td>
|
||||
<td>The number of expired remote list offsets per second. Omitting 'topic=(...), partition=(...)' will yield the all-topic rate</td>
|
||||
<td>kafka.server:type=DelayedRemoteListOffsetsMetrics,name=ExpiresPerSec,topic=([-.\w]+),partition=([0-9]+)</td>
|
||||
</tr>
|
||||
|
||||
</tbody>
|
||||
</table>
|
||||
|
@ -4501,9 +4511,16 @@ $ bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:
|
|||
<p>Downgrades are safe to perform by setting <code>eligible.leader.replicas.version=0</code>.</p>
|
||||
|
||||
<h4 class="anchor-heading"><a id="eligible_leader_replicas_tool" class="anchor-link"></a><a href="#eligible_leader_replicas_tool">Tool</a></h4>
|
||||
<p>The ELR fields can be checked through the API DescribeTopicPartitions. The admin client can fetch the ELR info by describing the topics.
|
||||
Also note that, if <code>min.insync.replicas</code> is updated for a topic, the ELR field will be cleaned. If cluster default min ISR is updated,
|
||||
all the ELR fields will be cleaned.</p>
|
||||
<p>The ELR fields can be checked through the API DescribeTopicPartitions. The admin client can fetch the ELR info by describing the topics.</p>
|
||||
<p>Note that when the ELR feature is enabled:</p>
|
||||
<ul>
|
||||
<li>The cluster-level <code>min.insync.replicas</code> config will be added if there is not any. The value is the same as the static config in the active controller.</li>
|
||||
<li>The removal of <code>min.insync.replicas</code> config at the cluster-level is not allowed.</li>
|
||||
<li>If the cluster-level <code>min.insync.replicas</code> is updated, even if the value is unchanged, all the ELR state will be cleaned.</li>
|
||||
<li>The previously set <code>min.insync.replicas</code> value at the broker-level config will be removed. Please set at the cluster-level if necessary.</li>
|
||||
<li>The alteration of <code>min.insync.replicas</code> config at the broker-level is not allowed.</li>
|
||||
<li>If <code>min.insync.replicas</code> is updated for a topic, the ELR state will be cleaned.</li>
|
||||
</ul>
|
||||
|
||||
</script>
|
||||
|
||||
|
|
|
@ -279,7 +279,7 @@
|
|||
<p>
|
||||
In stream processing, one of the most frequently asked question is "does my stream processing system guarantee that each record is processed once and only once, even if some failures are encountered in the middle of processing?"
|
||||
Failing to guarantee exactly-once stream processing is a deal-breaker for many applications that cannot tolerate any data-loss or data duplicates, and in that case a batch-oriented framework is usually used in addition
|
||||
to the stream processing pipeline, known as the <a href="http://lambda-architecture.net/">Lambda Architecture</a>.
|
||||
to the stream processing pipeline, known as the <a href="https://en.wikipedia.org/wiki/Lambda_architecture">Lambda Architecture</a>.
|
||||
Prior to 0.11.0.0, Kafka only provides at-least-once delivery guarantees and hence any stream processing systems that leverage it as the backend storage could not guarantee end-to-end exactly-once semantics.
|
||||
In fact, even for those stream processing systems that claim to support exactly-once processing, as long as they are reading from / writing to Kafka as the source / sink, their applications cannot actually guarantee that
|
||||
no duplicates will be generated throughout the pipeline.<br />
|
||||
|
|
|
@ -153,6 +153,7 @@
|
|||
<li><a href="#monitoring">6.7 Monitoring</a>
|
||||
<ul>
|
||||
<li><a href="#remote_jmx">Security Considerations for Remote Monitoring using JMX</a>
|
||||
<li><a href="#group_coordinator_monitoring">Group Coordinator Monitoring</a>
|
||||
<li><a href="#tiered_storage_monitoring">Tiered Storage Monitoring</a>
|
||||
<li><a href="#kraft_monitoring">KRaft Monitoring</a>
|
||||
<li><a href="#selector_monitoring">Selector Monitoring</a>
|
||||
|
|
|
@ -37,6 +37,9 @@
|
|||
<li>
|
||||
The support for MX4J library, enabled through <code>kafka_mx4jenable</code> system property, was deprecated and will be removed in Kafka 5.0.
|
||||
</li>
|
||||
<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>
|
||||
</ul>
|
||||
|
||||
<h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4>
|
||||
|
@ -57,6 +60,9 @@
|
|||
The logger class name for LogCleaner has been updated from <code>kafka.log.LogCleaner</code> to <code>org.apache.kafka.storage.internals.log.LogCleaner</code> in the log4j2.yaml configuration file.
|
||||
Added loggers for <code>org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread</code> and <code>org.apache.kafka.storage.internals.log.Cleaner</code> classes to CleanerAppender.
|
||||
</li>
|
||||
<li>
|
||||
The filename for rotated <code>state-change.log</code> files has been updated from <code>stage-change.log.[date]</code> to <code>state-change.log.[date]</code> in the log4j2.yaml configuration file.
|
||||
</li>
|
||||
</ul>
|
||||
</li>
|
||||
<li><b>Broker</b>
|
||||
|
@ -68,6 +74,8 @@
|
|||
</li>
|
||||
<li>
|
||||
The KIP-966 part 1: Eligible Leader Replicas(ELR) will be enabled by default on the new clusters.
|
||||
After the ELR feature enabled, the previously set <code>min.insync.replicas</code> value at the broker-level config will be removed.
|
||||
Please set at the cluster-level if necessary.
|
||||
For further details, please refer to <a href="/{{version}}/documentation.html#eligible_leader_replicas">here</a>.
|
||||
</li>
|
||||
</ul>
|
||||
|
@ -102,9 +110,9 @@
|
|||
</li>
|
||||
</ul>
|
||||
|
||||
<h4><a id="upgrade_4_0_0_from" href="#upgrade_4_0_0_from">Upgrading to 4.0.0</a></h4>
|
||||
<h4><a id="upgrade_4_0_1_from" href="#upgrade_4_0_1_from">Upgrading to 4.0.1</a></h4>
|
||||
|
||||
<h5><a id="upgrade_clients_4_0_0" href="#upgrade_clients_4_0_0">Upgrading Clients to 4.0.0</a></h5>
|
||||
<h5><a id="upgrade_clients_4_0_1" href="#upgrade_clients_4_0_1">Upgrading Clients to 4.0.1</a></h5>
|
||||
|
||||
<p><b>For a rolling upgrade:</b></p>
|
||||
|
||||
|
@ -115,7 +123,7 @@
|
|||
or <a href="https://cwiki.apache.org/confluence/x/y4kgF">KIP-1124</a>.</li>
|
||||
</ol>
|
||||
|
||||
<h5><a id="upgrade_4_0_0" href="#upgrade_4_0_0">Upgrading Servers to 4.0.0 from any version 3.3.x through 3.9.x</a></h5>
|
||||
<h5><a id="upgrade_4_0_1" href="#upgrade_4_0_1">Upgrading Servers to 4.0.1 from any version 3.3.x through 3.9.x</a></h5>
|
||||
|
||||
<p>Note: Apache Kafka 4.0 only supports KRaft mode - ZooKeeper mode has been removed. As such, <b>broker upgrades to 4.0.0 (and higher) require KRaft mode and
|
||||
the software and metadata versions must be at least 3.3.x</b> (the first version when KRaft mode was deemed production ready). For clusters in KRaft mode
|
||||
|
@ -139,7 +147,13 @@
|
|||
has a boolean parameter that indicates if there are metadata changes (i.e. <code>IBP_4_0_IV1(23, "4.0", "IV1", true)</code> means this version has metadata changes).
|
||||
Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between.</li>
|
||||
</ol>
|
||||
|
||||
<h5><a id="upgrade_servers_401_notable" href="#upgrade_servers_401_notable">Notable changes in 4.0.1</a></h5>
|
||||
<ul>
|
||||
<li>
|
||||
The filename for rotated <code>state-change.log</code> files has been updated from <code>stage-change.log.[date]</code> to <code>state-change.log.[date]</code> in the log4j2.yaml configuration file.
|
||||
See <a href="https://issues.apache.org/jira/browse/KAFKA-19576">KAFKA-19576</a> for details.
|
||||
</li>
|
||||
</ul>
|
||||
<h5><a id="upgrade_servers_400_notable" href="#upgrade_servers_400_notable">Notable changes in 4.0.0</a></h5>
|
||||
<ul>
|
||||
<li>
|
||||
|
@ -151,7 +165,7 @@
|
|||
</li>
|
||||
<li>
|
||||
Apache Kafka 4.0 only supports KRaft mode - ZooKeeper mode has been removed. About version upgrade,
|
||||
check <a href="/{{version}}/documentation.html#upgrade_4_0_0">Upgrading to 4.0.0 from any version 3.3.x through 3.9.x</a> for more info.
|
||||
check <a href="/{{version}}/documentation.html#upgrade_4_0_1">Upgrading to 4.0.1 from any version 3.3.x through 3.9.x</a> for more info.
|
||||
</li>
|
||||
<li>
|
||||
Apache Kafka 4.0 ships with a brand-new group coordinator implementation (See <a href="https://cwiki.apache.org/confluence/x/HhD1D">here</a>).
|
||||
|
@ -494,6 +508,10 @@
|
|||
<li> See <a href="https://cwiki.apache.org/confluence/x/B40ODg">KIP-890</a> and
|
||||
<a href="https://cwiki.apache.org/confluence/x/8ItyEg">KIP-1050</a> for more details </li>
|
||||
</ul>
|
||||
<li>
|
||||
The filename for rotated <code>state-change.log</code> files incorrectly rotates to <code>stage-change.log.[date]</code> (changing state to stage). This issue is corrected in 4.0.1.
|
||||
See <a href="https://issues.apache.org/jira/browse/KAFKA-19576">KAFKA-19576</a> for details.
|
||||
</li>
|
||||
</ul>
|
||||
</li>
|
||||
</ul>
|
||||
|
|
|
@ -123,7 +123,7 @@ versions += [
|
|||
scoverage: "2.0.11",
|
||||
slf4j: "1.7.36",
|
||||
snappy: "1.1.10.7",
|
||||
spotbugs: "4.8.6",
|
||||
spotbugs: "4.9.4",
|
||||
mockOAuth2Server: "2.2.1",
|
||||
zinc: "1.9.2",
|
||||
// When updating the zstd version, please do as well in docker/native/native-image-configs/resource-config.json
|
||||
|
|
|
@ -550,4 +550,189 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
|
|||
<Bug pattern="SING_SINGLETON_HAS_NONPRIVATE_CONSTRUCTOR"/>
|
||||
</Match>
|
||||
|
||||
<Match>
|
||||
<!-- New warning type added when we upgraded from spotbugs 4.8.6 to 4.9.4.
|
||||
These are possibly real bugs, and have not been evaluated, they were just bulk excluded to unblock upgrading Spotbugs.
|
||||
-->
|
||||
<Or>
|
||||
<Class name="org.apache.kafka.clients.producer.MockProducer"/>
|
||||
<Class name="org.apache.kafka.clients.producer.internals.ProducerBatch"/>
|
||||
<Class name="org.apache.kafka.common.utils.ChunkedBytesStream"/>
|
||||
<Class name="org.apache.kafka.common.security.authenticator.LoginManager"/>
|
||||
<Class name="org.apache.kafka.storage.internals.log.AbstractIndex"/>
|
||||
<Class name="org.apache.kafka.server.AssignmentsManager"/>
|
||||
<Class name="org.apache.kafka.connect.file.FileStreamSourceTask"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.DefaultStateUpdater$StateUpdaterThread"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.StreamThread"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.StreamsProducer"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.AbstractDualSchemaRocksDBSegmentedBytesStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.AbstractRocksDBSegmentedBytesStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.InMemorySessionStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueChangeBuffer"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.InMemoryWindowStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.ThreadCache"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.WorkerSinkTask"/>
|
||||
<Class name="org.apache.kafka.tools.VerifiableProducer"/>
|
||||
<Class name="kafka.coordinator.transaction.TransactionMetadata"/>
|
||||
<Class name="org.apache.kafka.tools.VerifiableShareConsumer"/>
|
||||
<Class name="org.apache.kafka.server.quota.ClientQuotaManager"/>
|
||||
<Class name="kafka.log.LogManager"/>
|
||||
<Class name="kafka.server.DelayedFetch"/>
|
||||
<Class name="kafka.server.KafkaApis"/>
|
||||
<Class name="kafka.server.metadata.KRaftMetadataCache"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.InternalTopologyBuilder"/>
|
||||
</Or>
|
||||
<Bug pattern="AT_NONATOMIC_OPERATIONS_ON_SHARED_VARIABLE"/>
|
||||
</Match>
|
||||
|
||||
<Match>
|
||||
<!-- New warning type added when we upgraded from spotbugs 4.8.6 to 4.9.4.
|
||||
These are possibly real bugs, and have not been evaluated, they were just bulk excluded to unblock upgrading Spotbugs.
|
||||
-->
|
||||
<Or>
|
||||
<Class name="org.apache.kafka.clients.producer.internals.ProducerBatch"/>
|
||||
<Class name="org.apache.kafka.clients.producer.internals.RecordAccumulator"/>
|
||||
<Class name="org.apache.kafka.common.security.kerberos.KerberosLogin"/>
|
||||
<Class name="org.apache.kafka.server.util.timer.TimingWheel"/>
|
||||
<Class name="org.apache.kafka.server.log.remote.metadata.storage.ConsumerTask"/>
|
||||
<Class name="org.apache.kafka.storage.internals.log.LogSegment"/>
|
||||
<Class name="org.apache.kafka.storage.internals.log.ProducerStateManager"/>
|
||||
<Class name="org.apache.kafka.image.publisher.SnapshotGenerator"/>
|
||||
<Class name="org.apache.kafka.image.publisher.SnapshotGenerator$Builder"/>
|
||||
<Class name="org.apache.kafka.connect.file.FileStreamSourceTask"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.DefaultStateUpdater$StateUpdaterThread"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.StreamThread"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.StreamsProducer"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueChangeBuffer"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.WorkerSinkTask"/>
|
||||
<Class name="org.apache.kafka.tools.ConsumerPerformance$ConsumerPerfRebListener"/>
|
||||
<Class name="kafka.coordinator.transaction.TransactionMetadata"/>
|
||||
<Class name="kafka.server.BrokerLifecycleManager"/>
|
||||
<Class name="kafka.server.CachedPartition"/>
|
||||
<Class name="kafka.server.ControllerRegistrationManager"/>
|
||||
<Class name="kafka.server.FetchSession"/>
|
||||
<Class name="kafka.server.share.SharePartition"/>
|
||||
<Class name="kafka.cluster.Partition"/>
|
||||
<Class name="kafka.coordinator.transaction.TransactionCoordinator"/>
|
||||
<Class name="kafka.coordinator.transaction.TransactionStateManager"/>
|
||||
<Class name="kafka.log.LogManager"/>
|
||||
<Class name="kafka.server.ReplicaManager"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.AbstractDualSchemaRocksDBSegmentedBytesStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.AbstractRocksDBSegmentedBytesStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.InMemorySessionStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.InMemoryWindowStore"/>
|
||||
</Or>
|
||||
<Bug pattern="AT_NONATOMIC_64BIT_PRIMITIVE"/>
|
||||
</Match>
|
||||
|
||||
<Match>
|
||||
<!-- New warning type added when we upgraded from spotbugs 4.8.6 to 4.9.4.
|
||||
These are possibly real bugs, and have not been evaluated, they were just bulk excluded to unblock upgrading Spotbugs.
|
||||
-->
|
||||
<Or>
|
||||
<Class name="org.apache.kafka.clients.consumer.MockConsumer"/>
|
||||
<Class name="org.apache.kafka.clients.consumer.internals.AbstractMembershipManager"/>
|
||||
<Class name="org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer"/>
|
||||
<Class name="org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer"/>
|
||||
<Class name="org.apache.kafka.clients.consumer.internals.ConsumerCoordinator"/>
|
||||
<Class name="org.apache.kafka.clients.consumer.internals.ShareConsumeRequestManager"/>
|
||||
<Class name="org.apache.kafka.clients.consumer.internals.StreamsMembershipManager"/>
|
||||
<Class name="org.apache.kafka.clients.producer.MockProducer"/>
|
||||
<Class name="org.apache.kafka.clients.producer.internals.ProducerBatch"/>
|
||||
<Class name="org.apache.kafka.clients.producer.internals.TransactionManager"/>
|
||||
<Class name="org.apache.kafka.common.network.Selector"/>
|
||||
<Class name="org.apache.kafka.common.security.authenticator.LoginManager"/>
|
||||
<Class name="org.apache.kafka.common.security.authenticator.SaslClientAuthenticator"/>
|
||||
<Class name="org.apache.kafka.common.security.kerberos.KerberosLogin"/>
|
||||
<Class name="org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallbackHandler"/>
|
||||
<Class name="org.apache.kafka.common.security.oauthbearer.internals.secured.RefreshingHttpsJwks"/>
|
||||
<Class name="org.apache.kafka.common.utils.ChunkedBytesStream"/>
|
||||
<Class name="org.apache.kafka.common.utils.Shell"/>
|
||||
<Class name="org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler"/>
|
||||
<Class name="org.apache.kafka.server.log.remote.metadata.storage.ConsumerTask"/>
|
||||
<Class name="org.apache.kafka.storage.internals.log.LogSegment"/>
|
||||
<Class name="org.apache.kafka.image.publisher.SnapshotGenerator$Builder"/>
|
||||
<Class name="org.apache.kafka.server.AssignmentsManager"/>
|
||||
<Class name="org.apache.kafka.connect.file.FileStreamSourceTask"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.InternalTopologyBuilder"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.StreamsProducer"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.TaskManager"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.AbstractRocksDBSegmentedBytesStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueChangeBuffer"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.RocksDBStore"/>
|
||||
<Class name="org.apache.kafka.streams.state.internals.TimeOrderedCachingWindowStore"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.AbstractWorkerSourceTask"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.ExactlyOnceWorkerSourceTask"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.WorkerSinkTask"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.distributed.DistributedHerder"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.distributed.WorkerCoordinator"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.distributed.WorkerGroupMember"/>
|
||||
<Class name="org.apache.kafka.connect.util.KafkaBasedLog"/>
|
||||
<Class name="org.apache.kafka.tools.VerifiableProducer"/>
|
||||
<Class name="kafka.coordinator.transaction.TransactionMetadata"/>
|
||||
<Class name="kafka.network.Acceptor"/>
|
||||
<Class name="kafka.network.Processor"/>
|
||||
<Class name="kafka.server.BrokerLifecycleManager"/>
|
||||
<Class name="kafka.server.BrokerServer"/>
|
||||
<Class name="kafka.server.CachedPartition"/>
|
||||
<Class name="kafka.server.ControllerRegistrationManager"/>
|
||||
<Class name="kafka.server.FetchSession"/>
|
||||
<Class name="kafka.server.KafkaConfig"/>
|
||||
<Class name="kafka.server.metadata.BrokerMetadataPublisher"/>
|
||||
<Class name="kafka.server.metadata.DelegationTokenPublisher"/>
|
||||
<Class name="org.apache.kafka.connect.mirror.MirrorSourceConnector"/>
|
||||
<Class name="org.apache.kafka.raft.LeaderState"/>
|
||||
<Class name="kafka.coordinator.transaction.TransactionCoordinator"/>
|
||||
<Class name="kafka.coordinator.transaction.TransactionMarkerRequestCompletionHandler"/>
|
||||
<Class name="kafka.coordinator.transaction.TransactionStateManager"/>
|
||||
<Class name="kafka.log.LogManager"/>
|
||||
<Class name="kafka.server.KafkaApis"/>
|
||||
<Class name="kafka.server.ReplicaManager"/>
|
||||
<Class name="org.apache.kafka.trogdor.coordinator.NodeManager$NodeHeartbeat"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.distributed.DistributedHerder$RebalanceListener"/>
|
||||
<Class name="org.apache.kafka.streams.processor.internals.StreamThread"/>
|
||||
</Or>
|
||||
<Bug pattern="AT_STALE_THREAD_WRITE_OF_PRIMITIVE"/>
|
||||
</Match>
|
||||
|
||||
<Match>
|
||||
<!-- New warning type added when we upgraded from spotbugs 4.8.6 to 4.9.4.
|
||||
These are possibly real bugs, and have not been evaluated, they were just bulk excluded to unblock upgrading Spotbugs.
|
||||
-->
|
||||
<Or>
|
||||
<Class name="kafka.coordinator.transaction.CompleteCommit"/>
|
||||
<Class name="kafka.coordinator.transaction.CompleteAbort"/>
|
||||
<Class name="kafka.coordinator.transaction.Dead"/>
|
||||
<Class name="kafka.coordinator.transaction.Empty"/>
|
||||
<Class name="kafka.coordinator.transaction.Ongoing"/>
|
||||
<Class name="kafka.coordinator.transaction.PrepareAbort"/>
|
||||
<Class name="kafka.coordinator.transaction.PrepareCommit"/>
|
||||
<Class name="kafka.coordinator.transaction.PrepareEpochFence"/>
|
||||
<Class name="kafka.server.Fetching"/>
|
||||
<Class name="kafka.server.Truncating"/>
|
||||
<Class name="kafka.server.addPartition"/>
|
||||
<Class name="kafka.server.defaultError"/>
|
||||
<Class name="kafka.server.genericErrorSupported"/>
|
||||
<Class name="kafka.server.metadata.DefaultClientIdEntity"/>
|
||||
<Class name="kafka.server.metadata.DefaultIpEntity"/>
|
||||
<Class name="kafka.server.metadata.DefaultUserDefaultClientIdEntity"/>
|
||||
<Class name="kafka.server.metadata.DefaultUserEntity"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.SinkConnectorConfig"/>
|
||||
<Class name="org.apache.kafka.connect.runtime.SourceConnectorConfig"/>
|
||||
</Or>
|
||||
<Bug pattern="HSM_HIDING_METHOD"/>
|
||||
</Match>
|
||||
|
||||
<Match>
|
||||
<!-- New warning type added when we upgraded from spotbugs 4.8.6 to 4.9.4.
|
||||
These are possibly real bugs, and have not been evaluated, they were just bulk excluded to unblock upgrading Spotbugs.
|
||||
-->
|
||||
<Or>
|
||||
<Class name="org.apache.kafka.common.security.ssl.SslFactory$SslEngineValidator"/>
|
||||
<Class name="org.apache.kafka.server.authorizer.Authorizer"/>
|
||||
</Or>
|
||||
<Bug pattern="SF_SWITCH_FALLTHROUGH"/>
|
||||
</Match>
|
||||
|
||||
</FindBugsFilter>
|
||||
|
|
|
@ -611,7 +611,7 @@ public class ConfigurationControlManager {
|
|||
|
||||
/**
|
||||
* Generate any configuration records that are needed to make it safe to enable ELR.
|
||||
* Specifically, we need to remove all cluster-level configurations for min.insync.replicas,
|
||||
* Specifically, we need to remove all broker-level configurations for min.insync.replicas,
|
||||
* and create a cluster-level configuration for min.insync.replicas. It is always safe to call
|
||||
* this function if ELR is already enabled; it will simply do nothing if the necessary
|
||||
* configurations already exist.
|
||||
|
|
|
@ -211,11 +211,7 @@ public final class TopicDelta {
|
|||
|
||||
try {
|
||||
PartitionRegistration prevPartition = image.partitions().get(entry.getKey());
|
||||
if (
|
||||
prevPartition == null ||
|
||||
prevPartition.directories == null ||
|
||||
prevPartition.directory(brokerId) != entry.getValue().directory(brokerId)
|
||||
) {
|
||||
if (prevPartition == null || prevPartition.directory(brokerId) != entry.getValue().directory(brokerId)) {
|
||||
directoryIds.put(
|
||||
new TopicIdPartition(id(), new TopicPartition(name(), entry.getKey())),
|
||||
entry.getValue().directory(brokerId)
|
||||
|
|
|
@ -415,6 +415,7 @@ public class Formatter {
|
|||
if (ensemble.emptyLogDirs().isEmpty()) {
|
||||
printStream.println("All of the log directories are already formatted.");
|
||||
} else {
|
||||
printStream.println("Bootstrap metadata: " + bootstrapMetadata);
|
||||
Map<String, DirectoryType> directoryTypes = new HashMap<>();
|
||||
for (String emptyLogDir : ensemble.emptyLogDirs()) {
|
||||
DirectoryType directoryType = DirectoryType.calculate(emptyLogDir,
|
||||
|
|
|
@ -89,7 +89,7 @@ public class FormatterTest {
|
|||
Formatter formatter = new Formatter().
|
||||
setNodeId(DEFAULT_NODE_ID).
|
||||
setClusterId(DEFAULT_CLUSTER_ID.toString());
|
||||
directories.forEach(d -> formatter.addDirectory(d));
|
||||
directories.forEach(formatter::addDirectory);
|
||||
formatter.setMetadataLogDirectory(directories.get(0));
|
||||
return new FormatterContext(formatter);
|
||||
}
|
||||
|
@ -170,7 +170,7 @@ public class FormatterTest {
|
|||
String expectedPrefix = "Error while writing meta.properties file";
|
||||
assertEquals(expectedPrefix,
|
||||
assertThrows(FormatterException.class,
|
||||
() -> formatter1.formatter.run()).
|
||||
formatter1.formatter::run).
|
||||
getMessage().substring(0, expectedPrefix.length()));
|
||||
}
|
||||
}
|
||||
|
@ -180,15 +180,15 @@ public class FormatterTest {
|
|||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.run();
|
||||
assertEquals("Formatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.latestProduction() + ".",
|
||||
formatter1.output().trim());
|
||||
assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() +
|
||||
"\nFormatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.latestProduction() + ".",
|
||||
formatter1.output().trim());
|
||||
|
||||
FormatterContext formatter2 = testEnv.newFormatter();
|
||||
formatter2.formatter.setIgnoreFormatted(true);
|
||||
formatter2.formatter.run();
|
||||
assertEquals("All of the log directories are already formatted.",
|
||||
formatter2.output().trim());
|
||||
assertTrue(formatter2.output().trim().contains("All of the log directories are already formatted."));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -201,7 +201,8 @@ public class FormatterTest {
|
|||
formatter1.formatter
|
||||
.setInitialControllers(DynamicVoters.parse("1@localhost:8020:" + originalDirectoryId))
|
||||
.run();
|
||||
assertEquals("Formatting dynamic metadata voter directory " + testEnv.directory(0) +
|
||||
assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() +
|
||||
"\nFormatting dynamic metadata voter directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.latestProduction() + ".",
|
||||
formatter1.output().trim());
|
||||
assertMetadataDirectoryId(testEnv, Uuid.fromString(originalDirectoryId));
|
||||
|
@ -244,9 +245,10 @@ public class FormatterTest {
|
|||
FormatterContext formatter2 = testEnv.newFormatter();
|
||||
formatter2.formatter.setIgnoreFormatted(true);
|
||||
formatter2.formatter.run();
|
||||
assertEquals("Formatting data directory " + testEnv.directory(1) + " with metadata.version " +
|
||||
MetadataVersion.latestProduction() + ".",
|
||||
formatter2.output().trim());
|
||||
assertEquals("Bootstrap metadata: " + formatter2.formatter.bootstrapMetadata() +
|
||||
"\nFormatting data directory " + testEnv.directory(1) + " with metadata.version " +
|
||||
MetadataVersion.latestProduction() + ".",
|
||||
formatter2.output().trim());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -256,9 +258,10 @@ public class FormatterTest {
|
|||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_5_IV0);
|
||||
formatter1.formatter.run();
|
||||
assertEquals("Formatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.IBP_3_5_IV0 + ".",
|
||||
formatter1.output().trim());
|
||||
assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() +
|
||||
"\nFormatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.IBP_3_5_IV0 + ".",
|
||||
formatter1.output().trim());
|
||||
BootstrapMetadata bootstrapMetadata =
|
||||
new BootstrapDirectory(testEnv.directory(0)).read();
|
||||
assertEquals(MetadataVersion.IBP_3_5_IV0, bootstrapMetadata.metadataVersion());
|
||||
|
@ -272,7 +275,7 @@ public class FormatterTest {
|
|||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setReleaseVersion(MetadataVersion.latestTesting());
|
||||
assertEquals("metadata.version " + MetadataVersion.latestTesting() + " is not yet stable.",
|
||||
assertThrows(FormatterException.class, () -> formatter1.formatter.run()).getMessage());
|
||||
assertThrows(FormatterException.class, formatter1.formatter::run).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -283,9 +286,10 @@ public class FormatterTest {
|
|||
formatter1.formatter.setReleaseVersion(MetadataVersion.latestTesting());
|
||||
formatter1.formatter.setUnstableFeatureVersionsEnabled(true);
|
||||
formatter1.formatter.run();
|
||||
assertEquals("Formatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.latestTesting() + ".",
|
||||
formatter1.output().trim());
|
||||
assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() +
|
||||
"\nFormatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.latestTesting() + ".",
|
||||
formatter1.output().trim());
|
||||
BootstrapMetadata bootstrapMetadata =
|
||||
new BootstrapDirectory(testEnv.directory(0)).read();
|
||||
assertEquals(MetadataVersion.latestTesting(), bootstrapMetadata.metadataVersion());
|
||||
|
@ -318,7 +322,7 @@ public class FormatterTest {
|
|||
"saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]"));
|
||||
assertEquals("SCRAM is only supported in metadata.version 3.5-IV2 or later.",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> formatter1.formatter.run()).getMessage());
|
||||
formatter1.formatter::run).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -333,9 +337,10 @@ public class FormatterTest {
|
|||
"SCRAM-SHA-512=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," +
|
||||
"saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]"));
|
||||
formatter1.formatter.run();
|
||||
assertEquals("Formatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.IBP_3_8_IV0 + ".",
|
||||
formatter1.output().trim());
|
||||
assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() +
|
||||
"\nFormatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.IBP_3_8_IV0 + ".",
|
||||
formatter1.output().trim());
|
||||
BootstrapMetadata bootstrapMetadata =
|
||||
new BootstrapDirectory(testEnv.directory(0)).read();
|
||||
assertEquals(MetadataVersion.IBP_3_8_IV0, bootstrapMetadata.metadataVersion());
|
||||
|
@ -406,7 +411,7 @@ public class FormatterTest {
|
|||
"are: eligible.leader.replicas.version, group.version, kraft.version, " +
|
||||
"share.version, streams.version, test.feature.version, transaction.version",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> formatter1.formatter.run()).
|
||||
formatter1.formatter::run).
|
||||
getMessage());
|
||||
}
|
||||
}
|
||||
|
@ -425,6 +430,7 @@ public class FormatterTest {
|
|||
formatter1.formatter.run();
|
||||
assertEquals((short) 1, formatter1.formatter.featureLevels.getOrDefault("kraft.version", (short) 0));
|
||||
assertEquals(List.of(
|
||||
"Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata(),
|
||||
String.format("Formatting data directory %s with %s %s.",
|
||||
testEnv.directory(1),
|
||||
MetadataVersion.FEATURE_NAME,
|
||||
|
@ -459,7 +465,7 @@ public class FormatterTest {
|
|||
"Cannot set kraft.version to 0 if one of the flags --standalone, --initial-controllers, or " +
|
||||
"--no-initial-controllers is used. For dynamic controllers support, try removing the " +
|
||||
"--feature flag for kraft.version.",
|
||||
assertThrows(FormatterException.class, () -> formatter1.formatter.run()).getMessage()
|
||||
assertThrows(FormatterException.class, formatter1.formatter::run).getMessage()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -475,7 +481,7 @@ public class FormatterTest {
|
|||
"Cannot set kraft.version to 1 unless one of the flags --standalone, --initial-controllers, or " +
|
||||
"--no-initial-controllers is used. For dynamic controllers support, try using one of " +
|
||||
"--standalone, --initial-controllers, or --no-initial-controllers.",
|
||||
assertThrows(FormatterException.class, () -> formatter1.formatter.run()).getMessage()
|
||||
assertThrows(FormatterException.class, formatter1.formatter::run).getMessage()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -492,7 +498,7 @@ public class FormatterTest {
|
|||
assertEquals("kraft.version could not be set to 1 because it depends on " +
|
||||
"metadata.version level 21",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> formatter1.formatter.run()).getMessage());
|
||||
formatter1.formatter::run).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -514,12 +520,12 @@ public class FormatterTest {
|
|||
formatter1.formatter.setInitialControllers(DynamicVoters.
|
||||
parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw"));
|
||||
if (metadataVersion.isAtLeast(MetadataVersion.IBP_4_0_IV1)) {
|
||||
assertDoesNotThrow(() -> formatter1.formatter.run());
|
||||
assertDoesNotThrow(formatter1.formatter::run);
|
||||
} else {
|
||||
assertEquals("eligible.leader.replicas.version could not be set to 1 because it depends on " +
|
||||
"metadata.version level 23",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> formatter1.formatter.run()).getMessage());
|
||||
formatter1.formatter::run).getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -539,6 +545,7 @@ public class FormatterTest {
|
|||
formatter1.formatter.run();
|
||||
assertEquals((short) 1, formatter1.formatter.featureLevels.getOrDefault("kraft.version", (short) 0));
|
||||
assertEquals(List.of(
|
||||
"Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata(),
|
||||
String.format("Formatting data directory %s with %s %s.",
|
||||
testEnv.directory(1),
|
||||
MetadataVersion.FEATURE_NAME,
|
||||
|
|
|
@ -40,8 +40,8 @@ public class FollowerState implements EpochState {
|
|||
private final Set<Integer> voters;
|
||||
// Used for tracking the expiration of both the Fetch and FetchSnapshot requests
|
||||
private final Timer fetchTimer;
|
||||
// Used to track when to send another update voter request
|
||||
private final Timer updateVoterPeriodTimer;
|
||||
// Used to track when to send another add, remove, or update voter request
|
||||
private final Timer updateVoterSetPeriodTimer;
|
||||
|
||||
/* Used to track if the replica has fetched successfully from the leader at least once since
|
||||
* the transition to follower in this epoch. If the replica has not yet fetched successfully,
|
||||
|
@ -76,7 +76,7 @@ public class FollowerState implements EpochState {
|
|||
this.votedKey = votedKey;
|
||||
this.voters = voters;
|
||||
this.fetchTimer = time.timer(fetchTimeoutMs);
|
||||
this.updateVoterPeriodTimer = time.timer(updateVoterPeriodMs());
|
||||
this.updateVoterSetPeriodTimer = time.timer(updateVoterPeriodMs());
|
||||
this.highWatermark = highWatermark;
|
||||
this.log = logContext.logger(FollowerState.class);
|
||||
}
|
||||
|
@ -154,19 +154,19 @@ public class FollowerState implements EpochState {
|
|||
return fetchTimeoutMs;
|
||||
}
|
||||
|
||||
public boolean hasUpdateVoterPeriodExpired(long currentTimeMs) {
|
||||
updateVoterPeriodTimer.update(currentTimeMs);
|
||||
return updateVoterPeriodTimer.isExpired();
|
||||
public boolean hasUpdateVoterSetPeriodExpired(long currentTimeMs) {
|
||||
updateVoterSetPeriodTimer.update(currentTimeMs);
|
||||
return updateVoterSetPeriodTimer.isExpired();
|
||||
}
|
||||
|
||||
public long remainingUpdateVoterPeriodMs(long currentTimeMs) {
|
||||
updateVoterPeriodTimer.update(currentTimeMs);
|
||||
return updateVoterPeriodTimer.remainingMs();
|
||||
public long remainingUpdateVoterSetPeriodMs(long currentTimeMs) {
|
||||
updateVoterSetPeriodTimer.update(currentTimeMs);
|
||||
return updateVoterSetPeriodTimer.remainingMs();
|
||||
}
|
||||
|
||||
public void resetUpdateVoterPeriod(long currentTimeMs) {
|
||||
updateVoterPeriodTimer.update(currentTimeMs);
|
||||
updateVoterPeriodTimer.reset(updateVoterPeriodMs());
|
||||
public void resetUpdateVoterSetPeriod(long currentTimeMs) {
|
||||
updateVoterSetPeriodTimer.update(currentTimeMs);
|
||||
updateVoterSetPeriodTimer.reset(updateVoterPeriodMs());
|
||||
}
|
||||
|
||||
public boolean hasUpdatedLeader() {
|
||||
|
|
|
@ -19,11 +19,13 @@ package org.apache.kafka.raft;
|
|||
import org.apache.kafka.clients.ClientResponse;
|
||||
import org.apache.kafka.clients.KafkaClient;
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.message.AddRaftVoterRequestData;
|
||||
import org.apache.kafka.common.message.ApiVersionsRequestData;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.FetchRequestData;
|
||||
import org.apache.kafka.common.message.FetchSnapshotRequestData;
|
||||
import org.apache.kafka.common.message.RemoveRaftVoterRequestData;
|
||||
import org.apache.kafka.common.message.UpdateRaftVoterRequestData;
|
||||
import org.apache.kafka.common.message.VoteRequestData;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
|
@ -31,11 +33,13 @@ import org.apache.kafka.common.protocol.ApiKeys;
|
|||
import org.apache.kafka.common.protocol.ApiMessage;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.requests.AbstractRequest;
|
||||
import org.apache.kafka.common.requests.AddRaftVoterRequest;
|
||||
import org.apache.kafka.common.requests.ApiVersionsRequest;
|
||||
import org.apache.kafka.common.requests.BeginQuorumEpochRequest;
|
||||
import org.apache.kafka.common.requests.EndQuorumEpochRequest;
|
||||
import org.apache.kafka.common.requests.FetchRequest;
|
||||
import org.apache.kafka.common.requests.FetchSnapshotRequest;
|
||||
import org.apache.kafka.common.requests.RemoveRaftVoterRequest;
|
||||
import org.apache.kafka.common.requests.UpdateRaftVoterRequest;
|
||||
import org.apache.kafka.common.requests.VoteRequest;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
|
@ -181,20 +185,25 @@ public class KafkaNetworkChannel implements NetworkChannel {
|
|||
static AbstractRequest.Builder<? extends AbstractRequest> buildRequest(ApiMessage requestData) {
|
||||
if (requestData instanceof VoteRequestData)
|
||||
return new VoteRequest.Builder((VoteRequestData) requestData);
|
||||
if (requestData instanceof BeginQuorumEpochRequestData)
|
||||
else if (requestData instanceof BeginQuorumEpochRequestData)
|
||||
return new BeginQuorumEpochRequest.Builder((BeginQuorumEpochRequestData) requestData);
|
||||
if (requestData instanceof EndQuorumEpochRequestData)
|
||||
else if (requestData instanceof EndQuorumEpochRequestData)
|
||||
return new EndQuorumEpochRequest.Builder((EndQuorumEpochRequestData) requestData);
|
||||
if (requestData instanceof FetchRequestData)
|
||||
else if (requestData instanceof FetchRequestData)
|
||||
return new FetchRequest.SimpleBuilder((FetchRequestData) requestData);
|
||||
if (requestData instanceof FetchSnapshotRequestData)
|
||||
else if (requestData instanceof FetchSnapshotRequestData)
|
||||
return new FetchSnapshotRequest.Builder((FetchSnapshotRequestData) requestData);
|
||||
if (requestData instanceof UpdateRaftVoterRequestData)
|
||||
else if (requestData instanceof UpdateRaftVoterRequestData)
|
||||
return new UpdateRaftVoterRequest.Builder((UpdateRaftVoterRequestData) requestData);
|
||||
if (requestData instanceof ApiVersionsRequestData)
|
||||
else if (requestData instanceof AddRaftVoterRequestData)
|
||||
return new AddRaftVoterRequest.Builder((AddRaftVoterRequestData) requestData);
|
||||
else if (requestData instanceof RemoveRaftVoterRequestData)
|
||||
return new RemoveRaftVoterRequest.Builder((RemoveRaftVoterRequestData) requestData);
|
||||
else if (requestData instanceof ApiVersionsRequestData)
|
||||
return new ApiVersionsRequest.Builder((ApiVersionsRequestData) requestData,
|
||||
ApiKeys.API_VERSIONS.oldestVersion(),
|
||||
ApiKeys.API_VERSIONS.latestVersion());
|
||||
throw new IllegalArgumentException("Unexpected type for requestData: " + requestData);
|
||||
else
|
||||
throw new IllegalArgumentException("Unexpected type for requestData: " + requestData);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -180,7 +180,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
private final Logger logger;
|
||||
private final Time time;
|
||||
private final int fetchMaxWaitMs;
|
||||
private final boolean followersAlwaysFlush;
|
||||
private final boolean canBecomeVoter;
|
||||
private final String clusterId;
|
||||
private final Endpoints localListeners;
|
||||
private final SupportedVersionRange localSupportedKRaftVersion;
|
||||
|
@ -229,7 +229,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
* non-participating observer.
|
||||
*
|
||||
* @param nodeDirectoryId the node directory id, cannot be the zero uuid
|
||||
* @param followersAlwaysFlush instruct followers to always fsync when appending to the log
|
||||
* @param canBecomeVoter instruct followers to always fsync when appending to the log
|
||||
*/
|
||||
public KafkaRaftClient(
|
||||
OptionalInt nodeId,
|
||||
|
@ -240,7 +240,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
Time time,
|
||||
ExpirationService expirationService,
|
||||
LogContext logContext,
|
||||
boolean followersAlwaysFlush,
|
||||
boolean canBecomeVoter,
|
||||
String clusterId,
|
||||
Collection<InetSocketAddress> bootstrapServers,
|
||||
Endpoints localListeners,
|
||||
|
@ -258,7 +258,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
time,
|
||||
expirationService,
|
||||
MAX_FETCH_WAIT_MS,
|
||||
followersAlwaysFlush,
|
||||
canBecomeVoter,
|
||||
clusterId,
|
||||
bootstrapServers,
|
||||
localListeners,
|
||||
|
@ -280,7 +280,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
Time time,
|
||||
ExpirationService expirationService,
|
||||
int fetchMaxWaitMs,
|
||||
boolean followersAlwaysFlush,
|
||||
boolean canBecomeVoter,
|
||||
String clusterId,
|
||||
Collection<InetSocketAddress> bootstrapServers,
|
||||
Endpoints localListeners,
|
||||
|
@ -308,7 +308,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
this.localListeners = localListeners;
|
||||
this.localSupportedKRaftVersion = localSupportedKRaftVersion;
|
||||
this.fetchMaxWaitMs = fetchMaxWaitMs;
|
||||
this.followersAlwaysFlush = followersAlwaysFlush;
|
||||
this.canBecomeVoter = canBecomeVoter;
|
||||
this.logger = logContext.logger(KafkaRaftClient.class);
|
||||
this.random = random;
|
||||
this.quorumConfig = quorumConfig;
|
||||
|
@ -1839,7 +1839,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
);
|
||||
}
|
||||
|
||||
if (quorum.isVoter() || followersAlwaysFlush) {
|
||||
if (quorum.isVoter() || canBecomeVoter) {
|
||||
// the leader only requires that voters have flushed their log before sending a Fetch
|
||||
// request. Because of reconfiguration some observers (that are getting added to the
|
||||
// voter set) need to flush the disk because the leader may assume that they are in the
|
||||
|
@ -2291,6 +2291,25 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
);
|
||||
}
|
||||
|
||||
private boolean handleAddVoterResponse(
|
||||
RaftResponse.Inbound responseMetadata,
|
||||
long currentTimeMs
|
||||
) {
|
||||
final AddRaftVoterResponseData data = (AddRaftVoterResponseData) responseMetadata.data();
|
||||
final Errors error = Errors.forCode(data.errorCode());
|
||||
|
||||
/* These error codes indicate the replica was successfully added or the leader is unable to
|
||||
* process the request. In either case, reset the update voter set timer to back off.
|
||||
*/
|
||||
if (error == Errors.NONE || error == Errors.REQUEST_TIMED_OUT ||
|
||||
error == Errors.DUPLICATE_VOTER) {
|
||||
quorum.followerStateOrThrow().resetUpdateVoterSetPeriod(currentTimeMs);
|
||||
return true;
|
||||
} else {
|
||||
return handleUnexpectedError(error, responseMetadata);
|
||||
}
|
||||
}
|
||||
|
||||
private CompletableFuture<RemoveRaftVoterResponseData> handleRemoveVoterRequest(
|
||||
RaftRequest.Inbound requestMetadata,
|
||||
long currentTimeMs
|
||||
|
@ -2334,6 +2353,25 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
);
|
||||
}
|
||||
|
||||
private boolean handleRemoveVoterResponse(
|
||||
RaftResponse.Inbound responseMetadata,
|
||||
long currentTimeMs
|
||||
) {
|
||||
final RemoveRaftVoterResponseData data = (RemoveRaftVoterResponseData) responseMetadata.data();
|
||||
final Errors error = Errors.forCode(data.errorCode());
|
||||
|
||||
/* These error codes indicate the replica was successfully removed or the leader is unable to
|
||||
* process the request. In either case, reset the update voter set timer to back off.
|
||||
*/
|
||||
if (error == Errors.NONE || error == Errors.REQUEST_TIMED_OUT ||
|
||||
error == Errors.VOTER_NOT_FOUND) {
|
||||
quorum.followerStateOrThrow().resetUpdateVoterSetPeriod(currentTimeMs);
|
||||
return true;
|
||||
} else {
|
||||
return handleUnexpectedError(error, responseMetadata);
|
||||
}
|
||||
}
|
||||
|
||||
private CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest(
|
||||
RaftRequest.Inbound requestMetadata,
|
||||
long currentTimeMs
|
||||
|
@ -2629,6 +2667,14 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
handledSuccessfully = handleUpdateVoterResponse(response, currentTimeMs);
|
||||
break;
|
||||
|
||||
case ADD_RAFT_VOTER:
|
||||
handledSuccessfully = handleAddVoterResponse(response, currentTimeMs);
|
||||
break;
|
||||
|
||||
case REMOVE_RAFT_VOTER:
|
||||
handledSuccessfully = handleRemoveVoterResponse(response, currentTimeMs);
|
||||
break;
|
||||
|
||||
default:
|
||||
throw new IllegalArgumentException("Received unexpected response type: " + apiKey);
|
||||
}
|
||||
|
@ -3247,7 +3293,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
logger.info("Transitioning to Prospective state due to fetch timeout");
|
||||
transitionToProspective(currentTimeMs);
|
||||
backoffMs = 0;
|
||||
} else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) {
|
||||
} else if (state.hasUpdateVoterSetPeriodExpired(currentTimeMs)) {
|
||||
final boolean resetUpdateVoterTimer;
|
||||
if (shouldSendUpdateVoteRequest(state)) {
|
||||
var sendResult = maybeSendUpdateVoterRequest(state, currentTimeMs);
|
||||
|
@ -3261,7 +3307,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
if (resetUpdateVoterTimer) {
|
||||
state.resetUpdateVoterPeriod(currentTimeMs);
|
||||
state.resetUpdateVoterSetPeriod(currentTimeMs);
|
||||
}
|
||||
} else {
|
||||
backoffMs = maybeSendFetchToBestNode(state, currentTimeMs);
|
||||
|
@ -3271,13 +3317,56 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
backoffMs,
|
||||
Math.min(
|
||||
state.remainingFetchTimeMs(currentTimeMs),
|
||||
state.remainingUpdateVoterPeriodMs(currentTimeMs)
|
||||
state.remainingUpdateVoterSetPeriodMs(currentTimeMs)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private boolean shouldSendAddOrRemoveVoterRequest(FollowerState state, long currentTimeMs) {
|
||||
/* When the cluster supports reconfiguration, only replicas that can become a voter
|
||||
* and are configured to auto join should attempt to automatically join the voter
|
||||
* set for the configured topic partition.
|
||||
*/
|
||||
return partitionState.lastKraftVersion().isReconfigSupported() && canBecomeVoter &&
|
||||
quorumConfig.autoJoin() && state.hasUpdateVoterSetPeriodExpired(currentTimeMs);
|
||||
}
|
||||
|
||||
private long pollFollowerAsObserver(FollowerState state, long currentTimeMs) {
|
||||
return maybeSendFetchToBestNode(state, currentTimeMs);
|
||||
GracefulShutdown shutdown = this.shutdown.get();
|
||||
final long backoffMs;
|
||||
if (shutdown != null) {
|
||||
// If we are an observer, then we can shutdown immediately. We want to
|
||||
// skip potentially sending any add or remove voter RPCs.
|
||||
backoffMs = 0;
|
||||
} else if (shouldSendAddOrRemoveVoterRequest(state, currentTimeMs)) {
|
||||
final var localReplicaKey = quorum.localReplicaKeyOrThrow();
|
||||
final var voters = partitionState.lastVoterSet();
|
||||
final RequestSendResult sendResult;
|
||||
if (voters.voterIds().contains(localReplicaKey.id())) {
|
||||
/* The replica's id is in the voter set but the replica is not a voter because
|
||||
* the directory id of the voter set entry is different. Remove the old voter.
|
||||
* Local replica is not in the voter set because the replica is an observer.
|
||||
*/
|
||||
final var oldVoter = voters.voterKeys()
|
||||
.stream()
|
||||
.filter(replicaKey -> replicaKey.id() == localReplicaKey.id())
|
||||
.findFirst()
|
||||
.get();
|
||||
sendResult = maybeSendRemoveVoterRequest(state, oldVoter, currentTimeMs);
|
||||
} else {
|
||||
sendResult = maybeSendAddVoterRequest(state, currentTimeMs);
|
||||
}
|
||||
backoffMs = sendResult.timeToWaitMs();
|
||||
if (sendResult.requestSent()) {
|
||||
state.resetUpdateVoterSetPeriod(currentTimeMs);
|
||||
}
|
||||
} else {
|
||||
backoffMs = maybeSendFetchToBestNode(state, currentTimeMs);
|
||||
}
|
||||
return Math.min(
|
||||
backoffMs,
|
||||
state.remainingUpdateVoterSetPeriodMs(currentTimeMs)
|
||||
);
|
||||
}
|
||||
|
||||
private long maybeSendFetchToBestNode(FollowerState state, long currentTimeMs) {
|
||||
|
@ -3329,6 +3418,23 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
);
|
||||
}
|
||||
|
||||
private AddRaftVoterRequestData buildAddVoterRequest() {
|
||||
return RaftUtil.addVoterRequest(
|
||||
clusterId,
|
||||
quorumConfig.requestTimeoutMs(),
|
||||
quorum.localReplicaKeyOrThrow(),
|
||||
localListeners,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
private RemoveRaftVoterRequestData buildRemoveVoterRequest(ReplicaKey replicaKey) {
|
||||
return RaftUtil.removeVoterRequest(
|
||||
clusterId,
|
||||
replicaKey
|
||||
);
|
||||
}
|
||||
|
||||
private RequestSendResult maybeSendUpdateVoterRequest(FollowerState state, long currentTimeMs) {
|
||||
return maybeSendRequest(
|
||||
currentTimeMs,
|
||||
|
@ -3337,6 +3443,29 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
);
|
||||
}
|
||||
|
||||
private RequestSendResult maybeSendAddVoterRequest(
|
||||
FollowerState state,
|
||||
long currentTimeMs
|
||||
) {
|
||||
return maybeSendRequest(
|
||||
currentTimeMs,
|
||||
state.leaderNode(channel.listenerName()),
|
||||
this::buildAddVoterRequest
|
||||
);
|
||||
}
|
||||
|
||||
private RequestSendResult maybeSendRemoveVoterRequest(
|
||||
FollowerState state,
|
||||
ReplicaKey replicaKey,
|
||||
long currentTimeMs
|
||||
) {
|
||||
return maybeSendRequest(
|
||||
currentTimeMs,
|
||||
state.leaderNode(channel.listenerName()),
|
||||
() -> buildRemoveVoterRequest(replicaKey)
|
||||
);
|
||||
}
|
||||
|
||||
private long pollUnattached(long currentTimeMs) {
|
||||
UnattachedState state = quorum.unattachedStateOrThrow();
|
||||
if (quorum.isVoter()) {
|
||||
|
|
|
@ -35,6 +35,7 @@ import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH;
|
|||
import static org.apache.kafka.common.config.ConfigDef.Importance.LOW;
|
||||
import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM;
|
||||
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
|
||||
import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN;
|
||||
import static org.apache.kafka.common.config.ConfigDef.Type.INT;
|
||||
import static org.apache.kafka.common.config.ConfigDef.Type.LIST;
|
||||
|
||||
|
@ -102,6 +103,11 @@ public class QuorumConfig {
|
|||
public static final String QUORUM_RETRY_BACKOFF_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MS_DOC;
|
||||
public static final int DEFAULT_QUORUM_RETRY_BACKOFF_MS = 20;
|
||||
|
||||
public static final String QUORUM_AUTO_JOIN_ENABLE_CONFIG = QUORUM_PREFIX + "auto.join.enable";
|
||||
public static final String QUORUM_AUTO_JOIN_ENABLE_DOC = "Controls whether a KRaft controller should automatically " +
|
||||
"join the cluster metadata partition for its cluster id.";
|
||||
public static final boolean DEFAULT_QUORUM_AUTO_JOIN_ENABLE = false;
|
||||
|
||||
public static final ConfigDef CONFIG_DEF = new ConfigDef()
|
||||
.define(QUORUM_VOTERS_CONFIG, LIST, DEFAULT_QUORUM_VOTERS, new ControllerQuorumVotersValidator(), HIGH, QUORUM_VOTERS_DOC)
|
||||
.define(QUORUM_BOOTSTRAP_SERVERS_CONFIG, LIST, DEFAULT_QUORUM_BOOTSTRAP_SERVERS, new ControllerQuorumBootstrapServersValidator(), HIGH, QUORUM_BOOTSTRAP_SERVERS_DOC)
|
||||
|
@ -110,7 +116,8 @@ public class QuorumConfig {
|
|||
.define(QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG, INT, DEFAULT_QUORUM_ELECTION_BACKOFF_MAX_MS, atLeast(0), HIGH, QUORUM_ELECTION_BACKOFF_MAX_MS_DOC)
|
||||
.define(QUORUM_LINGER_MS_CONFIG, INT, DEFAULT_QUORUM_LINGER_MS, atLeast(0), MEDIUM, QUORUM_LINGER_MS_DOC)
|
||||
.define(QUORUM_REQUEST_TIMEOUT_MS_CONFIG, INT, DEFAULT_QUORUM_REQUEST_TIMEOUT_MS, atLeast(0), MEDIUM, QUORUM_REQUEST_TIMEOUT_MS_DOC)
|
||||
.define(QUORUM_RETRY_BACKOFF_MS_CONFIG, INT, DEFAULT_QUORUM_RETRY_BACKOFF_MS, atLeast(0), LOW, QUORUM_RETRY_BACKOFF_MS_DOC);
|
||||
.define(QUORUM_RETRY_BACKOFF_MS_CONFIG, INT, DEFAULT_QUORUM_RETRY_BACKOFF_MS, atLeast(0), LOW, QUORUM_RETRY_BACKOFF_MS_DOC)
|
||||
.define(QUORUM_AUTO_JOIN_ENABLE_CONFIG, BOOLEAN, DEFAULT_QUORUM_AUTO_JOIN_ENABLE, LOW, QUORUM_AUTO_JOIN_ENABLE_DOC);
|
||||
|
||||
private final List<String> voters;
|
||||
private final List<String> bootstrapServers;
|
||||
|
@ -120,6 +127,7 @@ public class QuorumConfig {
|
|||
private final int electionBackoffMaxMs;
|
||||
private final int fetchTimeoutMs;
|
||||
private final int appendLingerMs;
|
||||
private final boolean autoJoin;
|
||||
|
||||
public QuorumConfig(AbstractConfig abstractConfig) {
|
||||
this.voters = abstractConfig.getList(QUORUM_VOTERS_CONFIG);
|
||||
|
@ -130,6 +138,7 @@ public class QuorumConfig {
|
|||
this.electionBackoffMaxMs = abstractConfig.getInt(QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG);
|
||||
this.fetchTimeoutMs = abstractConfig.getInt(QUORUM_FETCH_TIMEOUT_MS_CONFIG);
|
||||
this.appendLingerMs = abstractConfig.getInt(QUORUM_LINGER_MS_CONFIG);
|
||||
this.autoJoin = abstractConfig.getBoolean(QUORUM_AUTO_JOIN_ENABLE_CONFIG);
|
||||
}
|
||||
|
||||
public List<String> voters() {
|
||||
|
@ -164,6 +173,10 @@ public class QuorumConfig {
|
|||
return appendLingerMs;
|
||||
}
|
||||
|
||||
public boolean autoJoin() {
|
||||
return autoJoin;
|
||||
}
|
||||
|
||||
private static Integer parseVoterId(String idString) {
|
||||
try {
|
||||
return Integer.parseInt(idString);
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.errors.ApiException;
|
||||
import org.apache.kafka.raft.errors.BufferAllocationException;
|
||||
import org.apache.kafka.raft.errors.NotLeaderException;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
|
|
@ -64,6 +64,8 @@ public class RaftUtil {
|
|||
case FETCH_SNAPSHOT -> new FetchSnapshotResponseData().setErrorCode(error.code());
|
||||
case API_VERSIONS -> new ApiVersionsResponseData().setErrorCode(error.code());
|
||||
case UPDATE_RAFT_VOTER -> new UpdateRaftVoterResponseData().setErrorCode(error.code());
|
||||
case ADD_RAFT_VOTER -> new AddRaftVoterResponseData().setErrorCode(error.code());
|
||||
case REMOVE_RAFT_VOTER -> new RemoveRaftVoterResponseData().setErrorCode(error.code());
|
||||
default -> throw new IllegalArgumentException("Received response for unexpected request type: " + apiKey);
|
||||
};
|
||||
}
|
||||
|
@ -524,14 +526,16 @@ public class RaftUtil {
|
|||
String clusterId,
|
||||
int timeoutMs,
|
||||
ReplicaKey voter,
|
||||
Endpoints listeners
|
||||
Endpoints listeners,
|
||||
boolean ackWhenCommitted
|
||||
) {
|
||||
return new AddRaftVoterRequestData()
|
||||
.setClusterId(clusterId)
|
||||
.setTimeoutMs(timeoutMs)
|
||||
.setVoterId(voter.id())
|
||||
.setVoterDirectoryId(voter.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID))
|
||||
.setListeners(listeners.toAddVoterRequest());
|
||||
.setListeners(listeners.toAddVoterRequest())
|
||||
.setAckWhenCommitted(ackWhenCommitted);
|
||||
}
|
||||
|
||||
public static AddRaftVoterResponseData addVoterResponse(
|
||||
|
|
|
@ -22,12 +22,14 @@ import org.apache.kafka.common.Node;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.message.AddRaftVoterResponseData;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.message.FetchRequestData;
|
||||
import org.apache.kafka.common.message.FetchResponseData;
|
||||
import org.apache.kafka.common.message.FetchSnapshotResponseData;
|
||||
import org.apache.kafka.common.message.RemoveRaftVoterResponseData;
|
||||
import org.apache.kafka.common.message.UpdateRaftVoterResponseData;
|
||||
import org.apache.kafka.common.message.VoteResponseData;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
|
@ -36,6 +38,7 @@ import org.apache.kafka.common.protocol.ApiMessage;
|
|||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.requests.AbstractRequest;
|
||||
import org.apache.kafka.common.requests.AbstractResponse;
|
||||
import org.apache.kafka.common.requests.AddRaftVoterResponse;
|
||||
import org.apache.kafka.common.requests.ApiVersionsResponse;
|
||||
import org.apache.kafka.common.requests.BeginQuorumEpochRequest;
|
||||
import org.apache.kafka.common.requests.BeginQuorumEpochResponse;
|
||||
|
@ -44,6 +47,7 @@ import org.apache.kafka.common.requests.EndQuorumEpochResponse;
|
|||
import org.apache.kafka.common.requests.FetchRequest;
|
||||
import org.apache.kafka.common.requests.FetchResponse;
|
||||
import org.apache.kafka.common.requests.FetchSnapshotResponse;
|
||||
import org.apache.kafka.common.requests.RemoveRaftVoterResponse;
|
||||
import org.apache.kafka.common.requests.UpdateRaftVoterResponse;
|
||||
import org.apache.kafka.common.requests.VoteRequest;
|
||||
import org.apache.kafka.common.requests.VoteResponse;
|
||||
|
@ -88,7 +92,9 @@ public class KafkaNetworkChannelTest {
|
|||
ApiKeys.END_QUORUM_EPOCH,
|
||||
ApiKeys.FETCH,
|
||||
ApiKeys.FETCH_SNAPSHOT,
|
||||
ApiKeys.UPDATE_RAFT_VOTER
|
||||
ApiKeys.UPDATE_RAFT_VOTER,
|
||||
ApiKeys.ADD_RAFT_VOTER,
|
||||
ApiKeys.REMOVE_RAFT_VOTER
|
||||
);
|
||||
|
||||
private final int requestTimeoutMs = 30000;
|
||||
|
@ -316,6 +322,21 @@ public class KafkaNetworkChannelTest {
|
|||
Endpoints.empty()
|
||||
);
|
||||
|
||||
case ADD_RAFT_VOTER:
|
||||
return RaftUtil.addVoterRequest(
|
||||
clusterId,
|
||||
requestTimeoutMs,
|
||||
ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID),
|
||||
Endpoints.empty(),
|
||||
true
|
||||
);
|
||||
|
||||
case REMOVE_RAFT_VOTER:
|
||||
return RaftUtil.removeVoterRequest(
|
||||
clusterId,
|
||||
ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)
|
||||
);
|
||||
|
||||
default:
|
||||
throw new AssertionError("Unexpected api " + key);
|
||||
}
|
||||
|
@ -345,6 +366,8 @@ public class KafkaNetworkChannelTest {
|
|||
case FETCH -> new FetchResponseData().setErrorCode(error.code());
|
||||
case FETCH_SNAPSHOT -> new FetchSnapshotResponseData().setErrorCode(error.code());
|
||||
case UPDATE_RAFT_VOTER -> new UpdateRaftVoterResponseData().setErrorCode(error.code());
|
||||
case ADD_RAFT_VOTER -> new AddRaftVoterResponseData().setErrorCode(error.code());
|
||||
case REMOVE_RAFT_VOTER -> new RemoveRaftVoterResponseData().setErrorCode(error.code());
|
||||
default -> throw new AssertionError("Unexpected api " + key);
|
||||
};
|
||||
}
|
||||
|
@ -363,6 +386,10 @@ public class KafkaNetworkChannelTest {
|
|||
code = ((FetchSnapshotResponseData) response).errorCode();
|
||||
} else if (response instanceof UpdateRaftVoterResponseData) {
|
||||
code = ((UpdateRaftVoterResponseData) response).errorCode();
|
||||
} else if (response instanceof AddRaftVoterResponseData) {
|
||||
code = ((AddRaftVoterResponseData) response).errorCode();
|
||||
} else if (response instanceof RemoveRaftVoterResponseData) {
|
||||
code = ((RemoveRaftVoterResponseData) response).errorCode();
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected type for responseData: " + response);
|
||||
}
|
||||
|
@ -383,6 +410,10 @@ public class KafkaNetworkChannelTest {
|
|||
return new FetchSnapshotResponse((FetchSnapshotResponseData) responseData);
|
||||
} else if (responseData instanceof UpdateRaftVoterResponseData) {
|
||||
return new UpdateRaftVoterResponse((UpdateRaftVoterResponseData) responseData);
|
||||
} else if (responseData instanceof AddRaftVoterResponseData) {
|
||||
return new AddRaftVoterResponse((AddRaftVoterResponseData) responseData);
|
||||
} else if (responseData instanceof RemoveRaftVoterResponseData) {
|
||||
return new RemoveRaftVoterResponse((RemoveRaftVoterResponseData) responseData);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected type for responseData: " + responseData);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,323 @@
|
|||
/*
|
||||
* 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.raft;
|
||||
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.utils.BufferSupplier;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey;
|
||||
import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_595_PROTOCOL;
|
||||
import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL;
|
||||
|
||||
public class KafkaRaftClientAutoJoinTest {
|
||||
@Test
|
||||
public void testAutoRemoveOldVoter() throws Exception {
|
||||
final var leader = replicaKey(randomReplicaId(), true);
|
||||
final var oldFollower = replicaKey(leader.id() + 1, true);
|
||||
final var newFollowerKey = replicaKey(oldFollower.id(), true);
|
||||
final int epoch = 1;
|
||||
final var context = new RaftClientTestContext.Builder(
|
||||
newFollowerKey.id(),
|
||||
newFollowerKey.directoryId().get()
|
||||
)
|
||||
.withRaftProtocol(KIP_853_PROTOCOL)
|
||||
.withStartingVoters(
|
||||
VoterSetTest.voterSet(Stream.of(leader, oldFollower)), KRaftVersion.KRAFT_VERSION_1
|
||||
)
|
||||
.withElectedLeader(epoch, leader.id())
|
||||
.withAutoJoin(true)
|
||||
.withCanBecomeVoter(true)
|
||||
.build();
|
||||
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
// the next request should be a remove voter request
|
||||
pollAndDeliverRemoveVoter(context, oldFollower);
|
||||
|
||||
// after sending a remove voter the next request should be a fetch
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
// the replica should send remove voter again because the fetch did not update the voter set
|
||||
pollAndDeliverRemoveVoter(context, oldFollower);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoAddNewVoter() throws Exception {
|
||||
final var leader = replicaKey(randomReplicaId(), true);
|
||||
final var follower = replicaKey(leader.id() + 1, true);
|
||||
final var newVoter = replicaKey(follower.id() + 1, true);
|
||||
final int epoch = 1;
|
||||
final var context = new RaftClientTestContext.Builder(
|
||||
newVoter.id(),
|
||||
newVoter.directoryId().get()
|
||||
)
|
||||
.withRaftProtocol(KIP_853_PROTOCOL)
|
||||
.withStartingVoters(
|
||||
VoterSetTest.voterSet(Stream.of(leader, follower)), KRaftVersion.KRAFT_VERSION_1
|
||||
)
|
||||
.withElectedLeader(epoch, leader.id())
|
||||
.withAutoJoin(true)
|
||||
.withCanBecomeVoter(true)
|
||||
.build();
|
||||
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
// the next request should be an add voter request
|
||||
pollAndSendAddVoter(context, newVoter);
|
||||
|
||||
// expire the add voter request, the next request should be a fetch
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
// the replica should send add voter again because the completed fetch
|
||||
// did not update the voter set, and its timer has expired
|
||||
final var addVoterRequest = pollAndSendAddVoter(context, newVoter);
|
||||
|
||||
// deliver the add voter response, this is possible before a completed fetch because of KIP-1186
|
||||
context.deliverResponse(
|
||||
addVoterRequest.correlationId(),
|
||||
addVoterRequest.destination(),
|
||||
RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message())
|
||||
);
|
||||
|
||||
// verify the replica can perform a fetch to commit the new voter set
|
||||
pollAndDeliverFetchToUpdateVoterSet(
|
||||
context,
|
||||
epoch,
|
||||
VoterSetTest.voterSet(Stream.of(leader, newVoter))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testObserverRemovesOldVoterAndAutoJoins() throws Exception {
|
||||
final var leader = replicaKey(randomReplicaId(), true);
|
||||
final var oldFollower = replicaKey(leader.id() + 1, true);
|
||||
final var newFollowerKey = replicaKey(oldFollower.id(), true);
|
||||
final int epoch = 1;
|
||||
final var context = new RaftClientTestContext.Builder(
|
||||
newFollowerKey.id(),
|
||||
newFollowerKey.directoryId().get()
|
||||
)
|
||||
.withRaftProtocol(KIP_853_PROTOCOL)
|
||||
.withStartingVoters(
|
||||
VoterSetTest.voterSet(Stream.of(leader, oldFollower)), KRaftVersion.KRAFT_VERSION_1
|
||||
)
|
||||
.withElectedLeader(epoch, leader.id())
|
||||
.withAutoJoin(true)
|
||||
.withCanBecomeVoter(true)
|
||||
.build();
|
||||
|
||||
// advance time and complete a fetch to trigger the remove voter request
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
// the next request should be a remove voter request
|
||||
pollAndDeliverRemoveVoter(context, oldFollower);
|
||||
|
||||
// after sending a remove voter the next request should be a fetch
|
||||
// this fetch will remove the old follower from the voter set
|
||||
pollAndDeliverFetchToUpdateVoterSet(
|
||||
context,
|
||||
epoch,
|
||||
VoterSetTest.voterSet(Stream.of(leader))
|
||||
);
|
||||
|
||||
// advance time and complete a fetch to trigger the add voter request
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
// the next request should be an add voter request
|
||||
final var addVoterRequest = pollAndSendAddVoter(context, newFollowerKey);
|
||||
|
||||
// deliver the add voter response, this is possible before a completed fetch because of KIP-1186
|
||||
context.deliverResponse(
|
||||
addVoterRequest.correlationId(),
|
||||
addVoterRequest.destination(),
|
||||
RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message())
|
||||
);
|
||||
|
||||
// verify the replica can perform a fetch to commit the new voter set
|
||||
pollAndDeliverFetchToUpdateVoterSet(
|
||||
context,
|
||||
epoch,
|
||||
VoterSetTest.voterSet(Stream.of(leader, newFollowerKey))
|
||||
);
|
||||
|
||||
// advance time and complete a fetch and expire the update voter set timer
|
||||
// the next request should be a fetch because the log voter configuration is up-to-date
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
context.pollUntilRequest();
|
||||
context.assertSentFetchRequest();
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testObserversDoNotAutoJoin() throws Exception {
|
||||
final var leader = replicaKey(randomReplicaId(), true);
|
||||
final var follower = replicaKey(leader.id() + 1, true);
|
||||
final var newObserver = replicaKey(follower.id() + 1, true);
|
||||
final int epoch = 1;
|
||||
final var context = new RaftClientTestContext.Builder(
|
||||
newObserver.id(),
|
||||
newObserver.directoryId().get()
|
||||
)
|
||||
.withRaftProtocol(KIP_853_PROTOCOL)
|
||||
.withStartingVoters(
|
||||
VoterSetTest.voterSet(Stream.of(leader, follower)), KRaftVersion.KRAFT_VERSION_1
|
||||
)
|
||||
.withElectedLeader(epoch, leader.id())
|
||||
.withAutoJoin(true)
|
||||
.withCanBecomeVoter(false)
|
||||
.build();
|
||||
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
|
||||
// When canBecomeVoter == false, the replica should not send an add voter request
|
||||
final var fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testObserverDoesNotAddItselfWhenAutoJoinDisabled() throws Exception {
|
||||
final var leader = replicaKey(randomReplicaId(), true);
|
||||
final var follower = replicaKey(leader.id() + 1, true);
|
||||
final var observer = replicaKey(follower.id() + 1, true);
|
||||
final int epoch = 1;
|
||||
final var context = new RaftClientTestContext.Builder(
|
||||
observer.id(),
|
||||
observer.directoryId().get()
|
||||
)
|
||||
.withRaftProtocol(KIP_853_PROTOCOL)
|
||||
.withStartingVoters(
|
||||
VoterSetTest.voterSet(Stream.of(leader, follower)), KRaftVersion.KRAFT_VERSION_1
|
||||
)
|
||||
.withElectedLeader(epoch, leader.id())
|
||||
.withAutoJoin(false)
|
||||
.withCanBecomeVoter(true)
|
||||
.build();
|
||||
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
|
||||
// When autoJoin == false, the replica should not send an add voter request
|
||||
final var fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testObserverDoesNotAutoJoinWithKRaftVersion0() throws Exception {
|
||||
final var leader = replicaKey(randomReplicaId(), true);
|
||||
final var follower = replicaKey(leader.id() + 1, true);
|
||||
final var observer = replicaKey(follower.id() + 1, true);
|
||||
final int epoch = 1;
|
||||
final var context = new RaftClientTestContext.Builder(
|
||||
observer.id(),
|
||||
observer.directoryId().get()
|
||||
)
|
||||
.withRaftProtocol(KIP_595_PROTOCOL)
|
||||
.withStartingVoters(
|
||||
VoterSetTest.voterSet(Stream.of(leader, follower)), KRaftVersion.KRAFT_VERSION_0
|
||||
)
|
||||
.withElectedLeader(epoch, leader.id())
|
||||
.withAutoJoin(true)
|
||||
.withCanBecomeVoter(true)
|
||||
.build();
|
||||
|
||||
context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
|
||||
// When kraft.version == 0, the replica should not send an add voter request
|
||||
final var fetchRequest = context.assertSentFetchRequest();
|
||||
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
}
|
||||
|
||||
private void pollAndDeliverRemoveVoter(
|
||||
RaftClientTestContext context,
|
||||
ReplicaKey oldFollower
|
||||
) throws Exception {
|
||||
context.pollUntilRequest();
|
||||
final var removeRequest = context.assertSentRemoveVoterRequest(oldFollower);
|
||||
context.deliverResponse(
|
||||
removeRequest.correlationId(),
|
||||
removeRequest.destination(),
|
||||
RaftUtil.removeVoterResponse(Errors.NONE, Errors.NONE.message())
|
||||
);
|
||||
}
|
||||
|
||||
private RaftRequest.Outbound pollAndSendAddVoter(
|
||||
RaftClientTestContext context,
|
||||
ReplicaKey newVoter
|
||||
) throws Exception {
|
||||
context.pollUntilRequest();
|
||||
return context.assertSentAddVoterRequest(
|
||||
newVoter,
|
||||
context.client.quorum().localVoterNodeOrThrow().listeners()
|
||||
);
|
||||
}
|
||||
|
||||
private void pollAndDeliverFetchToUpdateVoterSet(
|
||||
RaftClientTestContext context,
|
||||
int epoch,
|
||||
VoterSet newVoterSet
|
||||
) throws Exception {
|
||||
context.pollUntilRequest();
|
||||
final var fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(
|
||||
fetchRequest,
|
||||
epoch,
|
||||
context.log.endOffset().offset(),
|
||||
context.log.lastFetchedEpoch(),
|
||||
context.client.highWatermark()
|
||||
);
|
||||
// deliver the fetch response with the updated voter set
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
fetchRequest.destination().id(),
|
||||
MemoryRecords.withVotersRecord(
|
||||
context.log.endOffset().offset(),
|
||||
context.time.milliseconds(),
|
||||
epoch,
|
||||
BufferSupplier.NO_CACHING.get(300),
|
||||
newVoterSet.toVotersRecord((short) 0)
|
||||
),
|
||||
context.log.endOffset().offset() + 1,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to update the replica's voter set
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
private int randomReplicaId() {
|
||||
return ThreadLocalRandom.current().nextInt(1025);
|
||||
}
|
||||
}
|
|
@ -78,8 +78,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
|||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class KafkaRaftClientReconfigTest {
|
||||
private static final int NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD = 1;
|
||||
|
||||
@Test
|
||||
public void testLeaderWritesBootstrapRecords() throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
|
@ -2225,28 +2223,8 @@ public class KafkaRaftClientReconfigTest {
|
|||
.build();
|
||||
|
||||
// waiting for FETCH requests until the UpdateRaftVoter request is sent
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
|
@ -2298,28 +2276,8 @@ public class KafkaRaftClientReconfigTest {
|
|||
.build();
|
||||
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
|
@ -2389,28 +2347,8 @@ public class KafkaRaftClientReconfigTest {
|
|||
.build();
|
||||
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
|
@ -2437,28 +2375,8 @@ public class KafkaRaftClientReconfigTest {
|
|||
context.pollUntilResponse();
|
||||
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0, context.client.highWatermark());
|
||||
context.advanceTimeAndCompleteFetch(newEpoch, voter1.id(), true);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
newEpoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
updateRequest = context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
|
@ -2723,29 +2641,9 @@ public class KafkaRaftClientReconfigTest {
|
|||
.build();
|
||||
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true);
|
||||
|
||||
// update voter should not be sent because the local listener is not different from the voter set
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
|
@ -2784,26 +2682,7 @@ public class KafkaRaftClientReconfigTest {
|
|||
.build();
|
||||
|
||||
// waiting up to the last FETCH request before the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
context.advanceTimeAndCompleteFetch(epoch, voter1.id(), false);
|
||||
|
||||
// expect one last FETCH request
|
||||
context.pollUntilRequest();
|
||||
|
@ -2864,28 +2743,8 @@ public class KafkaRaftClientReconfigTest {
|
|||
.build();
|
||||
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark());
|
||||
context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
|
|
|
@ -3786,7 +3786,7 @@ class KafkaRaftClientTest {
|
|||
|
||||
@ParameterizedTest
|
||||
@CsvSource({ "true, true", "true, false", "false, true", "false, false" })
|
||||
public void testObserverReplication(boolean withKip853Rpc, boolean alwaysFlush) throws Exception {
|
||||
public void testObserverReplication(boolean withKip853Rpc, boolean canBecomeVoter) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
|
@ -3795,7 +3795,7 @@ class KafkaRaftClientTest {
|
|||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.withAlwaysFlush(alwaysFlush)
|
||||
.withCanBecomeVoter(canBecomeVoter)
|
||||
.build();
|
||||
context.assertElectedLeader(epoch, otherNodeId);
|
||||
|
||||
|
@ -3812,7 +3812,7 @@ class KafkaRaftClientTest {
|
|||
|
||||
context.client.poll();
|
||||
assertEquals(2L, context.log.endOffset().offset());
|
||||
long firstUnflushedOffset = alwaysFlush ? 2L : 0L;
|
||||
long firstUnflushedOffset = canBecomeVoter ? 2L : 0L;
|
||||
assertEquals(firstUnflushedOffset, context.log.firstUnflushedOffset());
|
||||
}
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@ public class QuorumConfigTest {
|
|||
assertInvalidConfig(Map.of(QuorumConfig.QUORUM_LINGER_MS_CONFIG, "-1"));
|
||||
assertInvalidConfig(Map.of(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG, "-1"));
|
||||
assertInvalidConfig(Map.of(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG, "-1"));
|
||||
assertInvalidConfig(Map.of(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, "-1"));
|
||||
}
|
||||
|
||||
private void assertInvalidConfig(Map<String, Object> overrideConfig) {
|
||||
|
@ -46,6 +47,7 @@ public class QuorumConfigTest {
|
|||
props.put(QuorumConfig.QUORUM_LINGER_MS_CONFIG, "10");
|
||||
props.put(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG, "10");
|
||||
props.put(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG, "10");
|
||||
props.put(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, true);
|
||||
|
||||
props.putAll(overrideConfig);
|
||||
|
||||
|
|
|
@ -143,10 +143,12 @@ public final class RaftClientTestContext {
|
|||
// Used to determine which RPC request and response to construct
|
||||
final RaftProtocol raftProtocol;
|
||||
// Used to determine if the local kraft client was configured to always flush
|
||||
final boolean alwaysFlush;
|
||||
final boolean canBecomeVoter;
|
||||
|
||||
private final List<RaftResponse.Outbound> sentResponses = new ArrayList<>();
|
||||
|
||||
private static final int NUMBER_FETCH_TIMEOUTS_IN_UPDATE_VOTER_SET_PERIOD = 1;
|
||||
|
||||
public static final class Builder {
|
||||
static final int DEFAULT_ELECTION_TIMEOUT_MS = 10000;
|
||||
|
||||
|
@ -177,10 +179,11 @@ public final class RaftClientTestContext {
|
|||
private MemoryPool memoryPool = MemoryPool.NONE;
|
||||
private Optional<List<InetSocketAddress>> bootstrapServers = Optional.empty();
|
||||
private RaftProtocol raftProtocol = RaftProtocol.KIP_595_PROTOCOL;
|
||||
private boolean alwaysFlush = false;
|
||||
private boolean canBecomeVoter = false;
|
||||
private VoterSet startingVoters = VoterSet.empty();
|
||||
private Endpoints localListeners = Endpoints.empty();
|
||||
private boolean isStartingVotersStatic = false;
|
||||
private boolean autoJoin = false;
|
||||
|
||||
public Builder(int localId, Set<Integer> staticVoters) {
|
||||
this(OptionalInt.of(localId), staticVoters);
|
||||
|
@ -309,8 +312,8 @@ public final class RaftClientTestContext {
|
|||
return this;
|
||||
}
|
||||
|
||||
Builder withAlwaysFlush(boolean alwaysFlush) {
|
||||
this.alwaysFlush = alwaysFlush;
|
||||
Builder withCanBecomeVoter(boolean canBecomeVoter) {
|
||||
this.canBecomeVoter = canBecomeVoter;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -376,6 +379,11 @@ public final class RaftClientTestContext {
|
|||
return this;
|
||||
}
|
||||
|
||||
Builder withAutoJoin(boolean autoJoin) {
|
||||
this.autoJoin = autoJoin;
|
||||
return this;
|
||||
}
|
||||
|
||||
public RaftClientTestContext build() throws IOException {
|
||||
Metrics metrics = new Metrics(time);
|
||||
MockNetworkChannel channel = new MockNetworkChannel();
|
||||
|
@ -404,13 +412,14 @@ public final class RaftClientTestContext {
|
|||
Endpoints.empty() :
|
||||
this.localListeners;
|
||||
|
||||
Map<String, Integer> configMap = new HashMap<>();
|
||||
Map<String, Object> configMap = new HashMap<>();
|
||||
configMap.put(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs);
|
||||
configMap.put(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG, RETRY_BACKOFF_MS);
|
||||
configMap.put(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG, electionTimeoutMs);
|
||||
configMap.put(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG, ELECTION_BACKOFF_MAX_MS);
|
||||
configMap.put(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG, FETCH_TIMEOUT_MS);
|
||||
configMap.put(QuorumConfig.QUORUM_LINGER_MS_CONFIG, appendLingerMs);
|
||||
configMap.put(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, autoJoin);
|
||||
QuorumConfig quorumConfig = new QuorumConfig(new AbstractConfig(QuorumConfig.CONFIG_DEF, configMap));
|
||||
|
||||
List<InetSocketAddress> computedBootstrapServers = bootstrapServers.orElseGet(() -> {
|
||||
|
@ -436,7 +445,7 @@ public final class RaftClientTestContext {
|
|||
time,
|
||||
new MockExpirationService(time),
|
||||
FETCH_MAX_WAIT_MS,
|
||||
alwaysFlush,
|
||||
canBecomeVoter,
|
||||
clusterId,
|
||||
computedBootstrapServers,
|
||||
localListeners,
|
||||
|
@ -474,7 +483,7 @@ public final class RaftClientTestContext {
|
|||
.boxed()
|
||||
.collect(Collectors.toSet()),
|
||||
raftProtocol,
|
||||
alwaysFlush,
|
||||
canBecomeVoter,
|
||||
metrics,
|
||||
externalKRaftMetrics,
|
||||
listener
|
||||
|
@ -503,7 +512,7 @@ public final class RaftClientTestContext {
|
|||
VoterSet startingVoters,
|
||||
Set<Integer> bootstrapIds,
|
||||
RaftProtocol raftProtocol,
|
||||
boolean alwaysFlush,
|
||||
boolean canBecomeVoter,
|
||||
Metrics metrics,
|
||||
ExternalKRaftMetrics externalKRaftMetrics,
|
||||
MockListener listener
|
||||
|
@ -521,7 +530,7 @@ public final class RaftClientTestContext {
|
|||
this.startingVoters = startingVoters;
|
||||
this.bootstrapIds = bootstrapIds;
|
||||
this.raftProtocol = raftProtocol;
|
||||
this.alwaysFlush = alwaysFlush;
|
||||
this.canBecomeVoter = canBecomeVoter;
|
||||
this.metrics = metrics;
|
||||
this.externalKRaftMetrics = externalKRaftMetrics;
|
||||
this.listener = listener;
|
||||
|
@ -949,6 +958,51 @@ public final class RaftClientTestContext {
|
|||
channel.mockReceive(new RaftResponse.Inbound(correlationId, versionedResponse, source));
|
||||
}
|
||||
|
||||
/**
|
||||
* Advance time and complete an empty fetch to reset the fetch timer.
|
||||
* This is used to expire the update voter set timer without also expiring the fetch timer,
|
||||
* which is needed for add, remove, and update voter tests.
|
||||
* For voters and observers, polling after exiting this method expires the update voter set timer.
|
||||
* @param epoch - the current epoch
|
||||
* @param leaderId - the leader id
|
||||
* @param expireUpdateVoterSetTimer - if true, advance time again to expire this timer
|
||||
*/
|
||||
void advanceTimeAndCompleteFetch(
|
||||
int epoch,
|
||||
int leaderId,
|
||||
boolean expireUpdateVoterSetTimer
|
||||
) throws Exception {
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_VOTER_SET_PERIOD; i++) {
|
||||
time.sleep(fetchTimeoutMs - 1);
|
||||
pollUntilRequest();
|
||||
final var fetchRequest = assertSentFetchRequest();
|
||||
assertFetchRequestData(
|
||||
fetchRequest,
|
||||
epoch,
|
||||
log.endOffset().offset(),
|
||||
log.lastFetchedEpoch(),
|
||||
client.highWatermark()
|
||||
);
|
||||
|
||||
deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
fetchResponse(
|
||||
epoch,
|
||||
leaderId,
|
||||
MemoryRecords.EMPTY,
|
||||
log.endOffset().offset(),
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
client.poll();
|
||||
}
|
||||
if (expireUpdateVoterSetTimer) {
|
||||
time.sleep(fetchTimeoutMs - 1);
|
||||
}
|
||||
}
|
||||
|
||||
List<RaftRequest.Outbound> assertSentBeginQuorumEpochRequest(int epoch, Set<Integer> destinationIds) {
|
||||
List<RaftRequest.Outbound> requests = collectBeginEpochRequests(epoch);
|
||||
assertEquals(destinationIds.size(), requests.size());
|
||||
|
@ -1259,6 +1313,26 @@ public final class RaftClientTestContext {
|
|||
return sentRequests.get(0);
|
||||
}
|
||||
|
||||
RaftRequest.Outbound assertSentAddVoterRequest(
|
||||
ReplicaKey replicaKey,
|
||||
Endpoints endpoints
|
||||
) {
|
||||
final var sentRequests = channel.drainSentRequests(Optional.of(ApiKeys.ADD_RAFT_VOTER));
|
||||
assertEquals(1, sentRequests.size());
|
||||
|
||||
final var request = sentRequests.get(0);
|
||||
assertInstanceOf(AddRaftVoterRequestData.class, request.data());
|
||||
|
||||
final var addRaftVoterRequestData = (AddRaftVoterRequestData) request.data();
|
||||
assertEquals(clusterId, addRaftVoterRequestData.clusterId());
|
||||
assertEquals(replicaKey.id(), addRaftVoterRequestData.voterId());
|
||||
assertEquals(replicaKey.directoryId().get(), addRaftVoterRequestData.voterDirectoryId());
|
||||
assertEquals(endpoints, Endpoints.fromAddVoterRequest(addRaftVoterRequestData.listeners()));
|
||||
assertEquals(false, addRaftVoterRequestData.ackWhenCommitted());
|
||||
|
||||
return request;
|
||||
}
|
||||
|
||||
AddRaftVoterResponseData assertSentAddVoterResponse(Errors error) {
|
||||
List<RaftResponse.Outbound> sentResponses = drainSentResponses(ApiKeys.ADD_RAFT_VOTER);
|
||||
assertEquals(1, sentResponses.size());
|
||||
|
@ -1272,6 +1346,23 @@ public final class RaftClientTestContext {
|
|||
return addVoterResponse;
|
||||
}
|
||||
|
||||
RaftRequest.Outbound assertSentRemoveVoterRequest(
|
||||
ReplicaKey replicaKey
|
||||
) {
|
||||
final var sentRequests = channel.drainSentRequests(Optional.of(ApiKeys.REMOVE_RAFT_VOTER));
|
||||
assertEquals(1, sentRequests.size());
|
||||
|
||||
final var request = sentRequests.get(0);
|
||||
assertInstanceOf(RemoveRaftVoterRequestData.class, request.data());
|
||||
|
||||
final var removeRaftVoterRequestData = (RemoveRaftVoterRequestData) request.data();
|
||||
assertEquals(clusterId, removeRaftVoterRequestData.clusterId());
|
||||
assertEquals(replicaKey.id(), removeRaftVoterRequestData.voterId());
|
||||
assertEquals(replicaKey.directoryId().get(), removeRaftVoterRequestData.voterDirectoryId());
|
||||
|
||||
return request;
|
||||
}
|
||||
|
||||
RemoveRaftVoterResponseData assertSentRemoveVoterResponse(Errors error) {
|
||||
List<RaftResponse.Outbound> sentResponses = drainSentResponses(ApiKeys.REMOVE_RAFT_VOTER);
|
||||
assertEquals(1, sentResponses.size());
|
||||
|
@ -1707,7 +1798,7 @@ public final class RaftClientTestContext {
|
|||
|
||||
// Assert that voters have flushed up to the fetch offset
|
||||
if ((localId.isPresent() && startingVoters.voterIds().contains(localId.getAsInt())) ||
|
||||
alwaysFlush
|
||||
canBecomeVoter
|
||||
) {
|
||||
assertEquals(
|
||||
log.firstUnflushedOffset(),
|
||||
|
@ -1921,7 +2012,8 @@ public final class RaftClientTestContext {
|
|||
clusterId,
|
||||
timeoutMs,
|
||||
voter,
|
||||
endpoints
|
||||
endpoints,
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.kafka.common.compress.Compression;
|
|||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
||||
import org.apache.kafka.common.message.AddRaftVoterRequestData;
|
||||
import org.apache.kafka.common.message.AddRaftVoterRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.AddRaftVoterResponseData;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
|
||||
|
@ -42,6 +43,7 @@ import org.apache.kafka.common.message.FetchSnapshotRequestData;
|
|||
import org.apache.kafka.common.message.FetchSnapshotRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.FetchSnapshotResponseData;
|
||||
import org.apache.kafka.common.message.FetchSnapshotResponseDataJsonConverter;
|
||||
import org.apache.kafka.common.message.RemoveRaftVoterResponseData;
|
||||
import org.apache.kafka.common.message.VoteRequestData;
|
||||
import org.apache.kafka.common.message.VoteRequestDataJsonConverter;
|
||||
import org.apache.kafka.common.message.VoteResponseData;
|
||||
|
@ -93,6 +95,10 @@ public class RaftUtilTest {
|
|||
RaftUtil.errorResponse(ApiKeys.FETCH, Errors.NONE));
|
||||
assertEquals(new FetchSnapshotResponseData().setErrorCode(Errors.NONE.code()),
|
||||
RaftUtil.errorResponse(ApiKeys.FETCH_SNAPSHOT, Errors.NONE));
|
||||
assertEquals(new AddRaftVoterResponseData().setErrorCode(Errors.NONE.code()),
|
||||
RaftUtil.errorResponse(ApiKeys.ADD_RAFT_VOTER, Errors.NONE));
|
||||
assertEquals(new RemoveRaftVoterResponseData().setErrorCode(Errors.NONE.code()),
|
||||
RaftUtil.errorResponse(ApiKeys.REMOVE_RAFT_VOTER, Errors.NONE));
|
||||
assertThrows(IllegalArgumentException.class, () -> RaftUtil.errorResponse(ApiKeys.PRODUCE, Errors.NONE));
|
||||
}
|
||||
|
||||
|
|
|
@ -48,7 +48,7 @@ public class InFlightState {
|
|||
private String memberId;
|
||||
// The state of the records before the transition. In case we need to revert an in-flight state, we revert the above
|
||||
// attributes of InFlightState to this state, namely - state, deliveryCount and memberId.
|
||||
private InFlightState rollbackState;
|
||||
private RollbackState rollbackState;
|
||||
// The timer task for the acquisition lock timeout.
|
||||
private AcquisitionLockTimerTask acquisitionLockTimeoutTask;
|
||||
// The boolean determines if the record has achieved a terminal state of ARCHIVED from which it cannot transition
|
||||
|
@ -205,7 +205,7 @@ public class InFlightState {
|
|||
InFlightState currentState = new InFlightState(state, deliveryCount, memberId, acquisitionLockTimeoutTask);
|
||||
InFlightState updatedState = tryUpdateState(newState, ops, maxDeliveryCount, newMemberId);
|
||||
if (updatedState != null) {
|
||||
rollbackState = currentState;
|
||||
rollbackState = new RollbackState(currentState, maxDeliveryCount);
|
||||
}
|
||||
return updatedState;
|
||||
}
|
||||
|
@ -224,16 +224,23 @@ public class InFlightState {
|
|||
rollbackState = null;
|
||||
return;
|
||||
}
|
||||
InFlightState previousState = rollbackState.state();
|
||||
// Check is acquisition lock timeout task is expired then mark the message as Available.
|
||||
if (acquisitionLockTimeoutTask != null && acquisitionLockTimeoutTask.hasExpired()) {
|
||||
state = RecordState.AVAILABLE;
|
||||
// If the acquisition lock timeout task has expired, we should mark the record as available.
|
||||
// However, if the delivery count has reached the maximum delivery count, we should archive the record.
|
||||
state = previousState.deliveryCount() >= rollbackState.maxDeliveryCount ?
|
||||
RecordState.ARCHIVED : RecordState.AVAILABLE;
|
||||
memberId = EMPTY_MEMBER_ID;
|
||||
cancelAndClearAcquisitionLockTimeoutTask();
|
||||
} else {
|
||||
state = rollbackState.state;
|
||||
memberId = rollbackState.memberId;
|
||||
state = previousState.state();
|
||||
memberId = previousState.memberId();
|
||||
}
|
||||
deliveryCount = rollbackState.deliveryCount();
|
||||
// Do not revert the delivery count as the delivery count should not be reverted for the failed
|
||||
// state transition. However, in the current implementation, the delivery count is only incremented
|
||||
// when the state is updated to Acquired, hence reverting the delivery count is not needed when
|
||||
// the state transition fails.
|
||||
rollbackState = null;
|
||||
}
|
||||
|
||||
|
@ -271,4 +278,12 @@ public class InFlightState {
|
|||
", memberId=" + memberId +
|
||||
")";
|
||||
}
|
||||
|
||||
/**
|
||||
* This record is used to store the state before the transition. It is used to revert the state if the transition fails.
|
||||
* @param state The state of the records before the transition.
|
||||
* @param maxDeliveryCount The maximum delivery count for the record.
|
||||
*/
|
||||
private record RollbackState(InFlightState state, int maxDeliveryCount) {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.kafka.common.serialization.Serdes;
|
|||
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||
import org.apache.kafka.common.serialization.StringSerializer;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.streams.GroupProtocol;
|
||||
import org.apache.kafka.streams.KafkaStreams;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
|
@ -50,7 +51,8 @@ import org.junit.jupiter.api.Tag;
|
|||
import org.junit.jupiter.api.TestInfo;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -59,6 +61,7 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
@ -71,6 +74,7 @@ import java.util.regex.Matcher;
|
|||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.kafka.streams.KafkaStreams.State.ERROR;
|
||||
import static org.apache.kafka.streams.KafkaStreams.State.REBALANCING;
|
||||
|
@ -121,7 +125,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
CLUSTER.createTopic(outputTopic, 1, 1);
|
||||
}
|
||||
|
||||
private Properties createStreamsConfig(final String topologyOptimization) {
|
||||
private Properties createStreamsConfig(final String topologyOptimization, final boolean useNewProtocol) {
|
||||
final Properties streamsConfiguration = new Properties();
|
||||
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
|
||||
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||
|
@ -131,9 +135,23 @@ public class KStreamRepartitionIntegrationTest {
|
|||
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
|
||||
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||
streamsConfiguration.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, topologyOptimization);
|
||||
|
||||
if (useNewProtocol) {
|
||||
streamsConfiguration.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name().toLowerCase(Locale.getDefault()));
|
||||
}
|
||||
|
||||
return streamsConfiguration;
|
||||
}
|
||||
|
||||
private static Stream<Arguments> protocolAndOptimizationParameters() {
|
||||
return Stream.of(
|
||||
Arguments.of(StreamsConfig.OPTIMIZE, false), // OPTIMIZE with CLASSIC protocol
|
||||
Arguments.of(StreamsConfig.OPTIMIZE, true), // OPTIMIZE with STREAMS protocol
|
||||
Arguments.of(StreamsConfig.NO_OPTIMIZATION, false), // NO_OPTIMIZATION with CLASSIC protocol
|
||||
Arguments.of(StreamsConfig.NO_OPTIMIZATION, true) // NO_OPTIMIZATION with STREAMS protocol
|
||||
);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void whenShuttingDown() throws IOException {
|
||||
kafkaStreamsInstances.stream()
|
||||
|
@ -144,8 +162,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDoNotMatchSourceTopicWhenJoining(final String topologyOptimization) throws InterruptedException {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDoNotMatchSourceTopicWhenJoining(final String topologyOptimization, final boolean useNewProtocol) throws InterruptedException {
|
||||
final int topicBNumberOfPartitions = 6;
|
||||
final String inputTopicRepartitionName = "join-repartition-test";
|
||||
final AtomicReference<Throwable> expectedThrowable = new AtomicReference<>();
|
||||
|
@ -167,10 +185,12 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.join(topicBStream, (value1, value2) -> value2, JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofSeconds(10)))
|
||||
.to(outputTopic);
|
||||
|
||||
final Properties streamsConfiguration = createStreamsConfig(topologyOptimization);
|
||||
final Properties streamsConfiguration = createStreamsConfig(topologyOptimization, useNewProtocol);
|
||||
try (final KafkaStreams ks = new KafkaStreams(builder.build(streamsConfiguration), streamsConfiguration)) {
|
||||
ks.setUncaughtExceptionHandler(exception -> {
|
||||
expectedThrowable.set(exception);
|
||||
System.out.println(String.format("[%s Protocol] Exception caught: %s",
|
||||
useNewProtocol ? "STREAMS" : "CLASSIC", exception.getMessage()));
|
||||
return SHUTDOWN_CLIENT;
|
||||
});
|
||||
ks.start();
|
||||
|
@ -186,8 +206,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldDeductNumberOfPartitionsFromRepartitionOperation(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldDeductNumberOfPartitionsFromRepartitionOperation(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String topicBMapperName = "topic-b-mapper";
|
||||
final int topicBNumberOfPartitions = 6;
|
||||
final String inputTopicRepartitionName = "join-repartition-test";
|
||||
|
@ -220,7 +240,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.join(topicBStream, (value1, value2) -> value2, JoinWindows.of(Duration.ofSeconds(10)))
|
||||
.to(outputTopic);
|
||||
|
||||
final Properties streamsConfiguration = createStreamsConfig(topologyOptimization);
|
||||
final Properties streamsConfiguration = createStreamsConfig(topologyOptimization, useNewProtocol);
|
||||
builder.build(streamsConfiguration);
|
||||
|
||||
startStreams(builder, streamsConfiguration);
|
||||
|
@ -239,8 +259,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartitionOperation(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartitionOperation(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String topicBRepartitionedName = "topic-b-scale-up";
|
||||
final String inputTopicRepartitionedName = "input-topic-scale-up";
|
||||
|
||||
|
@ -278,7 +298,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.join(topicBStream, (value1, value2) -> value2, JoinWindows.of(Duration.ofSeconds(10)))
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
assertEquals(4, getNumberOfPartitionsForTopic(toRepartitionTopicName(topicBRepartitionedName)));
|
||||
assertEquals(4, getNumberOfPartitionsForTopic(toRepartitionTopicName(inputTopicRepartitionedName)));
|
||||
|
@ -291,8 +311,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldRepartitionToMultiplePartitions(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldRepartitionToMultiplePartitions(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String repartitionName = "broadcasting-partitioner-test";
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
final AtomicInteger partitionerInvocation = new AtomicInteger(0);
|
||||
|
@ -334,7 +354,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.repartition(repartitioned)
|
||||
.to(broadcastingOutputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
final String topic = toRepartitionTopicName(repartitionName);
|
||||
|
||||
|
@ -360,8 +380,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldUseStreamPartitionerForRepartitionOperation(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldUseStreamPartitionerForRepartitionOperation(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final int partition = 1;
|
||||
final String repartitionName = "partitioner-test";
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
@ -387,7 +407,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.repartition(repartitioned)
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
final String topic = toRepartitionTopicName(repartitionName);
|
||||
|
||||
|
@ -402,8 +422,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldPerformSelectKeyWithRepartitionOperation(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldPerformSelectKeyWithRepartitionOperation(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
sendEvents(
|
||||
|
@ -421,7 +441,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.repartition()
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
validateReceivedMessages(
|
||||
new IntegerDeserializer(),
|
||||
|
@ -438,8 +458,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String repartitionName = "dummy";
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
|
@ -457,7 +477,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.repartition(Repartitioned.as(repartitionName))
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
validateReceivedMessages(
|
||||
new IntegerDeserializer(),
|
||||
|
@ -475,8 +495,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKeySelector(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKeySelector(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String repartitionedName = "new-key";
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
|
@ -501,7 +521,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.toStream()
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
validateReceivedMessages(
|
||||
new StringDeserializer(),
|
||||
|
@ -521,8 +541,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String repartitionName = "new-partitions";
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
|
@ -543,7 +563,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.toStream()
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
validateReceivedMessages(
|
||||
new IntegerDeserializer(),
|
||||
|
@ -561,8 +581,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNumberOfPartitionsIsNotSpecified(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNumberOfPartitionsIsNotSpecified(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String repartitionName = "new-topic";
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
|
@ -583,7 +603,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.toStream()
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
validateReceivedMessages(
|
||||
new IntegerDeserializer(),
|
||||
|
@ -601,8 +621,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupByKey(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupByKey(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String repartitionName = "new-partitions";
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
|
@ -629,7 +649,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.toStream()
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
final String topology = builder.build().describe().toString();
|
||||
|
||||
|
@ -647,8 +667,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldGenerateRepartitionTopicWhenNameIsNotSpecified(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldGenerateRepartitionTopicWhenNameIsNotSpecified(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
sendEvents(
|
||||
|
@ -666,7 +686,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.repartition(Repartitioned.with(Serdes.String(), Serdes.String()))
|
||||
.to(outputTopic);
|
||||
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization));
|
||||
startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol));
|
||||
|
||||
validateReceivedMessages(
|
||||
new StringDeserializer(),
|
||||
|
@ -683,8 +703,8 @@ public class KStreamRepartitionIntegrationTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION})
|
||||
public void shouldGoThroughRebalancingCorrectly(final String topologyOptimization) throws Exception {
|
||||
@MethodSource("protocolAndOptimizationParameters")
|
||||
public void shouldGoThroughRebalancingCorrectly(final String topologyOptimization, final boolean useNewProtocol) throws Exception {
|
||||
final String repartitionName = "rebalancing-test";
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
|
@ -711,7 +731,7 @@ public class KStreamRepartitionIntegrationTest {
|
|||
.toStream()
|
||||
.to(outputTopic);
|
||||
|
||||
final Properties streamsConfiguration = createStreamsConfig(topologyOptimization);
|
||||
final Properties streamsConfiguration = createStreamsConfig(topologyOptimization, useNewProtocol);
|
||||
startStreams(builder, streamsConfiguration);
|
||||
final Properties streamsToCloseConfigs = new Properties();
|
||||
streamsToCloseConfigs.putAll(streamsConfiguration);
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse;
|
|||
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Timer;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.streams.GroupProtocol;
|
||||
import org.apache.kafka.streams.KafkaClientSupplier;
|
||||
|
@ -56,6 +57,7 @@ import org.apache.kafka.streams.errors.MissingSourceTopicException;
|
|||
import org.apache.kafka.streams.errors.StreamsException;
|
||||
import org.apache.kafka.streams.errors.TaskCorruptedException;
|
||||
import org.apache.kafka.streams.errors.TaskMigratedException;
|
||||
import org.apache.kafka.streams.errors.TopologyException;
|
||||
import org.apache.kafka.streams.internals.ConsumerWrapper;
|
||||
import org.apache.kafka.streams.internals.metrics.ClientMetrics;
|
||||
import org.apache.kafka.streams.internals.metrics.StreamsThreadMetricsDelegatingReporter;
|
||||
|
@ -372,6 +374,8 @@ public class StreamThread extends Thread implements ProcessingThread {
|
|||
private volatile KafkaFutureImpl<Uuid> restoreConsumerInstanceIdFuture = new KafkaFutureImpl<>();
|
||||
private volatile KafkaFutureImpl<Uuid> producerInstanceIdFuture = new KafkaFutureImpl<>();
|
||||
|
||||
private Timer topicsReadyTimer;
|
||||
|
||||
public static StreamThread create(final TopologyMetadata topologyMetadata,
|
||||
final StreamsConfig config,
|
||||
final KafkaClientSupplier clientSupplier,
|
||||
|
@ -1536,15 +1540,28 @@ public class StreamThread extends Thread implements ProcessingThread {
|
|||
|
||||
public void handleStreamsRebalanceData() {
|
||||
if (streamsRebalanceData.isPresent()) {
|
||||
boolean hasMissingSourceTopics = false;
|
||||
String missingTopicsDetail = null;
|
||||
|
||||
for (final StreamsGroupHeartbeatResponseData.Status status : streamsRebalanceData.get().statuses()) {
|
||||
if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.SHUTDOWN_APPLICATION.code()) {
|
||||
shutdownErrorHook.run();
|
||||
} else if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code()) {
|
||||
final String errorMsg = String.format("Missing source topics: %s", status.statusDetail());
|
||||
hasMissingSourceTopics = true;
|
||||
missingTopicsDetail = status.statusDetail();
|
||||
} else if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.INCORRECTLY_PARTITIONED_TOPICS.code()) {
|
||||
final String errorMsg = status.statusDetail();
|
||||
log.error(errorMsg);
|
||||
throw new MissingSourceTopicException(errorMsg);
|
||||
throw new TopologyException(errorMsg);
|
||||
}
|
||||
}
|
||||
|
||||
if (hasMissingSourceTopics) {
|
||||
handleMissingSourceTopicsWithTimeout(missingTopicsDetail);
|
||||
} else {
|
||||
// Reset timeout tracking when no missing source topics are reported
|
||||
topicsReadyTimer = null;
|
||||
}
|
||||
|
||||
final Map<StreamsRebalanceData.HostInfo, StreamsRebalanceData.EndpointPartitions> partitionsByEndpoint =
|
||||
streamsRebalanceData.get().partitionsByHost();
|
||||
|
@ -1563,6 +1580,30 @@ public class StreamThread extends Thread implements ProcessingThread {
|
|||
}
|
||||
}
|
||||
|
||||
private void handleMissingSourceTopicsWithTimeout(final String missingTopicsDetail) {
|
||||
// Start timeout tracking on first encounter with missing topics
|
||||
if (topicsReadyTimer == null) {
|
||||
topicsReadyTimer = time.timer(maxPollTimeMs);
|
||||
log.info("Missing source topics detected: {}. Will wait up to {}ms before failing.",
|
||||
missingTopicsDetail, maxPollTimeMs);
|
||||
} else {
|
||||
topicsReadyTimer.update();
|
||||
}
|
||||
|
||||
if (topicsReadyTimer.isExpired()) {
|
||||
final long elapsedTime = topicsReadyTimer.elapsedMs();
|
||||
final String errorMsg = String.format("Missing source topics: %s. Timeout exceeded after %dms.",
|
||||
missingTopicsDetail, elapsedTime);
|
||||
log.error(errorMsg);
|
||||
|
||||
throw new MissingSourceTopicException(errorMsg);
|
||||
} else {
|
||||
log.debug("Missing source topics: {}. Elapsed time: {}ms, timeout in: {}ms",
|
||||
missingTopicsDetail, topicsReadyTimer.elapsedMs(), topicsReadyTimer.remainingMs());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static Map<TopicPartition, PartitionInfo> getTopicPartitionInfo(final Map<HostInfo, Set<TopicPartition>> partitionsByHost) {
|
||||
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo = new HashMap<>();
|
||||
for (final Set<TopicPartition> value : partitionsByHost.values()) {
|
||||
|
|
|
@ -70,6 +70,7 @@ import org.apache.kafka.streams.errors.MissingSourceTopicException;
|
|||
import org.apache.kafka.streams.errors.StreamsException;
|
||||
import org.apache.kafka.streams.errors.TaskCorruptedException;
|
||||
import org.apache.kafka.streams.errors.TaskMigratedException;
|
||||
import org.apache.kafka.streams.errors.TopologyException;
|
||||
import org.apache.kafka.streams.kstream.Consumed;
|
||||
import org.apache.kafka.streams.kstream.Materialized;
|
||||
import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
|
||||
|
@ -162,6 +163,7 @@ import static org.hamcrest.Matchers.empty;
|
|||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.isA;
|
||||
import static org.hamcrest.core.IsInstanceOf.instanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
|
@ -3896,8 +3898,9 @@ public class StreamThreadTest {
|
|||
new LogContext(String.format("stream-client [%s] ", CLIENT_ID))
|
||||
);
|
||||
final StreamsConfig config = new StreamsConfig(props);
|
||||
final MockTime mockTime = new MockTime(1);
|
||||
thread = new StreamThread(
|
||||
new MockTime(1),
|
||||
mockTime,
|
||||
config,
|
||||
null,
|
||||
mainConsumer,
|
||||
|
@ -3929,8 +3932,77 @@ public class StreamThreadTest {
|
|||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
||||
.setStatusDetail("Missing source topics")
|
||||
));
|
||||
|
||||
// First call should not throw exception (within timeout)
|
||||
thread.runOnceWithoutProcessingThreads();
|
||||
|
||||
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
|
||||
mockTime.sleep(300001);
|
||||
|
||||
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithoutProcessingThreads());
|
||||
assertTrue(exception.getMessage().startsWith("Missing source topics"));
|
||||
assertTrue(exception.getMessage().contains("Missing source topics"));
|
||||
assertTrue(exception.getMessage().contains("Timeout exceeded"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStreamsProtocolIncorrectlyPartitionedTopics() {
|
||||
final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class);
|
||||
when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty());
|
||||
when(mainConsumer.poll(Mockito.any(Duration.class))).thenReturn(new ConsumerRecords<>(Map.of(), Map.of()));
|
||||
when(mainConsumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
||||
final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(
|
||||
UUID.randomUUID(),
|
||||
Optional.empty(),
|
||||
Map.of(),
|
||||
Map.of()
|
||||
);
|
||||
final Runnable shutdownErrorHook = mock(Runnable.class);
|
||||
|
||||
final Properties props = configProps(false, false, false);
|
||||
final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(
|
||||
new TopologyMetadata(internalTopologyBuilder, new StreamsConfig(props)),
|
||||
StreamsMetadataState.UNKNOWN_HOST,
|
||||
new LogContext(String.format("stream-client [%s] ", CLIENT_ID))
|
||||
);
|
||||
final StreamsConfig config = new StreamsConfig(props);
|
||||
final MockTime mockTime = new MockTime(1);
|
||||
thread = new StreamThread(
|
||||
mockTime,
|
||||
config,
|
||||
null,
|
||||
mainConsumer,
|
||||
consumer,
|
||||
changelogReader,
|
||||
null,
|
||||
mock(TaskManager.class),
|
||||
null,
|
||||
new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime),
|
||||
new TopologyMetadata(internalTopologyBuilder, config),
|
||||
PROCESS_ID,
|
||||
CLIENT_ID,
|
||||
new LogContext(""),
|
||||
null,
|
||||
new AtomicLong(Long.MAX_VALUE),
|
||||
new LinkedList<>(),
|
||||
shutdownErrorHook,
|
||||
HANDLER,
|
||||
null,
|
||||
Optional.of(streamsRebalanceData),
|
||||
streamsMetadataState
|
||||
).updateThreadMetadata(adminClientId(CLIENT_ID));
|
||||
|
||||
thread.setState(State.STARTING);
|
||||
|
||||
streamsRebalanceData.setStatuses(List.of(
|
||||
new StreamsGroupHeartbeatResponseData.Status()
|
||||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.INCORRECTLY_PARTITIONED_TOPICS.code())
|
||||
.setStatusDetail("Topics are incorrectly partitioned")
|
||||
));
|
||||
|
||||
// Should immediately throw TopologyException (no timeout like MISSING_SOURCE_TOPICS)
|
||||
final TopologyException exception = assertThrows(TopologyException.class,
|
||||
() -> thread.runOnceWithoutProcessingThreads());
|
||||
assertTrue(exception.getMessage().contains("Topics are incorrectly partitioned"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -4013,8 +4085,9 @@ public class StreamThreadTest {
|
|||
StreamsMetadataState.UNKNOWN_HOST,
|
||||
new LogContext(String.format("stream-client [%s] ", CLIENT_ID))
|
||||
);
|
||||
final MockTime mockTime = new MockTime(1);
|
||||
thread = new StreamThread(
|
||||
new MockTime(1),
|
||||
mockTime,
|
||||
config,
|
||||
null,
|
||||
mainConsumer,
|
||||
|
@ -4046,8 +4119,105 @@ public class StreamThreadTest {
|
|||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
||||
.setStatusDetail("Missing source topics")
|
||||
));
|
||||
|
||||
// First call should not throw exception (within timeout)
|
||||
thread.runOnceWithProcessingThreads();
|
||||
|
||||
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
|
||||
mockTime.sleep(300001);
|
||||
|
||||
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithProcessingThreads());
|
||||
assertTrue(exception.getMessage().startsWith("Missing source topics"));
|
||||
assertTrue(exception.getMessage().contains("Missing source topics"));
|
||||
assertTrue(exception.getMessage().contains("Timeout exceeded"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStreamsProtocolMissingSourceTopicRecovery() {
|
||||
final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class);
|
||||
when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty());
|
||||
when(mainConsumer.poll(Mockito.any(Duration.class))).thenReturn(new ConsumerRecords<>(Map.of(), Map.of()));
|
||||
when(mainConsumer.groupMetadata()).thenReturn(consumerGroupMetadata);
|
||||
final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(
|
||||
UUID.randomUUID(),
|
||||
Optional.empty(),
|
||||
Map.of(),
|
||||
Map.of()
|
||||
);
|
||||
|
||||
final Properties props = configProps(false, false, false);
|
||||
final Runnable shutdownErrorHook = mock(Runnable.class);
|
||||
final StreamsConfig config = new StreamsConfig(props);
|
||||
final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(
|
||||
new TopologyMetadata(internalTopologyBuilder, config),
|
||||
StreamsMetadataState.UNKNOWN_HOST,
|
||||
new LogContext(String.format("stream-client [%s] ", CLIENT_ID))
|
||||
);
|
||||
final MockTime mockTime = new MockTime(1);
|
||||
thread = new StreamThread(
|
||||
mockTime,
|
||||
config,
|
||||
null,
|
||||
mainConsumer,
|
||||
consumer,
|
||||
changelogReader,
|
||||
null,
|
||||
mock(TaskManager.class),
|
||||
null,
|
||||
new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime),
|
||||
new TopologyMetadata(internalTopologyBuilder, config),
|
||||
PROCESS_ID,
|
||||
CLIENT_ID,
|
||||
new LogContext(""),
|
||||
null,
|
||||
new AtomicLong(Long.MAX_VALUE),
|
||||
new LinkedList<>(),
|
||||
shutdownErrorHook,
|
||||
HANDLER,
|
||||
null,
|
||||
Optional.of(streamsRebalanceData),
|
||||
streamsMetadataState
|
||||
).updateThreadMetadata(adminClientId(CLIENT_ID));
|
||||
|
||||
thread.setState(State.STARTING);
|
||||
thread.runOnceWithoutProcessingThreads();
|
||||
|
||||
// Set missing source topics status
|
||||
streamsRebalanceData.setStatuses(List.of(
|
||||
new StreamsGroupHeartbeatResponseData.Status()
|
||||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
||||
.setStatusDetail("Missing source topics")
|
||||
));
|
||||
|
||||
// First call should not throw exception (within timeout)
|
||||
thread.runOnceWithoutProcessingThreads();
|
||||
|
||||
// Advance time but not beyond timeout
|
||||
mockTime.sleep(150000); // Half of max.poll.interval.ms
|
||||
|
||||
// Should still not throw exception
|
||||
thread.runOnceWithoutProcessingThreads();
|
||||
|
||||
// Clear the missing source topics (simulate recovery)
|
||||
streamsRebalanceData.setStatuses(List.of());
|
||||
|
||||
// Should complete without exception (recovery successful)
|
||||
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
|
||||
|
||||
// Set missing topics again - should reset the timeout
|
||||
streamsRebalanceData.setStatuses(List.of(
|
||||
new StreamsGroupHeartbeatResponseData.Status()
|
||||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
|
||||
.setStatusDetail("Different missing topics")
|
||||
));
|
||||
|
||||
// Advance time by 250 seconds to test if timer was reset
|
||||
// Total time from beginning: 150000 + 250000 = 400000ms (400s)
|
||||
// If timer was NOT reset: elapsed time = 400s > 300s → should throw
|
||||
// If timer WAS reset: elapsed time = 250s < 300s → should NOT throw
|
||||
mockTime.sleep(250000); // Advance by 250 seconds
|
||||
|
||||
// Should not throw because timer was reset - only 250s elapsed from reset point
|
||||
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -27,10 +27,12 @@ import kafka.server.SharedServer;
|
|||
|
||||
import org.apache.kafka.clients.CommonClientConfigs;
|
||||
import org.apache.kafka.clients.admin.AdminClientConfig;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol;
|
||||
import org.apache.kafka.common.test.api.TestKitDefaults;
|
||||
import org.apache.kafka.common.utils.ThreadUtils;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
@ -114,6 +116,8 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
private final String controllerListenerName;
|
||||
private final String brokerSecurityProtocol;
|
||||
private final String controllerSecurityProtocol;
|
||||
private boolean standalone;
|
||||
private Optional<Map<Integer, Uuid>> initialVoterSet = Optional.empty();
|
||||
private boolean deleteOnClose;
|
||||
|
||||
public Builder(TestKitNodes nodes) {
|
||||
|
@ -130,6 +134,16 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder setStandalone(boolean standalone) {
|
||||
this.standalone = standalone;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setInitialVoterSet(Map<Integer, Uuid> initialVoterSet) {
|
||||
this.initialVoterSet = Optional.of(initialVoterSet);
|
||||
return this;
|
||||
}
|
||||
|
||||
private KafkaConfig createNodeConfig(TestKitNode node) throws IOException {
|
||||
TestKitNode brokerNode = nodes.brokerNodes().get(node.id());
|
||||
TestKitNode controllerNode = nodes.controllerNodes().get(node.id());
|
||||
|
@ -184,6 +198,11 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
// reduce log cleaner offset map memory usage
|
||||
props.putIfAbsent(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152");
|
||||
|
||||
// do not include auto join config in broker nodes
|
||||
if (brokerNode != null) {
|
||||
props.remove(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG);
|
||||
}
|
||||
|
||||
// Add associated broker node property overrides
|
||||
if (brokerNode != null) {
|
||||
props.putAll(brokerNode.propertyOverrides());
|
||||
|
@ -323,6 +342,8 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
faultHandlerFactory,
|
||||
socketFactoryManager,
|
||||
jaasFile,
|
||||
standalone,
|
||||
initialVoterSet,
|
||||
deleteOnClose);
|
||||
}
|
||||
|
||||
|
@ -368,6 +389,8 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
private final PreboundSocketFactoryManager socketFactoryManager;
|
||||
private final String controllerListenerName;
|
||||
private final Optional<File> jaasFile;
|
||||
private final boolean standalone;
|
||||
private final Optional<Map<Integer, Uuid>> initialVoterSet;
|
||||
private final boolean deleteOnClose;
|
||||
|
||||
private KafkaClusterTestKit(
|
||||
|
@ -378,6 +401,8 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
SimpleFaultHandlerFactory faultHandlerFactory,
|
||||
PreboundSocketFactoryManager socketFactoryManager,
|
||||
Optional<File> jaasFile,
|
||||
boolean standalone,
|
||||
Optional<Map<Integer, Uuid>> initialVoterSet,
|
||||
boolean deleteOnClose
|
||||
) {
|
||||
/*
|
||||
|
@ -395,6 +420,8 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
this.socketFactoryManager = socketFactoryManager;
|
||||
this.controllerListenerName = nodes.controllerListenerName().value();
|
||||
this.jaasFile = jaasFile;
|
||||
this.standalone = standalone;
|
||||
this.initialVoterSet = initialVoterSet;
|
||||
this.deleteOnClose = deleteOnClose;
|
||||
}
|
||||
|
||||
|
@ -425,8 +452,9 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
boolean writeMetadataDirectory
|
||||
) {
|
||||
try {
|
||||
final var nodeId = ensemble.nodeId().getAsInt();
|
||||
Formatter formatter = new Formatter();
|
||||
formatter.setNodeId(ensemble.nodeId().getAsInt());
|
||||
formatter.setNodeId(nodeId);
|
||||
formatter.setClusterId(ensemble.clusterId().get());
|
||||
if (writeMetadataDirectory) {
|
||||
formatter.setDirectories(ensemble.logDirProps().keySet());
|
||||
|
@ -452,15 +480,50 @@ public class KafkaClusterTestKit implements AutoCloseable {
|
|||
if (nodes.bootstrapMetadata().featureLevel(KRaftVersion.FEATURE_NAME) > 0) {
|
||||
StringBuilder dynamicVotersBuilder = new StringBuilder();
|
||||
String prefix = "";
|
||||
for (TestKitNode controllerNode : nodes.controllerNodes().values()) {
|
||||
int port = socketFactoryManager.
|
||||
getOrCreatePortForListener(controllerNode.id(), controllerListenerName);
|
||||
dynamicVotersBuilder.append(prefix);
|
||||
prefix = ",";
|
||||
dynamicVotersBuilder.append(String.format("%d@localhost:%d:%s",
|
||||
controllerNode.id(), port, controllerNode.metadataDirectoryId()));
|
||||
if (standalone) {
|
||||
if (nodeId == TestKitDefaults.CONTROLLER_ID_OFFSET) {
|
||||
final var controllerNode = nodes.controllerNodes().get(nodeId);
|
||||
dynamicVotersBuilder.append(
|
||||
String.format(
|
||||
"%d@localhost:%d:%s",
|
||||
controllerNode.id(),
|
||||
socketFactoryManager.
|
||||
getOrCreatePortForListener(controllerNode.id(), controllerListenerName),
|
||||
controllerNode.metadataDirectoryId()
|
||||
)
|
||||
);
|
||||
formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString()));
|
||||
} else {
|
||||
formatter.setNoInitialControllersFlag(true);
|
||||
}
|
||||
} else if (initialVoterSet.isPresent()) {
|
||||
for (final var controllerNode : initialVoterSet.get().entrySet()) {
|
||||
final var voterId = controllerNode.getKey();
|
||||
final var voterDirectoryId = controllerNode.getValue();
|
||||
dynamicVotersBuilder.append(prefix);
|
||||
prefix = ",";
|
||||
dynamicVotersBuilder.append(
|
||||
String.format(
|
||||
"%d@localhost:%d:%s",
|
||||
voterId,
|
||||
socketFactoryManager.
|
||||
getOrCreatePortForListener(voterId, controllerListenerName),
|
||||
voterDirectoryId
|
||||
)
|
||||
);
|
||||
}
|
||||
formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString()));
|
||||
} else {
|
||||
for (TestKitNode controllerNode : nodes.controllerNodes().values()) {
|
||||
int port = socketFactoryManager.
|
||||
getOrCreatePortForListener(controllerNode.id(), controllerListenerName);
|
||||
dynamicVotersBuilder.append(prefix);
|
||||
prefix = ",";
|
||||
dynamicVotersBuilder.append(String.format("%d@localhost:%d:%s",
|
||||
controllerNode.id(), port, controllerNode.metadataDirectoryId()));
|
||||
}
|
||||
formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString()));
|
||||
}
|
||||
formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString()));
|
||||
}
|
||||
formatter.run();
|
||||
} catch (Exception e) {
|
||||
|
|
|
@ -18,40 +18,17 @@
|
|||
package org.apache.kafka.tools.reassign;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* The state of a partition reassignment. The current replicas and target replicas
|
||||
* may overlap.
|
||||
*
|
||||
* @param currentReplicas The current replicas.
|
||||
* @param targetReplicas The target replicas.
|
||||
* @param done True if the reassignment is done.
|
||||
*/
|
||||
final class PartitionReassignmentState {
|
||||
public final List<Integer> currentReplicas;
|
||||
|
||||
public final List<Integer> targetReplicas;
|
||||
|
||||
public final boolean done;
|
||||
|
||||
/**
|
||||
* @param currentReplicas The current replicas.
|
||||
* @param targetReplicas The target replicas.
|
||||
* @param done True if the reassignment is done.
|
||||
*/
|
||||
public PartitionReassignmentState(List<Integer> currentReplicas, List<Integer> targetReplicas, boolean done) {
|
||||
this.currentReplicas = currentReplicas;
|
||||
this.targetReplicas = targetReplicas;
|
||||
this.done = done;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
PartitionReassignmentState state = (PartitionReassignmentState) o;
|
||||
return done == state.done && Objects.equals(currentReplicas, state.currentReplicas) && Objects.equals(targetReplicas, state.targetReplicas);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(currentReplicas, targetReplicas, done);
|
||||
}
|
||||
}
|
||||
record PartitionReassignmentState(
|
||||
List<Integer> currentReplicas,
|
||||
List<Integer> targetReplicas,
|
||||
boolean done
|
||||
) { }
|
||||
|
|
|
@ -279,12 +279,12 @@ public class ReassignPartitionsCommand {
|
|||
bld.add("Status of partition reassignment:");
|
||||
states.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).forEach(topicPartition -> {
|
||||
PartitionReassignmentState state = states.get(topicPartition);
|
||||
if (state.done) {
|
||||
if (state.currentReplicas.equals(state.targetReplicas)) {
|
||||
if (state.done()) {
|
||||
if (state.currentReplicas().equals(state.targetReplicas())) {
|
||||
bld.add(String.format("Reassignment of partition %s is completed.", topicPartition));
|
||||
} else {
|
||||
String currentReplicaStr = state.currentReplicas.stream().map(String::valueOf).collect(Collectors.joining(","));
|
||||
String targetReplicaStr = state.targetReplicas.stream().map(String::valueOf).collect(Collectors.joining(","));
|
||||
String currentReplicaStr = state.currentReplicas().stream().map(String::valueOf).collect(Collectors.joining(","));
|
||||
String targetReplicaStr = state.targetReplicas().stream().map(String::valueOf).collect(Collectors.joining(","));
|
||||
|
||||
bld.add("There is no active reassignment of partition " + topicPartition + ", " +
|
||||
"but replica set is " + currentReplicaStr + " rather than " +
|
||||
|
|
|
@ -21,49 +21,21 @@ import org.apache.kafka.common.TopicPartition;
|
|||
import org.apache.kafka.common.TopicPartitionReplica;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A result returned from verifyAssignment.
|
||||
* @param partStates A map from partitions to reassignment states.
|
||||
* @param partsOngoing True if there are any ongoing partition reassignments.
|
||||
* @param moveStates A map from log directories to movement states.
|
||||
* @param movesOngoing True if there are any ongoing moves that we know about.
|
||||
*/
|
||||
public final class VerifyAssignmentResult {
|
||||
public final Map<TopicPartition, PartitionReassignmentState> partStates;
|
||||
public final boolean partsOngoing;
|
||||
public final Map<TopicPartitionReplica, LogDirMoveState> moveStates;
|
||||
public final boolean movesOngoing;
|
||||
|
||||
public record VerifyAssignmentResult(
|
||||
Map<TopicPartition, PartitionReassignmentState> partStates,
|
||||
boolean partsOngoing,
|
||||
Map<TopicPartitionReplica, LogDirMoveState> moveStates,
|
||||
boolean movesOngoing
|
||||
) {
|
||||
public VerifyAssignmentResult(Map<TopicPartition, PartitionReassignmentState> partStates) {
|
||||
this(partStates, false, Map.of(), false);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param partStates A map from partitions to reassignment states.
|
||||
* @param partsOngoing True if there are any ongoing partition reassignments.
|
||||
* @param moveStates A map from log directories to movement states.
|
||||
* @param movesOngoing True if there are any ongoing moves that we know about.
|
||||
*/
|
||||
public VerifyAssignmentResult(
|
||||
Map<TopicPartition, PartitionReassignmentState> partStates,
|
||||
boolean partsOngoing,
|
||||
Map<TopicPartitionReplica, LogDirMoveState> moveStates,
|
||||
boolean movesOngoing
|
||||
) {
|
||||
this.partStates = partStates;
|
||||
this.partsOngoing = partsOngoing;
|
||||
this.moveStates = moveStates;
|
||||
this.movesOngoing = movesOngoing;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
VerifyAssignmentResult that = (VerifyAssignmentResult) o;
|
||||
return partsOngoing == that.partsOngoing && movesOngoing == that.movesOngoing && Objects.equals(partStates, that.partStates) && Objects.equals(moveStates, that.moveStates);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(partStates, partsOngoing, moveStates, movesOngoing);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,12 +51,10 @@ import java.io.BufferedWriter;
|
|||
import java.io.File;
|
||||
import java.io.FileWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.Set;
|
||||
|
||||
import static java.time.Duration.ofMillis;
|
||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForEmptyConsumerGroup;
|
||||
|
@ -176,7 +174,7 @@ public abstract class AbstractResetIntegrationTest {
|
|||
}
|
||||
|
||||
private void add10InputElements() {
|
||||
final List<KeyValue<Long, String>> records = Arrays.asList(KeyValue.pair(0L, "aaa"),
|
||||
final List<KeyValue<Long, String>> records = List.of(KeyValue.pair(0L, "aaa"),
|
||||
KeyValue.pair(1L, "bbb"),
|
||||
KeyValue.pair(0L, "ccc"),
|
||||
KeyValue.pair(1L, "ddd"),
|
||||
|
@ -189,7 +187,7 @@ public abstract class AbstractResetIntegrationTest {
|
|||
|
||||
for (final KeyValue<Long, String> record : records) {
|
||||
mockTime.sleep(10);
|
||||
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(INPUT_TOPIC, Collections.singleton(record), producerConfig, mockTime.milliseconds());
|
||||
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(INPUT_TOPIC, Set.of(record), producerConfig, mockTime.milliseconds());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -211,7 +209,7 @@ public abstract class AbstractResetIntegrationTest {
|
|||
|
||||
final List<String> internalTopics = cluster.getAllTopicsInCluster().stream()
|
||||
.filter(StreamsResetter::matchesInternalTopicFormat)
|
||||
.collect(Collectors.toList());
|
||||
.toList();
|
||||
cleanGlobal(false,
|
||||
"--internal-topics",
|
||||
String.join(",", internalTopics.subList(1, internalTopics.size())),
|
||||
|
@ -288,7 +286,7 @@ public abstract class AbstractResetIntegrationTest {
|
|||
if (!useRepartitioned) {
|
||||
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
|
||||
INTERMEDIATE_USER_TOPIC,
|
||||
Collections.singleton(badMessage),
|
||||
Set.of(badMessage),
|
||||
producerConfig,
|
||||
mockTime.milliseconds());
|
||||
}
|
||||
|
@ -375,7 +373,7 @@ public abstract class AbstractResetIntegrationTest {
|
|||
final String resetScenarioArg,
|
||||
final String appID) throws Exception {
|
||||
final List<String> parameterList = new ArrayList<>(
|
||||
Arrays.asList("--application-id", appID,
|
||||
List.of("--application-id", appID,
|
||||
"--bootstrap-server", cluster.bootstrapServers(),
|
||||
"--input-topics", INPUT_TOPIC
|
||||
));
|
||||
|
|
|
@ -31,8 +31,8 @@ import org.apache.kafka.common.test.api.ClusterTestDefaults;
|
|||
import org.apache.kafka.server.config.ServerConfigs;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -56,7 +56,7 @@ public class BrokerApiVersionsCommandTest {
|
|||
|
||||
NodeApiVersions nodeApiVersions = new NodeApiVersions(
|
||||
ApiVersionsResponse.filterApis(listenerType, true, true),
|
||||
Collections.emptyList());
|
||||
List.of());
|
||||
Iterator<ApiKeys> apiKeysIter = ApiKeys.clientApis().iterator();
|
||||
while (apiKeysIter.hasNext()) {
|
||||
ApiKeys apiKey = apiKeysIter.next();
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.junit.jupiter.api.Test;
|
|||
import org.mockito.ArgumentCaptor;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -209,7 +208,7 @@ public class ClientMetricsCommandTest {
|
|||
ClientMetricsCommand.ClientMetricsService service = new ClientMetricsCommand.ClientMetricsService(adminClient);
|
||||
|
||||
ConfigResource cr = new ConfigResource(ConfigResource.Type.CLIENT_METRICS, clientMetricsName);
|
||||
Config cfg = new Config(Collections.singleton(new ConfigEntry("metrics", "org.apache.kafka.producer.")));
|
||||
Config cfg = new Config(Set.of(new ConfigEntry("metrics", "org.apache.kafka.producer.")));
|
||||
DescribeConfigsResult describeResult = AdminClientTestUtils.describeConfigsResult(cr, cfg);
|
||||
when(adminClient.describeConfigs(any())).thenReturn(describeResult);
|
||||
AlterConfigsResult alterResult = AdminClientTestUtils.alterConfigsResult(cr);
|
||||
|
@ -237,7 +236,7 @@ public class ClientMetricsCommandTest {
|
|||
ConfigResource.Type.CLIENT_METRICS, Set.of(clientMetricsName)
|
||||
));
|
||||
when(adminClient.listConfigResources(any(), any())).thenReturn(listConfigResourcesResult);
|
||||
Config cfg = new Config(Collections.singleton(new ConfigEntry("metrics", "org.apache.kafka.producer.")));
|
||||
Config cfg = new Config(Set.of(new ConfigEntry("metrics", "org.apache.kafka.producer.")));
|
||||
DescribeConfigsResult describeResult = AdminClientTestUtils.describeConfigsResult(cr, cfg);
|
||||
when(adminClient.describeConfigs(any())).thenReturn(describeResult);
|
||||
|
||||
|
@ -284,7 +283,7 @@ public class ClientMetricsCommandTest {
|
|||
ListConfigResourcesResult result = AdminClientTestUtils.listConfigResourcesResult(clientMetricsName);
|
||||
when(adminClient.listConfigResources(any(), any())).thenReturn(result);
|
||||
ConfigResource cr = new ConfigResource(ConfigResource.Type.CLIENT_METRICS, clientMetricsName);
|
||||
Config cfg = new Config(Collections.singleton(new ConfigEntry("metrics", "org.apache.kafka.producer.")));
|
||||
Config cfg = new Config(Set.of(new ConfigEntry("metrics", "org.apache.kafka.producer.")));
|
||||
DescribeConfigsResult describeResult = AdminClientTestUtils.describeConfigsResult(cr, cfg);
|
||||
when(adminClient.describeConfigs(any())).thenReturn(describeResult);
|
||||
|
||||
|
@ -326,7 +325,7 @@ public class ClientMetricsCommandTest {
|
|||
ListConfigResourcesResult result = AdminClientTestUtils.listConfigResourcesResult(Errors.UNSUPPORTED_VERSION.exception());
|
||||
when(adminClient.listConfigResources(any(), any())).thenReturn(result);
|
||||
|
||||
assertThrows(ExecutionException.class, () -> service.listClientMetrics());
|
||||
assertThrows(ExecutionException.class, service::listClientMetrics);
|
||||
}
|
||||
|
||||
private void assertInitializeInvalidOptionsExitCode(int expected, String[] options) {
|
||||
|
|
|
@ -29,7 +29,6 @@ import java.io.PrintStream;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -71,10 +70,10 @@ public class ClusterToolTest {
|
|||
|
||||
@ClusterTest(brokers = 2, types = {Type.KRAFT, Type.CO_KRAFT})
|
||||
public void testListEndpointsArgumentWithBootstrapServer(ClusterInstance clusterInstance) {
|
||||
List<Integer> brokerIds = clusterInstance.brokerIds().stream().collect(Collectors.toList());
|
||||
List<Integer> brokerIds = clusterInstance.brokerIds().stream().toList();
|
||||
clusterInstance.shutdownBroker(brokerIds.get(0));
|
||||
|
||||
List<String> ports = Arrays.stream(clusterInstance.bootstrapServers().split(",")).map(b -> b.split(":")[1]).collect(Collectors.toList());
|
||||
List<String> ports = Arrays.stream(clusterInstance.bootstrapServers().split(",")).map(b -> b.split(":")[1]).toList();
|
||||
String format = "%-10s %-9s %-10s %-10s %-10s %-15s%n%-10s %-9s %-10s %-10s %-10s %-15s%n%-10s %-9s %-10s %-10s %-10s %-6s";
|
||||
String expected = String.format(format,
|
||||
"ID", "HOST", "PORT", "RACK", "STATE", "ENDPOINT_TYPE",
|
||||
|
@ -110,7 +109,7 @@ public class ClusterToolTest {
|
|||
int id = clusterInstance.controllerIds().iterator().next();
|
||||
String format = "%-10s %-9s %-10s %-10s %-15s%n%-10s %-9s %-10s %-10s %-10s";
|
||||
String expected = String.format(format, "ID", "HOST", "PORT", "RACK", "ENDPOINT_TYPE", id, "localhost", port, "null", "controller");
|
||||
assertTrue(output.equals(expected));
|
||||
assertEquals(expected, output);
|
||||
}
|
||||
|
||||
@ClusterTest(brokers = 3, types = {Type.KRAFT, Type.CO_KRAFT})
|
||||
|
|
|
@ -52,10 +52,6 @@ import java.util.function.Consumer;
|
|||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.singleton;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG;
|
||||
import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG;
|
||||
import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG;
|
||||
|
@ -246,27 +242,27 @@ public class ConfigCommandIntegrationTest {
|
|||
|
||||
try (Admin client = cluster.admin()) {
|
||||
// Add config
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "110000"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.empty(), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "120000"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId), Map.of(MESSAGE_MAX_BYTES_CONFIG, "110000"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.empty(), Map.of(MESSAGE_MAX_BYTES_CONFIG, "120000"), alterOpts);
|
||||
|
||||
// Change config
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "130000"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.empty(), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "140000"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId), Map.of(MESSAGE_MAX_BYTES_CONFIG, "130000"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.empty(), Map.of(MESSAGE_MAX_BYTES_CONFIG, "140000"), alterOpts);
|
||||
|
||||
// Delete config
|
||||
deleteAndVerifyConfigValue(client, defaultBrokerId, singleton(MESSAGE_MAX_BYTES_CONFIG), true, alterOpts);
|
||||
deleteAndVerifyConfigValue(client, defaultBrokerId, Set.of(MESSAGE_MAX_BYTES_CONFIG), true, alterOpts);
|
||||
|
||||
// Listener configs: should work only with listener name
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
|
||||
singletonMap("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"), alterOpts);
|
||||
Map.of("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"), alterOpts);
|
||||
// Per-broker config configured at default cluster-level should fail
|
||||
assertThrows(ExecutionException.class,
|
||||
() -> alterConfigWithAdmin(client, Optional.empty(),
|
||||
singletonMap("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"), alterOpts));
|
||||
Map.of("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"), alterOpts));
|
||||
deleteAndVerifyConfigValue(client, defaultBrokerId,
|
||||
singleton("listener.name.internal.ssl.keystore.location"), false, alterOpts);
|
||||
Set.of("listener.name.internal.ssl.keystore.location"), false, alterOpts);
|
||||
alterConfigWithAdmin(client, Optional.of(defaultBrokerId),
|
||||
singletonMap("listener.name.external.ssl.keystore.password", "secret"), alterOpts);
|
||||
Map.of("listener.name.external.ssl.keystore.password", "secret"), alterOpts);
|
||||
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("listener.name.external.ssl.keystore.password", "secret");
|
||||
|
@ -281,12 +277,11 @@ public class ConfigCommandIntegrationTest {
|
|||
@ClusterTest
|
||||
public void testGroupConfigUpdateUsingKraft() throws Exception {
|
||||
List<String> alterOpts = Stream.concat(entityOp(Optional.of(defaultGroupName)).stream(),
|
||||
Stream.of("--entity-type", "groups", "--alter"))
|
||||
.collect(Collectors.toList());
|
||||
Stream.of("--entity-type", "groups", "--alter")).toList();
|
||||
verifyGroupConfigUpdate(alterOpts);
|
||||
|
||||
// Test for the --group alias
|
||||
verifyGroupConfigUpdate(asList("--group", defaultGroupName, "--alter"));
|
||||
verifyGroupConfigUpdate(List.of("--group", defaultGroupName, "--alter"));
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
|
@ -343,7 +338,7 @@ public class ConfigCommandIntegrationTest {
|
|||
deleteAndVerifyGroupConfigValue(client, defaultGroupName, configs, alterOpts);
|
||||
|
||||
// Unknown config configured should fail
|
||||
assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, singletonMap("unknown.config", "20000"), alterOpts));
|
||||
assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Map.of("unknown.config", "20000"), alterOpts));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -351,12 +346,11 @@ public class ConfigCommandIntegrationTest {
|
|||
@ClusterTest(types = {Type.KRAFT})
|
||||
public void testClientMetricsConfigUpdate() throws Exception {
|
||||
List<String> alterOpts = Stream.concat(entityOp(Optional.of(defaultClientMetricsName)).stream(),
|
||||
Stream.of("--entity-type", "client-metrics", "--alter"))
|
||||
.collect(Collectors.toList());
|
||||
Stream.of("--entity-type", "client-metrics", "--alter")).toList();
|
||||
verifyClientMetricsConfigUpdate(alterOpts);
|
||||
|
||||
// Test for the --client-metrics alias
|
||||
verifyClientMetricsConfigUpdate(asList("--client-metrics", defaultClientMetricsName, "--alter"));
|
||||
verifyClientMetricsConfigUpdate(List.of("--client-metrics", defaultClientMetricsName, "--alter"));
|
||||
}
|
||||
|
||||
private void verifyClientMetricsConfigUpdate(List<String> alterOpts) throws Exception {
|
||||
|
@ -373,7 +367,7 @@ public class ConfigCommandIntegrationTest {
|
|||
deleteAndVerifyClientMetricsConfigValue(client, defaultClientMetricsName, configs, alterOpts);
|
||||
|
||||
// Unknown config configured should fail
|
||||
assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, singletonMap("unknown.config", "20000"), alterOpts));
|
||||
assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Map.of("unknown.config", "20000"), alterOpts));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -384,13 +378,13 @@ public class ConfigCommandIntegrationTest {
|
|||
try (Admin client = cluster.admin()) {
|
||||
assertThrows(ExecutionException.class,
|
||||
() -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId),
|
||||
singletonMap(AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false"), alterOpts));
|
||||
Map.of(AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false"), alterOpts));
|
||||
assertThrows(ExecutionException.class,
|
||||
() -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId),
|
||||
singletonMap(AUTO_LEADER_REBALANCE_ENABLE_CONFIG, "false"), alterOpts));
|
||||
Map.of(AUTO_LEADER_REBALANCE_ENABLE_CONFIG, "false"), alterOpts));
|
||||
assertThrows(ExecutionException.class,
|
||||
() -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId),
|
||||
singletonMap("broker.id", "1"), alterOpts));
|
||||
Map.of("broker.id", "1"), alterOpts));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -400,11 +394,11 @@ public class ConfigCommandIntegrationTest {
|
|||
|
||||
try (Admin client = cluster.admin()) {
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
|
||||
singletonMap("log.flush.interval.messages", "100"), alterOpts);
|
||||
Map.of("log.flush.interval.messages", "100"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
|
||||
singletonMap("log.retention.bytes", "20"), alterOpts);
|
||||
Map.of("log.retention.bytes", "20"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
|
||||
singletonMap("log.retention.ms", "2"), alterOpts);
|
||||
Map.of("log.retention.ms", "2"), alterOpts);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -415,13 +409,13 @@ public class ConfigCommandIntegrationTest {
|
|||
|
||||
try (Admin client = cluster.admin()) {
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
|
||||
singletonMap(listenerName + "ssl.truststore.type", "PKCS12"), alterOpts);
|
||||
Map.of(listenerName + "ssl.truststore.type", "PKCS12"), alterOpts);
|
||||
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
|
||||
singletonMap(listenerName + "ssl.truststore.location", "/temp/test.jks"), alterOpts);
|
||||
Map.of(listenerName + "ssl.truststore.location", "/temp/test.jks"), alterOpts);
|
||||
alterConfigWithAdmin(client, Optional.of(defaultBrokerId),
|
||||
singletonMap(listenerName + "ssl.truststore.password", "password"), alterOpts);
|
||||
Map.of(listenerName + "ssl.truststore.password", "password"), alterOpts);
|
||||
verifyConfigSecretValue(client, Optional.of(defaultBrokerId),
|
||||
singleton(listenerName + "ssl.truststore.password"));
|
||||
Set.of(listenerName + "ssl.truststore.password"));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -432,13 +426,13 @@ public class ConfigCommandIntegrationTest {
|
|||
try (Admin client = cluster.admin()) {
|
||||
assertThrows(ExecutionException.class,
|
||||
() -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId),
|
||||
singletonMap(SSL_TRUSTSTORE_TYPE_CONFIG, "PKCS12"), alterOpts));
|
||||
Map.of(SSL_TRUSTSTORE_TYPE_CONFIG, "PKCS12"), alterOpts));
|
||||
assertThrows(ExecutionException.class,
|
||||
() -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId),
|
||||
singletonMap(SSL_TRUSTSTORE_LOCATION_CONFIG, "/temp/test.jks"), alterOpts));
|
||||
Map.of(SSL_TRUSTSTORE_LOCATION_CONFIG, "/temp/test.jks"), alterOpts));
|
||||
assertThrows(ExecutionException.class,
|
||||
() -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId),
|
||||
singletonMap(SSL_TRUSTSTORE_PASSWORD_CONFIG, "password"), alterOpts));
|
||||
Map.of(SSL_TRUSTSTORE_PASSWORD_CONFIG, "password"), alterOpts));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -469,7 +463,7 @@ public class ConfigCommandIntegrationTest {
|
|||
|
||||
@ClusterTest
|
||||
public void testUpdateInvalidTopicConfigs() throws ExecutionException, InterruptedException {
|
||||
List<String> alterOpts = asList("--bootstrap-server", cluster.bootstrapServers(), "--entity-type", "topics", "--alter");
|
||||
List<String> alterOpts = List.of("--bootstrap-server", cluster.bootstrapServers(), "--entity-type", "topics", "--alter");
|
||||
try (Admin client = cluster.admin()) {
|
||||
client.createTopics(List.of(new NewTopic("test-config-topic", 1, (short) 1))).all().get();
|
||||
assertInstanceOf(
|
||||
|
@ -480,7 +474,7 @@ public class ConfigCommandIntegrationTest {
|
|||
client,
|
||||
new ConfigCommand.ConfigCommandOptions(
|
||||
toArray(alterOpts,
|
||||
asList("--add-config", "invalid=2", "--entity-type", "topics", "--entity-name", "test-config-topic"))))
|
||||
List.of("--add-config", "invalid=2", "--entity-type", "topics", "--entity-name", "test-config-topic"))))
|
||||
).getCause()
|
||||
);
|
||||
}
|
||||
|
@ -494,14 +488,14 @@ public class ConfigCommandIntegrationTest {
|
|||
public void testUpdateBrokerConfigNotAffectedByInvalidConfig() {
|
||||
try (Admin client = cluster.admin()) {
|
||||
ConfigCommand.alterConfig(client, new ConfigCommand.ConfigCommandOptions(
|
||||
toArray(asList("--bootstrap-server", cluster.bootstrapServers(),
|
||||
toArray(List.of("--bootstrap-server", cluster.bootstrapServers(),
|
||||
"--alter",
|
||||
"--add-config", "log.cleaner.threadzz=2",
|
||||
"--entity-type", "brokers",
|
||||
"--entity-default"))));
|
||||
|
||||
ConfigCommand.alterConfig(client, new ConfigCommand.ConfigCommandOptions(
|
||||
toArray(asList("--bootstrap-server", cluster.bootstrapServers(),
|
||||
toArray(List.of("--bootstrap-server", cluster.bootstrapServers(),
|
||||
"--alter",
|
||||
"--add-config", "log.cleaner.threads=2",
|
||||
"--entity-type", "brokers",
|
||||
|
@ -531,15 +525,15 @@ public class ConfigCommandIntegrationTest {
|
|||
.incrementalAlterConfigs(anyMap(), any(AlterConfigsOptions.class));
|
||||
assertEquals(
|
||||
"The INCREMENTAL_ALTER_CONFIGS API is not supported by the cluster. The API is supported starting from version 2.3.0. You may want to use an older version of this tool to interact with your cluster, or upgrade your brokers to version 2.3.0 or newer to avoid this error.",
|
||||
assertThrows(UnsupportedVersionException.class, () -> {
|
||||
assertThrows(UnsupportedVersionException.class, () ->
|
||||
ConfigCommand.alterConfig(spyAdmin, new ConfigCommand.ConfigCommandOptions(
|
||||
toArray(asList(
|
||||
toArray(List.of(
|
||||
"--bootstrap-server", cluster.bootstrapServers(),
|
||||
"--alter",
|
||||
"--add-config", "log.cleaner.threads=2",
|
||||
"--entity-type", "brokers",
|
||||
"--entity-default"))));
|
||||
}).getMessage()
|
||||
"--entity-default"))))
|
||||
).getMessage()
|
||||
);
|
||||
Mockito.verify(spyAdmin).incrementalAlterConfigs(anyMap(), any(AlterConfigsOptions.class));
|
||||
}
|
||||
|
@ -585,12 +579,12 @@ public class ConfigCommandIntegrationTest {
|
|||
}
|
||||
|
||||
private List<String> entityOp(Optional<String> entityId) {
|
||||
return entityId.map(id -> asList("--entity-name", id))
|
||||
.orElse(singletonList("--entity-default"));
|
||||
return entityId.map(id -> List.of("--entity-name", id))
|
||||
.orElse(List.of("--entity-default"));
|
||||
}
|
||||
|
||||
private List<String> generateDefaultAlterOpts(String bootstrapServers) {
|
||||
return asList("--bootstrap-server", bootstrapServers,
|
||||
return List.of("--bootstrap-server", bootstrapServers,
|
||||
"--entity-type", "brokers", "--alter");
|
||||
}
|
||||
|
||||
|
@ -620,23 +614,23 @@ public class ConfigCommandIntegrationTest {
|
|||
|
||||
private void alterConfigWithAdmin(Admin client, Optional<String> resourceName, Map<String, String> config, List<String> alterOpts) {
|
||||
String configStr = transferConfigMapToString(config);
|
||||
List<String> bootstrapOpts = quorumArgs().collect(Collectors.toList());
|
||||
List<String> bootstrapOpts = quorumArgs().toList();
|
||||
ConfigCommand.ConfigCommandOptions addOpts =
|
||||
new ConfigCommand.ConfigCommandOptions(toArray(bootstrapOpts,
|
||||
entityOp(resourceName),
|
||||
alterOpts,
|
||||
asList("--add-config", configStr)));
|
||||
List.of("--add-config", configStr)));
|
||||
addOpts.checkArgs();
|
||||
ConfigCommand.alterConfig(client, addOpts);
|
||||
}
|
||||
|
||||
private void alterConfigWithAdmin(Admin client, Map<String, String> config, List<String> alterOpts) {
|
||||
String configStr = transferConfigMapToString(config);
|
||||
List<String> bootstrapOpts = quorumArgs().collect(Collectors.toList());
|
||||
List<String> bootstrapOpts = quorumArgs().toList();
|
||||
ConfigCommand.ConfigCommandOptions addOpts =
|
||||
new ConfigCommand.ConfigCommandOptions(toArray(bootstrapOpts,
|
||||
alterOpts,
|
||||
asList("--add-config", configStr)));
|
||||
List.of("--add-config", configStr)));
|
||||
addOpts.checkArgs();
|
||||
ConfigCommand.alterConfig(client, addOpts);
|
||||
}
|
||||
|
@ -675,7 +669,7 @@ public class ConfigCommandIntegrationTest {
|
|||
|
||||
private Stream<ConfigEntry> getConfigEntryStream(Admin client,
|
||||
ConfigResource configResource) throws InterruptedException, ExecutionException {
|
||||
return client.describeConfigs(singletonList(configResource))
|
||||
return client.describeConfigs(List.of(configResource))
|
||||
.all()
|
||||
.get()
|
||||
.values()
|
||||
|
@ -689,8 +683,8 @@ public class ConfigCommandIntegrationTest {
|
|||
boolean hasDefaultValue,
|
||||
List<String> alterOpts) throws Exception {
|
||||
ConfigCommand.ConfigCommandOptions deleteOpts =
|
||||
new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, asList("--entity-name", brokerId),
|
||||
asList("--delete-config", String.join(",", config))));
|
||||
new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, List.of("--entity-name", brokerId),
|
||||
List.of("--delete-config", String.join(",", config))));
|
||||
deleteOpts.checkArgs();
|
||||
ConfigCommand.alterConfig(client, deleteOpts);
|
||||
verifyPerBrokerConfigValue(client, brokerId, config, hasDefaultValue);
|
||||
|
@ -700,11 +694,11 @@ public class ConfigCommandIntegrationTest {
|
|||
String groupName,
|
||||
Map<String, String> defaultConfigs,
|
||||
List<String> alterOpts) throws Exception {
|
||||
List<String> bootstrapOpts = quorumArgs().collect(Collectors.toList());
|
||||
List<String> bootstrapOpts = quorumArgs().toList();
|
||||
ConfigCommand.ConfigCommandOptions deleteOpts =
|
||||
new ConfigCommand.ConfigCommandOptions(toArray(bootstrapOpts,
|
||||
alterOpts,
|
||||
asList("--delete-config", String.join(",", defaultConfigs.keySet()))));
|
||||
List.of("--delete-config", String.join(",", defaultConfigs.keySet()))));
|
||||
deleteOpts.checkArgs();
|
||||
ConfigCommand.alterConfig(client, deleteOpts);
|
||||
verifyGroupConfig(client, groupName, defaultConfigs);
|
||||
|
@ -714,11 +708,11 @@ public class ConfigCommandIntegrationTest {
|
|||
String clientMetricsName,
|
||||
Map<String, String> defaultConfigs,
|
||||
List<String> alterOpts) throws Exception {
|
||||
List<String> bootstrapOpts = quorumArgs().collect(Collectors.toList());
|
||||
List<String> bootstrapOpts = quorumArgs().toList();
|
||||
ConfigCommand.ConfigCommandOptions deleteOpts =
|
||||
new ConfigCommand.ConfigCommandOptions(toArray(bootstrapOpts,
|
||||
alterOpts,
|
||||
asList("--delete-config", String.join(",", defaultConfigs.keySet()))));
|
||||
List.of("--delete-config", String.join(",", defaultConfigs.keySet()))));
|
||||
deleteOpts.checkArgs();
|
||||
ConfigCommand.alterConfig(client, deleteOpts);
|
||||
verifyClientMetricsConfig(client, clientMetricsName, defaultConfigs);
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.junit.jupiter.params.provider.ValueSource;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.AbstractMap.SimpleImmutableEntry;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -85,8 +84,8 @@ import static org.mockito.Mockito.verify;
|
|||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class ConfigCommandTest {
|
||||
private static final List<String> BROKER_BOOTSTRAP = Arrays.asList("--bootstrap-server", "localhost:9092");
|
||||
private static final List<String> CONTROLLER_BOOTSTRAP = Arrays.asList("--bootstrap-controller", "localhost:9093");
|
||||
private static final List<String> BROKER_BOOTSTRAP = List.of("--bootstrap-server", "localhost:9092");
|
||||
private static final List<String> CONTROLLER_BOOTSTRAP = List.of("--bootstrap-controller", "localhost:9093");
|
||||
|
||||
@Test
|
||||
public void shouldExitWithNonZeroStatusOnArgError() {
|
||||
|
@ -95,7 +94,7 @@ public class ConfigCommandTest {
|
|||
|
||||
@Test
|
||||
public void shouldExitWithNonZeroStatusAlterUserQuotaWithoutEntityName() {
|
||||
assertNonZeroStatusExit(toArray(BROKER_BOOTSTRAP, Arrays.asList(
|
||||
assertNonZeroStatusExit(toArray(BROKER_BOOTSTRAP, List.of(
|
||||
"--entity-type", "users",
|
||||
"--alter", "--add-config", "consumer_byte_rate=20000")));
|
||||
}
|
||||
|
@ -110,7 +109,7 @@ public class ConfigCommandTest {
|
|||
|
||||
@Test
|
||||
public void shouldExitWithNonZeroStatusIfBothBootstrapServerAndBootstrapControllerGiven() {
|
||||
assertNonZeroStatusExit(toArray(BROKER_BOOTSTRAP, CONTROLLER_BOOTSTRAP, Arrays.asList(
|
||||
assertNonZeroStatusExit(toArray(BROKER_BOOTSTRAP, CONTROLLER_BOOTSTRAP, List.of(
|
||||
"--describe", "--broker-defaults")));
|
||||
}
|
||||
|
||||
|
@ -405,9 +404,9 @@ public class ConfigCommandTest {
|
|||
|
||||
File file = TestUtils.tempFile(fileContents);
|
||||
|
||||
List<String> addConfigFileArgs = Arrays.asList("--add-config-file", file.getPath());
|
||||
List<String> addConfigFileArgs = List.of("--add-config-file", file.getPath());
|
||||
|
||||
ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092",
|
||||
ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092",
|
||||
"--entity-name", "1",
|
||||
"--entity-type", "brokers",
|
||||
"--alter"),
|
||||
|
@ -423,7 +422,7 @@ public class ConfigCommandTest {
|
|||
}
|
||||
|
||||
public void testExpectedEntityTypeNames(List<String> expectedTypes, List<String> expectedNames, List<String> connectOpts, String... args) {
|
||||
ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList(connectOpts.get(0), connectOpts.get(1), "--describe"), Arrays.asList(args)));
|
||||
ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(List.of(connectOpts.get(0), connectOpts.get(1), "--describe"), List.of(args)));
|
||||
createOpts.checkArgs();
|
||||
assertEquals(createOpts.entityTypes().toSeq(), seq(expectedTypes));
|
||||
assertEquals(createOpts.entityNames().toSeq(), seq(expectedNames));
|
||||
|
@ -431,31 +430,31 @@ public class ConfigCommandTest {
|
|||
|
||||
@Test
|
||||
public void testOptionEntityTypeNames() {
|
||||
List<String> connectOpts = Arrays.asList("--bootstrap-server", "localhost:9092");
|
||||
List<String> connectOpts = List.of("--bootstrap-server", "localhost:9092");
|
||||
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.TOPIC.value()), Collections.singletonList("A"), connectOpts, "--entity-type", "topics", "--entity-name", "A");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.IP.value()), Collections.singletonList("1.2.3.4"), connectOpts, "--entity-name", "1.2.3.4", "--entity-type", "ips");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.CLIENT_METRICS.value()), Collections.singletonList("A"), connectOpts, "--entity-type", "client-metrics", "--entity-name", "A");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.GROUP.value()), Collections.singletonList("A"), connectOpts, "--entity-type", "groups", "--entity-name", "A");
|
||||
testExpectedEntityTypeNames(Arrays.asList(ConfigType.USER.value(), ConfigType.CLIENT.value()), Arrays.asList("A", ""), connectOpts,
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.TOPIC.value()), List.of("A"), connectOpts, "--entity-type", "topics", "--entity-name", "A");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.IP.value()), List.of("1.2.3.4"), connectOpts, "--entity-name", "1.2.3.4", "--entity-type", "ips");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.CLIENT_METRICS.value()), List.of("A"), connectOpts, "--entity-type", "client-metrics", "--entity-name", "A");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.GROUP.value()), List.of("A"), connectOpts, "--entity-type", "groups", "--entity-name", "A");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.USER.value(), ConfigType.CLIENT.value()), List.of("A", ""), connectOpts,
|
||||
"--entity-type", "users", "--entity-type", "clients", "--entity-name", "A", "--entity-default");
|
||||
testExpectedEntityTypeNames(Arrays.asList(ConfigType.USER.value(), ConfigType.CLIENT.value()), Arrays.asList("", "B"), connectOpts,
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.USER.value(), ConfigType.CLIENT.value()), List.of("", "B"), connectOpts,
|
||||
"--entity-default", "--entity-name", "B", "--entity-type", "users", "--entity-type", "clients");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.TOPIC.value()), Collections.singletonList("A"), connectOpts, "--topic", "A");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.IP.value()), Collections.singletonList("1.2.3.4"), connectOpts, "--ip", "1.2.3.4");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.GROUP.value()), Collections.singletonList("A"), connectOpts, "--group", "A");
|
||||
testExpectedEntityTypeNames(Arrays.asList(ConfigType.CLIENT.value(), ConfigType.USER.value()), Arrays.asList("B", "A"), connectOpts, "--client", "B", "--user", "A");
|
||||
testExpectedEntityTypeNames(Arrays.asList(ConfigType.CLIENT.value(), ConfigType.USER.value()), Arrays.asList("B", ""), connectOpts, "--client", "B", "--user-defaults");
|
||||
testExpectedEntityTypeNames(Arrays.asList(ConfigType.CLIENT.value(), ConfigType.USER.value()), Collections.singletonList("A"), connectOpts,
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.TOPIC.value()), List.of("A"), connectOpts, "--topic", "A");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.IP.value()), List.of("1.2.3.4"), connectOpts, "--ip", "1.2.3.4");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.GROUP.value()), List.of("A"), connectOpts, "--group", "A");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.CLIENT.value(), ConfigType.USER.value()), List.of("B", "A"), connectOpts, "--client", "B", "--user", "A");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.CLIENT.value(), ConfigType.USER.value()), List.of("B", ""), connectOpts, "--client", "B", "--user-defaults");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.CLIENT.value(), ConfigType.USER.value()), List.of("A"), connectOpts,
|
||||
"--entity-type", "clients", "--entity-type", "users", "--entity-name", "A");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.TOPIC.value()), Collections.emptyList(), connectOpts, "--entity-type", "topics");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.IP.value()), Collections.emptyList(), connectOpts, "--entity-type", "ips");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.GROUP.value()), Collections.emptyList(), connectOpts, "--entity-type", "groups");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.CLIENT_METRICS.value()), Collections.emptyList(), connectOpts, "--entity-type", "client-metrics");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.BROKER.value()), Collections.singletonList("0"), connectOpts, "--entity-name", "0", "--entity-type", "brokers");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.BROKER.value()), Collections.singletonList("0"), connectOpts, "--broker", "0");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.USER.value()), Collections.emptyList(), connectOpts, "--entity-type", "users");
|
||||
testExpectedEntityTypeNames(Collections.singletonList(ConfigType.BROKER.value()), Collections.emptyList(), connectOpts, "--entity-type", "brokers");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.TOPIC.value()), List.of(), connectOpts, "--entity-type", "topics");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.IP.value()), List.of(), connectOpts, "--entity-type", "ips");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.GROUP.value()), List.of(), connectOpts, "--entity-type", "groups");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.CLIENT_METRICS.value()), List.of(), connectOpts, "--entity-type", "client-metrics");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.BROKER.value()), List.of("0"), connectOpts, "--entity-name", "0", "--entity-type", "brokers");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.BROKER.value()), List.of("0"), connectOpts, "--broker", "0");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.USER.value()), List.of(), connectOpts, "--entity-type", "users");
|
||||
testExpectedEntityTypeNames(List.of(ConfigType.BROKER.value()), List.of(), connectOpts, "--entity-type", "brokers");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -501,31 +500,23 @@ public class ConfigCommandTest {
|
|||
}
|
||||
|
||||
private Entry<List<String>, Map<String, String>> argsAndExpectedEntity(Optional<String> entityName, String entityType) {
|
||||
String command;
|
||||
switch (entityType) {
|
||||
case ClientQuotaEntity.USER:
|
||||
command = "users";
|
||||
break;
|
||||
case ClientQuotaEntity.CLIENT_ID:
|
||||
command = "clients";
|
||||
break;
|
||||
case ClientQuotaEntity.IP:
|
||||
command = "ips";
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown command: " + entityType);
|
||||
}
|
||||
String command = switch (entityType) {
|
||||
case ClientQuotaEntity.USER -> ConfigType.USER.value();
|
||||
case ClientQuotaEntity.CLIENT_ID -> ConfigType.CLIENT.value();
|
||||
case ClientQuotaEntity.IP -> ConfigType.IP.value();
|
||||
default -> throw new IllegalArgumentException("Unknown command: " + entityType);
|
||||
};
|
||||
|
||||
return entityName.map(name -> {
|
||||
if (name.isEmpty())
|
||||
return new SimpleImmutableEntry<>(Arrays.asList("--entity-type", command, "--entity-default"), Collections.singletonMap(entityType, (String) null));
|
||||
return new SimpleImmutableEntry<>(Arrays.asList("--entity-type", command, "--entity-name", name), Collections.singletonMap(entityType, name));
|
||||
}).orElse(new SimpleImmutableEntry<>(Collections.emptyList(), Collections.emptyMap()));
|
||||
return new SimpleImmutableEntry<>(List.of("--entity-type", command, "--entity-default"), Collections.singletonMap(entityType, (String) null));
|
||||
return new SimpleImmutableEntry<>(List.of("--entity-type", command, "--entity-name", name), Map.of(entityType, name));
|
||||
}).orElse(new SimpleImmutableEntry<>(List.of(), Map.of()));
|
||||
}
|
||||
|
||||
private void verifyAlterCommandFails(String expectedErrorMessage, List<String> alterOpts) {
|
||||
Admin mockAdminClient = mock(Admin.class);
|
||||
ConfigCommand.ConfigCommandOptions opts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092",
|
||||
ConfigCommand.ConfigCommandOptions opts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092",
|
||||
"--alter"), alterOpts));
|
||||
IllegalArgumentException e = assertThrows(IllegalArgumentException.class, () -> ConfigCommand.alterConfig(mockAdminClient, opts));
|
||||
assertTrue(e.getMessage().contains(expectedErrorMessage), "Unexpected exception: " + e);
|
||||
|
@ -535,25 +526,25 @@ public class ConfigCommandTest {
|
|||
public void shouldNotAlterNonQuotaIpConfigsUsingBootstrapServer() {
|
||||
// when using --bootstrap-server, it should be illegal to alter anything that is not a connection quota
|
||||
// for ip entities
|
||||
List<String> ipEntityOpts = Arrays.asList("--entity-type", "ips", "--entity-name", "127.0.0.1");
|
||||
List<String> ipEntityOpts = List.of("--entity-type", "ips", "--entity-name", "127.0.0.1");
|
||||
String invalidProp = "some_config";
|
||||
verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, Arrays.asList("--add-config", "connection_creation_rate=10000,some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, Arrays.asList("--add-config", "some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, Arrays.asList("--delete-config", "connection_creation_rate=10000,some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, Arrays.asList("--delete-config", "some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, List.of("--add-config", "connection_creation_rate=10000,some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, List.of("--add-config", "some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, List.of("--delete-config", "connection_creation_rate=10000,some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, List.of("--delete-config", "some_config=10")));
|
||||
}
|
||||
|
||||
private void verifyDescribeQuotas(List<String> describeArgs, ClientQuotaFilter expectedFilter) {
|
||||
ConfigCommand.ConfigCommandOptions describeOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092",
|
||||
ConfigCommand.ConfigCommandOptions describeOpts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092",
|
||||
"--describe"), describeArgs));
|
||||
KafkaFutureImpl<Map<ClientQuotaEntity, Map<String, Double>>> describeFuture = new KafkaFutureImpl<>();
|
||||
describeFuture.complete(Collections.emptyMap());
|
||||
describeFuture.complete(Map.of());
|
||||
DescribeClientQuotasResult describeResult = mock(DescribeClientQuotasResult.class);
|
||||
when(describeResult.entities()).thenReturn(describeFuture);
|
||||
|
||||
AtomicBoolean describedConfigs = new AtomicBoolean();
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public DescribeClientQuotasResult describeClientQuotas(ClientQuotaFilter filter, DescribeClientQuotasOptions options) {
|
||||
assertTrue(filter.strict());
|
||||
|
@ -570,24 +561,24 @@ public class ConfigCommandTest {
|
|||
public void testDescribeIpConfigs() {
|
||||
String entityType = ClientQuotaEntity.IP;
|
||||
String knownHost = "1.2.3.4";
|
||||
ClientQuotaFilter defaultIpFilter = ClientQuotaFilter.containsOnly(Collections.singletonList(ClientQuotaFilterComponent.ofDefaultEntity(entityType)));
|
||||
ClientQuotaFilter singleIpFilter = ClientQuotaFilter.containsOnly(Collections.singletonList(ClientQuotaFilterComponent.ofEntity(entityType, knownHost)));
|
||||
ClientQuotaFilter allIpsFilter = ClientQuotaFilter.containsOnly(Collections.singletonList(ClientQuotaFilterComponent.ofEntityType(entityType)));
|
||||
verifyDescribeQuotas(Arrays.asList("--entity-default", "--entity-type", "ips"), defaultIpFilter);
|
||||
verifyDescribeQuotas(Collections.singletonList("--ip-defaults"), defaultIpFilter);
|
||||
verifyDescribeQuotas(Arrays.asList("--entity-type", "ips", "--entity-name", knownHost), singleIpFilter);
|
||||
verifyDescribeQuotas(Arrays.asList("--ip", knownHost), singleIpFilter);
|
||||
verifyDescribeQuotas(Arrays.asList("--entity-type", "ips"), allIpsFilter);
|
||||
ClientQuotaFilter defaultIpFilter = ClientQuotaFilter.containsOnly(List.of(ClientQuotaFilterComponent.ofDefaultEntity(entityType)));
|
||||
ClientQuotaFilter singleIpFilter = ClientQuotaFilter.containsOnly(List.of(ClientQuotaFilterComponent.ofEntity(entityType, knownHost)));
|
||||
ClientQuotaFilter allIpsFilter = ClientQuotaFilter.containsOnly(List.of(ClientQuotaFilterComponent.ofEntityType(entityType)));
|
||||
verifyDescribeQuotas(List.of("--entity-default", "--entity-type", "ips"), defaultIpFilter);
|
||||
verifyDescribeQuotas(List.of("--ip-defaults"), defaultIpFilter);
|
||||
verifyDescribeQuotas(List.of("--entity-type", "ips", "--entity-name", knownHost), singleIpFilter);
|
||||
verifyDescribeQuotas(List.of("--ip", knownHost), singleIpFilter);
|
||||
verifyDescribeQuotas(List.of("--entity-type", "ips"), allIpsFilter);
|
||||
}
|
||||
|
||||
public void verifyAlterQuotas(List<String> alterOpts, ClientQuotaEntity expectedAlterEntity,
|
||||
Map<String, Double> expectedProps, Set<ClientQuotaAlteration.Op> expectedAlterOps) {
|
||||
ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092",
|
||||
ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092",
|
||||
"--alter"), alterOpts));
|
||||
|
||||
AtomicBoolean describedConfigs = new AtomicBoolean();
|
||||
KafkaFutureImpl<Map<ClientQuotaEntity, Map<String, Double>>> describeFuture = new KafkaFutureImpl<>();
|
||||
describeFuture.complete(Collections.singletonMap(expectedAlterEntity, expectedProps));
|
||||
describeFuture.complete(Map.of(expectedAlterEntity, expectedProps));
|
||||
DescribeClientQuotasResult describeResult = mock(DescribeClientQuotasResult.class);
|
||||
when(describeResult.entities()).thenReturn(describeFuture);
|
||||
|
||||
|
@ -606,7 +597,7 @@ public class ConfigCommandTest {
|
|||
when(alterResult.all()).thenReturn(alterFuture);
|
||||
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public DescribeClientQuotasResult describeClientQuotas(ClientQuotaFilter filter, DescribeClientQuotasOptions options) {
|
||||
assertTrue(filter.strict());
|
||||
|
@ -638,11 +629,11 @@ public class ConfigCommandTest {
|
|||
Entry<List<String>, Map<String, String>> defaultIpArgsAndEntity = argsAndExpectedEntity(Optional.of(""), ClientQuotaEntity.IP);
|
||||
|
||||
|
||||
List<String> deleteArgs = Arrays.asList("--delete-config", "connection_creation_rate");
|
||||
List<String> deleteArgs = List.of("--delete-config", "connection_creation_rate");
|
||||
Set<ClientQuotaAlteration.Op> deleteAlterationOps = Set.of(new ClientQuotaAlteration.Op("connection_creation_rate", null));
|
||||
Map<String, Double> propsToDelete = Collections.singletonMap("connection_creation_rate", 50.0);
|
||||
Map<String, Double> propsToDelete = Map.of("connection_creation_rate", 50.0);
|
||||
|
||||
List<String> addArgs = Arrays.asList("--add-config", "connection_creation_rate=100");
|
||||
List<String> addArgs = List.of("--add-config", "connection_creation_rate=100");
|
||||
Set<ClientQuotaAlteration.Op> addAlterationOps = Set.of(new ClientQuotaAlteration.Op("connection_creation_rate", 100.0));
|
||||
|
||||
verifyAlterQuotas(
|
||||
|
@ -653,7 +644,7 @@ public class ConfigCommandTest {
|
|||
verifyAlterQuotas(
|
||||
concat(singleIpArgsAndEntity.getKey(), addArgs),
|
||||
new ClientQuotaEntity(singleIpArgsAndEntity.getValue()),
|
||||
Collections.emptyMap(),
|
||||
Map.of(),
|
||||
addAlterationOps);
|
||||
verifyAlterQuotas(
|
||||
concat(defaultIpArgsAndEntity.getKey(), deleteArgs),
|
||||
|
@ -663,14 +654,14 @@ public class ConfigCommandTest {
|
|||
verifyAlterQuotas(
|
||||
concat(defaultIpArgsAndEntity.getKey(), addArgs),
|
||||
new ClientQuotaEntity(defaultIpArgsAndEntity.getValue()),
|
||||
Collections.emptyMap(),
|
||||
Map.of(),
|
||||
addAlterationOps);
|
||||
}
|
||||
|
||||
private void verifyAlterUserClientQuotas(String user, String client) {
|
||||
List<String> alterArgs = Arrays.asList("--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000",
|
||||
List<String> alterArgs = List.of("--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000",
|
||||
"--delete-config", "request_percentage");
|
||||
Map<String, Double> propsToDelete = Collections.singletonMap("request_percentage", 50.0);
|
||||
Map<String, Double> propsToDelete = Map.of("request_percentage", 50.0);
|
||||
|
||||
Set<ClientQuotaAlteration.Op> alterationOps = Set.of(
|
||||
new ClientQuotaAlteration.Op("consumer_byte_rate", 20000d),
|
||||
|
@ -700,10 +691,10 @@ public class ConfigCommandTest {
|
|||
verifyAlterUserClientQuotas(null, "");
|
||||
}
|
||||
|
||||
private final List<String> userEntityOpts = Arrays.asList("--entity-type", "users", "--entity-name", "admin");
|
||||
private final List<String> clientEntityOpts = Arrays.asList("--entity-type", "clients", "--entity-name", "admin");
|
||||
private final List<String> addScramOpts = Arrays.asList("--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]");
|
||||
private final List<String> deleteScramOpts = Arrays.asList("--delete-config", "SCRAM-SHA-256");
|
||||
private final List<String> userEntityOpts = List.of("--entity-type", "users", "--entity-name", "admin");
|
||||
private final List<String> clientEntityOpts = List.of("--entity-type", "clients", "--entity-name", "admin");
|
||||
private final List<String> addScramOpts = List.of("--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]");
|
||||
private final List<String> deleteScramOpts = List.of("--delete-config", "SCRAM-SHA-256");
|
||||
|
||||
@Test
|
||||
public void shouldNotAlterNonQuotaNonScramUserOrClientConfigUsingBootstrapServer() {
|
||||
|
@ -711,13 +702,13 @@ public class ConfigCommandTest {
|
|||
// for both user and client entities
|
||||
String invalidProp = "some_config";
|
||||
verifyAlterCommandFails(invalidProp, concat(userEntityOpts,
|
||||
Arrays.asList("-add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10")));
|
||||
List.of("-add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(userEntityOpts,
|
||||
Arrays.asList("--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(clientEntityOpts, Arrays.asList("--add-config", "some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(userEntityOpts, Arrays.asList("--delete-config", "consumer_byte_rate,some_config")));
|
||||
verifyAlterCommandFails(invalidProp, concat(userEntityOpts, Arrays.asList("--delete-config", "SCRAM-SHA-256,some_config")));
|
||||
verifyAlterCommandFails(invalidProp, concat(clientEntityOpts, Arrays.asList("--delete-config", "some_config")));
|
||||
List.of("--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(clientEntityOpts, List.of("--add-config", "some_config=10")));
|
||||
verifyAlterCommandFails(invalidProp, concat(userEntityOpts, List.of("--delete-config", "consumer_byte_rate,some_config")));
|
||||
verifyAlterCommandFails(invalidProp, concat(userEntityOpts, List.of("--delete-config", "SCRAM-SHA-256,some_config")));
|
||||
verifyAlterCommandFails(invalidProp, concat(clientEntityOpts, List.of("--delete-config", "some_config")));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -731,16 +722,16 @@ public class ConfigCommandTest {
|
|||
public void shouldNotCreateUserScramCredentialConfigWithUnderMinimumIterationsUsingBootstrapServer() {
|
||||
// when using --bootstrap-server, it should be illegal to create a SCRAM credential for a user
|
||||
// with an iterations value less than the minimum
|
||||
verifyAlterCommandFails("SCRAM-SHA-256", concat(userEntityOpts, Arrays.asList("--add-config", "SCRAM-SHA-256=[iterations=100,password=foo-secret]")));
|
||||
verifyAlterCommandFails("SCRAM-SHA-256", concat(userEntityOpts, List.of("--add-config", "SCRAM-SHA-256=[iterations=100,password=foo-secret]")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAlterUserScramCredentialAndClientQuotaConfigsSimultaneouslyUsingBootstrapServer() {
|
||||
// when using --bootstrap-server, it should be illegal to alter both SCRAM credentials and quotas for user entities
|
||||
String expectedErrorMessage = "SCRAM-SHA-256";
|
||||
List<String> secondUserEntityOpts = Arrays.asList("--entity-type", "users", "--entity-name", "admin1");
|
||||
List<String> addQuotaOpts = Arrays.asList("--add-config", "consumer_byte_rate=20000");
|
||||
List<String> deleteQuotaOpts = Arrays.asList("--delete-config", "consumer_byte_rate");
|
||||
List<String> secondUserEntityOpts = List.of("--entity-type", "users", "--entity-name", "admin1");
|
||||
List<String> addQuotaOpts = List.of("--add-config", "consumer_byte_rate=20000");
|
||||
List<String> deleteQuotaOpts = List.of("--delete-config", "consumer_byte_rate");
|
||||
|
||||
verifyAlterCommandFails(expectedErrorMessage, concat(userEntityOpts, addScramOpts, userEntityOpts, deleteQuotaOpts));
|
||||
verifyAlterCommandFails(expectedErrorMessage, concat(userEntityOpts, addScramOpts, secondUserEntityOpts, deleteQuotaOpts));
|
||||
|
@ -758,11 +749,11 @@ public class ConfigCommandTest {
|
|||
// User SCRAM credentials should not be described when specifying
|
||||
// --describe --entity-type users --entity-default (or --user-defaults) with --bootstrap-server
|
||||
KafkaFutureImpl<Map<ClientQuotaEntity, Map<String, Double>>> describeFuture = new KafkaFutureImpl<>();
|
||||
describeFuture.complete(Collections.singletonMap(new ClientQuotaEntity(Collections.singletonMap("", "")), Collections.singletonMap("request_percentage", 50.0)));
|
||||
describeFuture.complete(Map.of(new ClientQuotaEntity(Map.of("", "")), Map.of("request_percentage", 50.0)));
|
||||
DescribeClientQuotasResult describeClientQuotasResult = mock(DescribeClientQuotasResult.class);
|
||||
when(describeClientQuotasResult.entities()).thenReturn(describeFuture);
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public DescribeClientQuotasResult describeClientQuotas(ClientQuotaFilter filter, DescribeClientQuotasOptions options) {
|
||||
return describeClientQuotasResult;
|
||||
|
@ -773,15 +764,15 @@ public class ConfigCommandTest {
|
|||
throw new IllegalStateException("Incorrectly described SCRAM credentials when specifying --entity-default with --bootstrap-server");
|
||||
}
|
||||
};
|
||||
ConfigCommand.ConfigCommandOptions opts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092", "--describe"), requestOpts));
|
||||
ConfigCommand.ConfigCommandOptions opts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092", "--describe"), requestOpts));
|
||||
ConfigCommand.describeConfig(mockAdminClient, opts); // fails if describeUserScramCredentials() is invoked
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotDescribeUserScramCredentialsWithEntityDefaultUsingBootstrapServer() {
|
||||
String expectedMsg = "The use of --entity-default or --user-defaults is not allowed with User SCRAM Credentials using --bootstrap-server.";
|
||||
List<String> defaultUserOpt = Collections.singletonList("--user-defaults");
|
||||
List<String> verboseDefaultUserOpts = Arrays.asList("--entity-type", "users", "--entity-default");
|
||||
List<String> defaultUserOpt = List.of("--user-defaults");
|
||||
List<String> verboseDefaultUserOpts = List.of("--entity-type", "users", "--entity-default");
|
||||
verifyAlterCommandFails(expectedMsg, concat(verboseDefaultUserOpts, addScramOpts));
|
||||
verifyAlterCommandFails(expectedMsg, concat(verboseDefaultUserOpts, deleteScramOpts));
|
||||
verifyUserScramCredentialsNotDescribed(verboseDefaultUserOpts);
|
||||
|
@ -813,9 +804,9 @@ public class ConfigCommandTest {
|
|||
AtomicBoolean alteredConfigs = new AtomicBoolean();
|
||||
|
||||
ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, resourceName);
|
||||
List<ConfigEntry> configEntries = Arrays.asList(newConfigEntry("min.insync.replicas", "1"), newConfigEntry("unclean.leader.election.enable", "1"));
|
||||
List<ConfigEntry> configEntries = List.of(newConfigEntry("min.insync.replicas", "1"), newConfigEntry("unclean.leader.election.enable", "1"));
|
||||
KafkaFutureImpl<Map<ConfigResource, Config>> future = new KafkaFutureImpl<>();
|
||||
future.complete(Collections.singletonMap(resource, new Config(configEntries)));
|
||||
future.complete(Map.of(resource, new Config(configEntries)));
|
||||
DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class);
|
||||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
|
@ -825,14 +816,14 @@ public class ConfigCommandTest {
|
|||
when(alterResult.all()).thenReturn(alterFuture);
|
||||
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertFalse(options.includeSynonyms(), "Config synonyms requested unnecessarily");
|
||||
assertEquals(1, resources.size());
|
||||
ConfigResource res = resources.iterator().next();
|
||||
assertEquals(res.type(), ConfigResource.Type.TOPIC);
|
||||
assertEquals(res.name(), resourceName);
|
||||
assertEquals(ConfigResource.Type.TOPIC, res.type());
|
||||
assertEquals(resourceName, res.name());
|
||||
return describeResult;
|
||||
}
|
||||
|
||||
|
@ -869,7 +860,7 @@ public class ConfigCommandTest {
|
|||
}
|
||||
|
||||
public ConfigEntry newConfigEntry(String name, String value) {
|
||||
return ConfigTest.newConfigEntry(name, value, ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, Collections.emptyList());
|
||||
return ConfigTest.newConfigEntry(name, value, ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, List.of());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -883,16 +874,16 @@ public class ConfigCommandTest {
|
|||
|
||||
ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, resourceName);
|
||||
KafkaFutureImpl<Map<ConfigResource, Config>> future = new KafkaFutureImpl<>();
|
||||
future.complete(Collections.singletonMap(resource, new Config(Collections.emptyList())));
|
||||
future.complete(Map.of(resource, new Config(List.of())));
|
||||
DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class);
|
||||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertTrue(options.includeSynonyms(), "Synonyms not requested");
|
||||
assertEquals(Collections.singleton(resource), new HashSet<>(resources));
|
||||
assertEquals(Set.of(resource), new HashSet<>(resources));
|
||||
return describeResult;
|
||||
}
|
||||
};
|
||||
|
@ -903,7 +894,7 @@ public class ConfigCommandTest {
|
|||
@Test
|
||||
public void shouldAddBrokerLoggerConfig() {
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
verifyAlterBrokerLoggerConfig(node, "1", "1", Arrays.asList(
|
||||
verifyAlterBrokerLoggerConfig(node, "1", "1", List.of(
|
||||
new ConfigEntry("kafka.log.LogCleaner", "INFO"),
|
||||
new ConfigEntry("kafka.server.ReplicaManager", "INFO"),
|
||||
new ConfigEntry("kafka.server.KafkaApi", "INFO")
|
||||
|
@ -970,7 +961,7 @@ public class ConfigCommandTest {
|
|||
Node node = new Node(1, "localhost", 9092);
|
||||
// verifyAlterBrokerLoggerConfig tries to alter kafka.log.LogCleaner, kafka.server.ReplicaManager and kafka.server.KafkaApi
|
||||
// yet, we make it so DescribeConfigs returns only one logger, implying that kafka.server.ReplicaManager and kafka.log.LogCleaner are invalid
|
||||
assertThrows(InvalidConfigurationException.class, () -> verifyAlterBrokerLoggerConfig(node, "1", "1", Collections.singletonList(
|
||||
assertThrows(InvalidConfigurationException.class, () -> verifyAlterBrokerLoggerConfig(node, "1", "1", List.of(
|
||||
new ConfigEntry("kafka.server.KafkaApi", "INFO")
|
||||
)));
|
||||
}
|
||||
|
@ -978,17 +969,17 @@ public class ConfigCommandTest {
|
|||
@Test
|
||||
public void shouldAddDefaultBrokerDynamicConfig() {
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
verifyAlterBrokerConfig(node, "", Collections.singletonList("--entity-default"));
|
||||
verifyAlterBrokerConfig(node, "", List.of("--entity-default"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldAddBrokerDynamicConfig() {
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
verifyAlterBrokerConfig(node, "1", Arrays.asList("--entity-name", "1"));
|
||||
verifyAlterBrokerConfig(node, "1", List.of("--entity-name", "1"));
|
||||
}
|
||||
|
||||
public void verifyAlterBrokerConfig(Node node, String resourceName, List<String> resourceOpts) {
|
||||
String[] optsList = toArray(Arrays.asList("--bootstrap-server", "localhost:9092",
|
||||
String[] optsList = toArray(List.of("--bootstrap-server", "localhost:9092",
|
||||
"--entity-type", "brokers",
|
||||
"--alter",
|
||||
"--add-config", "message.max.bytes=10,leader.replication.throttled.rate=10"), resourceOpts);
|
||||
|
@ -997,9 +988,9 @@ public class ConfigCommandTest {
|
|||
brokerConfigs.put("num.io.threads", "5");
|
||||
|
||||
ConfigResource resource = new ConfigResource(ConfigResource.Type.BROKER, resourceName);
|
||||
List<ConfigEntry> configEntries = Collections.singletonList(new ConfigEntry("num.io.threads", "5"));
|
||||
List<ConfigEntry> configEntries = List.of(new ConfigEntry("num.io.threads", "5"));
|
||||
KafkaFutureImpl<Map<ConfigResource, Config>> future = new KafkaFutureImpl<>();
|
||||
future.complete(Collections.singletonMap(resource, new Config(configEntries)));
|
||||
future.complete(Map.of(resource, new Config(configEntries)));
|
||||
DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class);
|
||||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
|
@ -1008,7 +999,7 @@ public class ConfigCommandTest {
|
|||
AlterConfigsResult alterResult = mock(AlterConfigsResult.class);
|
||||
when(alterResult.all()).thenReturn(alterFuture);
|
||||
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertFalse(options.includeSynonyms(), "Config synonyms requested unnecessarily");
|
||||
|
@ -1049,7 +1040,7 @@ public class ConfigCommandTest {
|
|||
ConfigResource resourceCustom = new ConfigResource(ConfigResource.Type.BROKER, "1");
|
||||
ConfigResource resourceDefault = new ConfigResource(ConfigResource.Type.BROKER, brokerDefaultEntityName);
|
||||
KafkaFutureImpl<Map<ConfigResource, Config>> future = new KafkaFutureImpl<>();
|
||||
Config emptyConfig = new Config(Collections.emptyList());
|
||||
Config emptyConfig = new Config(List.of());
|
||||
Map<ConfigResource, Config> resultMap = new HashMap<>();
|
||||
resultMap.put(resourceCustom, emptyConfig);
|
||||
resultMap.put(resourceDefault, emptyConfig);
|
||||
|
@ -1059,7 +1050,7 @@ public class ConfigCommandTest {
|
|||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertTrue(options.includeSynonyms(), "Synonyms not requested");
|
||||
|
@ -1087,7 +1078,7 @@ public class ConfigCommandTest {
|
|||
|
||||
ConfigResource resource = new ConfigResource(ConfigResource.Type.BROKER_LOGGER, resourceName);
|
||||
KafkaFutureImpl<Map<ConfigResource, Config>> future = new KafkaFutureImpl<>();
|
||||
future.complete(Collections.singletonMap(resource, new Config(describeConfigEntries)));
|
||||
future.complete(Map.of(resource, new Config(describeConfigEntries)));
|
||||
DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class);
|
||||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
|
@ -1096,7 +1087,7 @@ public class ConfigCommandTest {
|
|||
AlterConfigsResult alterResult = mock(AlterConfigsResult.class);
|
||||
when(alterResult.all()).thenReturn(alterFuture);
|
||||
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertEquals(1, resources.size());
|
||||
|
@ -1115,7 +1106,7 @@ public class ConfigCommandTest {
|
|||
assertEquals(ConfigResource.Type.BROKER_LOGGER, res.type());
|
||||
assertEquals(3, alterConfigOps.size());
|
||||
|
||||
List<AlterConfigOp> expectedConfigOps = Arrays.asList(
|
||||
List<AlterConfigOp> expectedConfigOps = List.of(
|
||||
new AlterConfigOp(new ConfigEntry("kafka.server.ReplicaManager", ""), AlterConfigOp.OpType.DELETE),
|
||||
new AlterConfigOp(new ConfigEntry("kafka.server.KafkaApi", ""), AlterConfigOp.OpType.DELETE),
|
||||
new AlterConfigOp(new ConfigEntry("kafka.log.LogCleaner", "DEBUG"), AlterConfigOp.OpType.SET)
|
||||
|
@ -1174,14 +1165,14 @@ public class ConfigCommandTest {
|
|||
"--delete-config", "missing_config1, missing_config2"));
|
||||
|
||||
ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, resourceName);
|
||||
List<ConfigEntry> configEntries = Collections.emptyList();
|
||||
List<ConfigEntry> configEntries = List.of();
|
||||
KafkaFutureImpl<Map<ConfigResource, Config>> future = new KafkaFutureImpl<>();
|
||||
future.complete(Collections.singletonMap(resource, new Config(configEntries)));
|
||||
future.complete(Map.of(resource, new Config(configEntries)));
|
||||
DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class);
|
||||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertEquals(1, resources.size());
|
||||
|
@ -1199,15 +1190,15 @@ public class ConfigCommandTest {
|
|||
@Test
|
||||
public void shouldAlterClientMetricsConfig() {
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
verifyAlterClientMetricsConfig(node, "1", Arrays.asList("--entity-type", "client-metrics", "--entity-name", "1"));
|
||||
verifyAlterClientMetricsConfig(node, "1", List.of("--entity-type", "client-metrics", "--entity-name", "1"));
|
||||
|
||||
// Test for the --client-metrics alias
|
||||
node = new Node(1, "localhost", 9092);
|
||||
verifyAlterClientMetricsConfig(node, "1", Arrays.asList("--client-metrics", "1"));
|
||||
verifyAlterClientMetricsConfig(node, "1", List.of("--client-metrics", "1"));
|
||||
}
|
||||
|
||||
private void verifyAlterClientMetricsConfig(Node node, String resourceName, List<String> resourceOpts) {
|
||||
List<String> optsList = concat(Arrays.asList("--bootstrap-server", "localhost:9092",
|
||||
List<String> optsList = concat(List.of("--bootstrap-server", "localhost:9092",
|
||||
"--alter",
|
||||
"--delete-config", "interval.ms",
|
||||
"--add-config", "metrics=org.apache.kafka.consumer.," +
|
||||
|
@ -1215,11 +1206,11 @@ public class ConfigCommandTest {
|
|||
ConfigCommand.ConfigCommandOptions alterOpts = new ConfigCommand.ConfigCommandOptions(toArray(optsList));
|
||||
|
||||
ConfigResource resource = new ConfigResource(ConfigResource.Type.CLIENT_METRICS, resourceName);
|
||||
List<ConfigEntry> configEntries = Collections.singletonList(new ConfigEntry("interval.ms", "1000",
|
||||
ConfigEntry.ConfigSource.DYNAMIC_CLIENT_METRICS_CONFIG, false, false, Collections.emptyList(),
|
||||
List<ConfigEntry> configEntries = List.of(new ConfigEntry("interval.ms", "1000",
|
||||
ConfigEntry.ConfigSource.DYNAMIC_CLIENT_METRICS_CONFIG, false, false, List.of(),
|
||||
ConfigEntry.ConfigType.UNKNOWN, null));
|
||||
KafkaFutureImpl<Map<ConfigResource, Config>> future = new KafkaFutureImpl<>();
|
||||
future.complete(Collections.singletonMap(resource, new Config(configEntries)));
|
||||
future.complete(Map.of(resource, new Config(configEntries)));
|
||||
DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class);
|
||||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
|
@ -1228,7 +1219,7 @@ public class ConfigCommandTest {
|
|||
AlterConfigsResult alterResult = mock(AlterConfigsResult.class);
|
||||
when(alterResult.all()).thenReturn(alterFuture);
|
||||
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertFalse(options.includeSynonyms(), "Config synonyms requested unnecessarily");
|
||||
|
@ -1248,7 +1239,7 @@ public class ConfigCommandTest {
|
|||
assertEquals(ConfigResource.Type.CLIENT_METRICS, res.type());
|
||||
assertEquals(3, alterConfigOps.size());
|
||||
|
||||
List<AlterConfigOp> expectedConfigOps = Arrays.asList(
|
||||
List<AlterConfigOp> expectedConfigOps = List.of(
|
||||
new AlterConfigOp(new ConfigEntry("interval.ms", ""), AlterConfigOp.OpType.DELETE),
|
||||
new AlterConfigOp(new ConfigEntry("match", "client_software_name=kafka.python,client_software_version=1\\.2\\..*"), AlterConfigOp.OpType.SET),
|
||||
new AlterConfigOp(new ConfigEntry("metrics", "org.apache.kafka.consumer."), AlterConfigOp.OpType.SET)
|
||||
|
@ -1279,7 +1270,7 @@ public class ConfigCommandTest {
|
|||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertTrue(options.includeSynonyms());
|
||||
|
@ -1287,12 +1278,12 @@ public class ConfigCommandTest {
|
|||
ConfigResource resource = resources.iterator().next();
|
||||
assertEquals(ConfigResource.Type.CLIENT_METRICS, resource.type());
|
||||
assertEquals(resourceCustom.name(), resource.name());
|
||||
future.complete(Collections.singletonMap(resourceCustom, new Config(Collections.singletonList(configEntry))));
|
||||
future.complete(Map.of(resourceCustom, new Config(List.of(configEntry))));
|
||||
return describeResult;
|
||||
}
|
||||
};
|
||||
mockAdminClient.incrementalAlterConfigs(Collections.singletonMap(resourceCustom,
|
||||
Collections.singletonList(new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET))), new AlterConfigsOptions());
|
||||
mockAdminClient.incrementalAlterConfigs(Map.of(resourceCustom,
|
||||
List.of(new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET))), new AlterConfigsOptions());
|
||||
ConfigCommand.describeConfig(mockAdminClient, describeOpts);
|
||||
verify(describeResult).all();
|
||||
}
|
||||
|
@ -1311,25 +1302,25 @@ public class ConfigCommandTest {
|
|||
@Test
|
||||
public void shouldAlterGroupConfig() {
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
verifyAlterGroupConfig(node, "group", Arrays.asList("--entity-type", "groups", "--entity-name", "group"));
|
||||
verifyAlterGroupConfig(node, "group", List.of("--entity-type", "groups", "--entity-name", "group"));
|
||||
|
||||
// Test for the --group alias
|
||||
verifyAlterGroupConfig(node, "groupUsingAlias", Arrays.asList("--group", "groupUsingAlias"));
|
||||
verifyAlterGroupConfig(node, "groupUsingAlias", List.of("--group", "groupUsingAlias"));
|
||||
}
|
||||
|
||||
private void verifyAlterGroupConfig(Node node, String resourceName, List<String> resourceOpts) {
|
||||
List<String> optsList = concat(Arrays.asList("--bootstrap-server", "localhost:9092",
|
||||
List<String> optsList = concat(List.of("--bootstrap-server", "localhost:9092",
|
||||
"--alter",
|
||||
"--delete-config", "consumer.session.timeout.ms",
|
||||
"--add-config", "consumer.heartbeat.interval.ms=6000"), resourceOpts);
|
||||
ConfigCommand.ConfigCommandOptions alterOpts = new ConfigCommand.ConfigCommandOptions(toArray(optsList));
|
||||
|
||||
ConfigResource resource = new ConfigResource(ConfigResource.Type.GROUP, resourceName);
|
||||
List<ConfigEntry> configEntries = Collections.singletonList(new ConfigEntry("consumer.session.timeout.ms", "45000",
|
||||
ConfigEntry.ConfigSource.DYNAMIC_GROUP_CONFIG, false, false, Collections.emptyList(),
|
||||
List<ConfigEntry> configEntries = List.of(new ConfigEntry("consumer.session.timeout.ms", "45000",
|
||||
ConfigEntry.ConfigSource.DYNAMIC_GROUP_CONFIG, false, false, List.of(),
|
||||
ConfigEntry.ConfigType.UNKNOWN, null));
|
||||
KafkaFutureImpl<Map<ConfigResource, Config>> future = new KafkaFutureImpl<>();
|
||||
future.complete(Collections.singletonMap(resource, new Config(configEntries)));
|
||||
future.complete(Map.of(resource, new Config(configEntries)));
|
||||
DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class);
|
||||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
|
@ -1338,7 +1329,7 @@ public class ConfigCommandTest {
|
|||
AlterConfigsResult alterResult = mock(AlterConfigsResult.class);
|
||||
when(alterResult.all()).thenReturn(alterFuture);
|
||||
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertFalse(options.includeSynonyms(), "Config synonyms requested unnecessarily");
|
||||
|
@ -1358,7 +1349,7 @@ public class ConfigCommandTest {
|
|||
assertEquals(ConfigResource.Type.GROUP, res.type());
|
||||
assertEquals(2, alterConfigOps.size());
|
||||
|
||||
List<AlterConfigOp> expectedConfigOps = Arrays.asList(
|
||||
List<AlterConfigOp> expectedConfigOps = List.of(
|
||||
new AlterConfigOp(new ConfigEntry("consumer.session.timeout.ms", ""), AlterConfigOp.OpType.DELETE),
|
||||
new AlterConfigOp(new ConfigEntry("consumer.heartbeat.interval.ms", "6000"), AlterConfigOp.OpType.SET)
|
||||
);
|
||||
|
@ -1392,7 +1383,7 @@ public class ConfigCommandTest {
|
|||
when(describeResult.all()).thenReturn(future);
|
||||
|
||||
Node node = new Node(1, "localhost", 9092);
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) {
|
||||
MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) {
|
||||
@Override
|
||||
public synchronized DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
|
||||
assertTrue(options.includeSynonyms());
|
||||
|
@ -1400,12 +1391,12 @@ public class ConfigCommandTest {
|
|||
ConfigResource resource = resources.iterator().next();
|
||||
assertEquals(ConfigResource.Type.GROUP, resource.type());
|
||||
assertEquals(resourceCustom.name(), resource.name());
|
||||
future.complete(Collections.singletonMap(resourceCustom, new Config(Collections.singletonList(configEntry))));
|
||||
future.complete(Map.of(resourceCustom, new Config(List.of(configEntry))));
|
||||
return describeResult;
|
||||
}
|
||||
};
|
||||
mockAdminClient.incrementalAlterConfigs(Collections.singletonMap(resourceCustom,
|
||||
Collections.singletonList(new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET))), new AlterConfigsOptions());
|
||||
mockAdminClient.incrementalAlterConfigs(Map.of(resourceCustom,
|
||||
List.of(new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET))), new AlterConfigsOptions());
|
||||
ConfigCommand.describeConfig(mockAdminClient, describeOpts);
|
||||
verify(describeResult).all();
|
||||
}
|
||||
|
@ -1432,7 +1423,7 @@ public class ConfigCommandTest {
|
|||
|
||||
@SafeVarargs
|
||||
public static List<String> concat(List<String>... lists) {
|
||||
return Stream.of(lists).flatMap(List::stream).collect(Collectors.toList());
|
||||
return Stream.of(lists).flatMap(List::stream).toList();
|
||||
}
|
||||
|
||||
@SafeVarargs
|
||||
|
@ -1448,7 +1439,7 @@ public class ConfigCommandTest {
|
|||
|
||||
static class DummyAdminClient extends MockAdminClient {
|
||||
public DummyAdminClient(Node node) {
|
||||
super(Collections.singletonList(node), node);
|
||||
super(List.of(node), node);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -568,7 +568,7 @@ public class ConnectPluginPathTest {
|
|||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
return new WorkerConfig(path, Arrays.asList(pluginPathElements));
|
||||
return new WorkerConfig(path, List.of(pluginPathElements));
|
||||
}
|
||||
|
||||
private static class CommandResult {
|
||||
|
|
|
@ -35,9 +35,9 @@ import java.io.OutputStream;
|
|||
import java.nio.file.Files;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
@ -93,7 +93,7 @@ public class ConsoleProducerTest {
|
|||
ConsoleProducerOptions opts = new ConsoleProducerOptions(BOOTSTRAP_SERVER_VALID_ARGS);
|
||||
ProducerConfig producerConfig = new ProducerConfig(opts.producerProps());
|
||||
|
||||
assertEquals(asList("localhost:1003", "localhost:1004"), producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
|
||||
assertEquals(List.of("localhost:1003", "localhost:1004"), producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -147,7 +147,7 @@ public class ConsoleProducerTest {
|
|||
ConsoleProducerOptions opts = new ConsoleProducerOptions(BOOTSTRAP_SERVER_OVERRIDE);
|
||||
ProducerConfig producerConfig = new ProducerConfig(opts.producerProps());
|
||||
|
||||
assertEquals(Collections.singletonList("localhost:1002"), producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
|
||||
assertEquals(List.of("localhost:1002"), producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -24,8 +24,6 @@ import org.apache.kafka.common.utils.Exit;
|
|||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
|
@ -55,12 +53,12 @@ public class DelegationTokenCommandTest {
|
|||
|
||||
tokens = DelegationTokenCommand.describeToken(adminClient, getDescribeOpts(""));
|
||||
assertEquals(2, tokens.size());
|
||||
assertEquals(Arrays.asList(token1, token2), tokens);
|
||||
assertEquals(List.of(token1, token2), tokens);
|
||||
|
||||
//get tokens for renewer2
|
||||
tokens = DelegationTokenCommand.describeToken(adminClient, getDescribeOpts(renewer2));
|
||||
assertEquals(1, tokens.size());
|
||||
assertEquals(Collections.singletonList(token2), tokens);
|
||||
assertEquals(List.of(token2), tokens);
|
||||
|
||||
//test renewing tokens
|
||||
Long expiryTimestamp = DelegationTokenCommand.renewToken(adminClient, getRenewOpts(token1.hmacAsBase64String()));
|
||||
|
|
|
@ -33,12 +33,11 @@ import org.junit.jupiter.api.Test;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
@ -62,7 +61,7 @@ public class DeleteRecordsCommandTest {
|
|||
"Offset json file contains duplicate topic partitions: t-0"
|
||||
);
|
||||
|
||||
admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get();
|
||||
admin.createTopics(Set.of(new NewTopic("t", 1, (short) 1))).all().get();
|
||||
|
||||
Properties props = new Properties();
|
||||
|
||||
|
@ -159,8 +158,8 @@ public class DeleteRecordsCommandTest {
|
|||
);
|
||||
|
||||
assertEquals(2, res.size());
|
||||
assertEquals(Arrays.asList(0L, 2L, 0L), res.get(new TopicPartition("t", 0)));
|
||||
assertEquals(Collections.singletonList(1L), res.get(new TopicPartition("t", 1)));
|
||||
assertEquals(List.of(0L, 2L, 0L), res.get(new TopicPartition("t", 0)));
|
||||
assertEquals(List.of(1L), res.get(new TopicPartition("t", 1)));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -28,15 +28,11 @@ import net.sourceforge.argparse4j.inf.Namespace;
|
|||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static java.lang.String.format;
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType.SAFE_DOWNGRADE;
|
||||
import static org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
|
@ -46,7 +42,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
|||
|
||||
public class FeatureCommandTest {
|
||||
|
||||
private final List<Feature> testingFeatures = Arrays.stream(Feature.FEATURES).collect(Collectors.toList());
|
||||
private final List<Feature> testingFeatures = Arrays.stream(Feature.FEATURES).toList();
|
||||
|
||||
@ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_3_IV3)
|
||||
public void testDescribeWithKRaft(ClusterInstance cluster) {
|
||||
|
@ -54,7 +50,7 @@ public class FeatureCommandTest {
|
|||
assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe"))
|
||||
);
|
||||
|
||||
List<String> features = Arrays.stream(commandOutput.split("\n")).sorted().collect(Collectors.toList());
|
||||
List<String> features = Arrays.stream(commandOutput.split("\n")).sorted().toList();
|
||||
|
||||
// Change expected message to reflect latest MetadataVersion (SupportedMaxVersion increases when adding a new version)
|
||||
assertEquals("Feature: eligible.leader.replicas.version\tSupportedMinVersion: 0\t" +
|
||||
|
@ -80,7 +76,7 @@ public class FeatureCommandTest {
|
|||
assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-controller", cluster.bootstrapControllers(), "describe"))
|
||||
);
|
||||
|
||||
List<String> features = Arrays.stream(commandOutput.split("\n")).sorted().collect(Collectors.toList());
|
||||
List<String> features = Arrays.stream(commandOutput.split("\n")).sorted().toList();
|
||||
|
||||
// Change expected message to reflect latest MetadataVersion (SupportedMaxVersion increases when adding a new version)
|
||||
assertEquals("Feature: eligible.leader.replicas.version\tSupportedMinVersion: 0\t" +
|
||||
|
@ -223,10 +219,10 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testDowngradeType() {
|
||||
assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType(
|
||||
new Namespace(singletonMap("unsafe", Boolean.FALSE))));
|
||||
new Namespace(Map.of("unsafe", Boolean.FALSE))));
|
||||
assertEquals(UNSAFE_DOWNGRADE, FeatureCommand.downgradeType(
|
||||
new Namespace(singletonMap("unsafe", Boolean.TRUE))));
|
||||
assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType(new Namespace(emptyMap())));
|
||||
new Namespace(Map.of("unsafe", Boolean.TRUE))));
|
||||
assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType(new Namespace(Map.of())));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -275,7 +271,7 @@ public class FeatureCommandTest {
|
|||
public void testHandleUpgradeToUnsupportedMetadataVersion() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("metadata", "3.3-IV1");
|
||||
namespace.put("feature", Collections.singletonList("foo.bar=6"));
|
||||
namespace.put("feature", List.of("foo.bar=6"));
|
||||
namespace.put("dry_run", false);
|
||||
Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient()));
|
||||
assertTrue(t.getMessage().contains("Unknown metadata.version 3.3-IV1"));
|
||||
|
@ -285,7 +281,7 @@ public class FeatureCommandTest {
|
|||
public void testHandleUpgradeToLowerVersion() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("metadata", "3.3-IV3");
|
||||
namespace.put("feature", Collections.singletonList("foo.bar=6"));
|
||||
namespace.put("feature", List.of("foo.bar=6"));
|
||||
namespace.put("dry_run", false);
|
||||
String upgradeOutput = ToolsTestUtils.captureStandardOut(() -> {
|
||||
Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient()));
|
||||
|
@ -300,7 +296,7 @@ public class FeatureCommandTest {
|
|||
public void testHandleUpgradeToLowerVersionDryRun() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("metadata", "3.3-IV3");
|
||||
namespace.put("feature", Collections.singletonList("foo.bar=6"));
|
||||
namespace.put("feature", List.of("foo.bar=6"));
|
||||
namespace.put("dry_run", true);
|
||||
String upgradeOutput = ToolsTestUtils.captureStandardOut(() -> {
|
||||
Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient()));
|
||||
|
@ -315,7 +311,7 @@ public class FeatureCommandTest {
|
|||
public void testHandleDowngrade() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("metadata", "3.7-IV0");
|
||||
namespace.put("feature", Collections.singletonList("foo.bar=1"));
|
||||
namespace.put("feature", List.of("foo.bar=1"));
|
||||
namespace.put("dry_run", false);
|
||||
String downgradeOutput = ToolsTestUtils.captureStandardOut(() -> {
|
||||
Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDowngrade(new Namespace(namespace), buildAdminClient()));
|
||||
|
@ -330,7 +326,7 @@ public class FeatureCommandTest {
|
|||
public void testHandleDowngradeDryRun() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("metadata", "3.7-IV0");
|
||||
namespace.put("feature", Collections.singletonList("foo.bar=1"));
|
||||
namespace.put("feature", List.of("foo.bar=1"));
|
||||
namespace.put("dry_run", true);
|
||||
String downgradeOutput = ToolsTestUtils.captureStandardOut(() -> {
|
||||
Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDowngrade(new Namespace(namespace), buildAdminClient()));
|
||||
|
@ -344,7 +340,7 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testHandleDisable() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("feature", Arrays.asList("foo.bar", "metadata.version", "quux"));
|
||||
namespace.put("feature", List.of("foo.bar", "metadata.version", "quux"));
|
||||
namespace.put("dry_run", false);
|
||||
String disableOutput = ToolsTestUtils.captureStandardOut(() -> {
|
||||
Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDisable(new Namespace(namespace), buildAdminClient()));
|
||||
|
@ -358,7 +354,7 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testHandleDisableDryRun() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("feature", Arrays.asList("foo.bar", "metadata.version", "quux"));
|
||||
namespace.put("feature", List.of("foo.bar", "metadata.version", "quux"));
|
||||
namespace.put("dry_run", true);
|
||||
String disableOutput = ToolsTestUtils.captureStandardOut(() -> {
|
||||
Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDisable(new Namespace(namespace), buildAdminClient()));
|
||||
|
@ -383,7 +379,7 @@ public class FeatureCommandTest {
|
|||
public void testIncompatibleUpgradeFlags() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("release_version", "3.3-IV3");
|
||||
namespace.put("feature", Arrays.asList("foo.bar", "metadata.version", "quux"));
|
||||
namespace.put("feature", List.of("foo.bar", "metadata.version", "quux"));
|
||||
ToolsTestUtils.captureStandardOut(() -> {
|
||||
Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient()));
|
||||
assertTrue(t.getMessage().contains("Can not specify `release-version` with other feature flags."));
|
||||
|
@ -474,7 +470,7 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testHandleFeatureDependenciesForFeatureWithDependencies() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("feature", Collections.singletonList("test.feature.version=2"));
|
||||
namespace.put("feature", List.of("test.feature.version=2"));
|
||||
|
||||
String output = ToolsTestUtils.captureStandardOut(() -> {
|
||||
try {
|
||||
|
@ -496,7 +492,7 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testHandleFeatureDependenciesForFeatureWithNoDependencies() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("feature", Collections.singletonList("metadata.version=17"));
|
||||
namespace.put("feature", List.of("metadata.version=17"));
|
||||
|
||||
String output = ToolsTestUtils.captureStandardOut(() -> {
|
||||
try {
|
||||
|
@ -512,7 +508,7 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testHandleFeatureDependenciesForUnknownFeature() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("feature", Collections.singletonList("unknown.feature=1"));
|
||||
namespace.put("feature", List.of("unknown.feature=1"));
|
||||
|
||||
Exception exception = assertThrows(
|
||||
TerseException.class,
|
||||
|
@ -525,7 +521,7 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testHandleFeatureDependenciesForFeatureWithUnknownFeatureVersion() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("feature", Collections.singletonList("transaction.version=1000"));
|
||||
namespace.put("feature", List.of("transaction.version=1000"));
|
||||
|
||||
Exception exception = assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
|
@ -538,7 +534,7 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testHandleFeatureDependenciesForInvalidVersionFormat() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("feature", Collections.singletonList("metadata.version=invalid"));
|
||||
namespace.put("feature", List.of("metadata.version=invalid"));
|
||||
|
||||
RuntimeException exception = assertThrows(
|
||||
RuntimeException.class,
|
||||
|
@ -554,7 +550,7 @@ public class FeatureCommandTest {
|
|||
@Test
|
||||
public void testHandleFeatureDependenciesForMultipleFeatures() {
|
||||
Map<String, Object> namespace = new HashMap<>();
|
||||
namespace.put("feature", Arrays.asList(
|
||||
namespace.put("feature", List.of(
|
||||
"transaction.version=2",
|
||||
"group.version=1",
|
||||
"test.feature.version=2"
|
||||
|
|
|
@ -46,7 +46,6 @@ import org.apache.kafka.test.TestUtils;
|
|||
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;
|
||||
|
@ -85,7 +84,7 @@ public class GetOffsetShellTest {
|
|||
}
|
||||
|
||||
private void setUp() {
|
||||
setupTopics(this::getTopicName, Collections.emptyMap());
|
||||
setupTopics(this::getTopicName, Map.of());
|
||||
sendProducerRecords(this::getTopicName);
|
||||
}
|
||||
|
||||
|
@ -139,7 +138,7 @@ public class GetOffsetShellTest {
|
|||
serverProperties.put(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, "100");
|
||||
serverProperties.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP, "EXTERNAL");
|
||||
|
||||
return Collections.singletonList(
|
||||
return List.of(
|
||||
ClusterConfig.defaultBuilder()
|
||||
.setTypes(Stream.of(KRAFT, CO_KRAFT).collect(Collectors.toSet()))
|
||||
.setServerProperties(serverProperties)
|
||||
|
@ -238,7 +237,7 @@ public class GetOffsetShellTest {
|
|||
setUp();
|
||||
|
||||
List<Row> offsets = executeAndParse("--partitions", "0,1");
|
||||
assertEquals(expectedTestTopicOffsets().stream().filter(r -> r.partition <= 1).collect(Collectors.toList()), offsets);
|
||||
assertEquals(expectedTestTopicOffsets().stream().filter(r -> r.partition <= 1).toList(), offsets);
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
|
@ -247,7 +246,7 @@ public class GetOffsetShellTest {
|
|||
|
||||
List<Row> offsets = executeAndParse("--topic", "topic.*", "--partitions", "0,1");
|
||||
|
||||
assertEquals(expectedTestTopicOffsets().stream().filter(r -> r.partition <= 1).collect(Collectors.toList()), offsets);
|
||||
assertEquals(expectedTestTopicOffsets().stream().filter(r -> r.partition <= 1).toList(), offsets);
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
|
@ -257,7 +256,7 @@ public class GetOffsetShellTest {
|
|||
createConsumerAndPoll();
|
||||
|
||||
List<Row> offsets = executeAndParse("--topic-partitions", "topic1:0,topic2:1,topic(3|4):2,__.*:3");
|
||||
List<Row> expected = Arrays.asList(
|
||||
List<Row> expected = List.of(
|
||||
new Row("__consumer_offsets", 3, 0L),
|
||||
new Row("topic1", 0, 1L),
|
||||
new Row("topic2", 1, 2L),
|
||||
|
@ -274,7 +273,7 @@ public class GetOffsetShellTest {
|
|||
|
||||
for (String time : new String[] {"-1", "latest"}) {
|
||||
List<Row> offsets = executeAndParse("--topic-partitions", "topic.*:0", "--time", time);
|
||||
List<Row> expected = Arrays.asList(
|
||||
List<Row> expected = List.of(
|
||||
new Row("topic1", 0, 1L),
|
||||
new Row("topic2", 0, 2L),
|
||||
new Row("topic3", 0, 3L),
|
||||
|
@ -291,7 +290,7 @@ public class GetOffsetShellTest {
|
|||
|
||||
for (String time : new String[] {"-2", "earliest"}) {
|
||||
List<Row> offsets = executeAndParse("--topic-partitions", "topic.*:0", "--time", time);
|
||||
List<Row> expected = Arrays.asList(
|
||||
List<Row> expected = List.of(
|
||||
new Row("topic1", 0, 0L),
|
||||
new Row("topic2", 0, 0L),
|
||||
new Row("topic3", 0, 0L),
|
||||
|
@ -324,7 +323,7 @@ public class GetOffsetShellTest {
|
|||
// test topics disable remote log storage
|
||||
// as remote log disabled, broker return the same result as earliest offset
|
||||
TestUtils.waitForCondition(() ->
|
||||
Arrays.asList(
|
||||
List.of(
|
||||
new Row("topic1", 0, 0L),
|
||||
new Row("topic2", 0, 0L),
|
||||
new Row("topic3", 0, 0L),
|
||||
|
@ -334,7 +333,7 @@ public class GetOffsetShellTest {
|
|||
|
||||
// test topics enable remote log storage
|
||||
TestUtils.waitForCondition(() ->
|
||||
Arrays.asList(
|
||||
List.of(
|
||||
new Row("topicRLS1", 0, 0L),
|
||||
new Row("topicRLS2", 0, 1L),
|
||||
new Row("topicRLS3", 0, 2L),
|
||||
|
@ -353,13 +352,13 @@ public class GetOffsetShellTest {
|
|||
// test topics disable remote log storage
|
||||
// as remote log not enabled, broker return unknown offset for each topic partition and these
|
||||
// unknown offsets are ignored by GetOffsetShell hence we have empty result here.
|
||||
assertEquals(Collections.emptyList(),
|
||||
assertEquals(List.of(),
|
||||
executeAndParse("--topic-partitions", "topic\\d+:0", "--time", time));
|
||||
|
||||
// test topics enable remote log storage
|
||||
// topicRLS1 has no result because there's no log segments being uploaded to the remote storage
|
||||
TestUtils.waitForCondition(() ->
|
||||
Arrays.asList(
|
||||
List.of(
|
||||
new Row("topicRLS2", 0, 0L),
|
||||
new Row("topicRLS3", 0, 1L),
|
||||
new Row("topicRLS4", 0, 2L))
|
||||
|
@ -375,7 +374,7 @@ public class GetOffsetShellTest {
|
|||
String time = String.valueOf(System.currentTimeMillis() / 2);
|
||||
|
||||
List<Row> offsets = executeAndParse("--topic-partitions", "topic.*:0", "--time", time);
|
||||
List<Row> expected = Arrays.asList(
|
||||
List<Row> expected = List.of(
|
||||
new Row("topic1", 0, 0L),
|
||||
new Row("topic2", 0, 0L),
|
||||
new Row("topic3", 0, 0L),
|
||||
|
@ -401,7 +400,7 @@ public class GetOffsetShellTest {
|
|||
setUp();
|
||||
|
||||
List<Row> offsets = executeAndParse("--topic-partitions", "topic1:0,topic2:1,topic(3|4):2,__.*:3", "--exclude-internal-topics");
|
||||
List<Row> expected = Arrays.asList(
|
||||
List<Row> expected = List.of(
|
||||
new Row("topic1", 0, 1L),
|
||||
new Row("topic2", 1, 2L),
|
||||
new Row("topic3", 2, 3L),
|
||||
|
@ -419,7 +418,7 @@ public class GetOffsetShellTest {
|
|||
|
||||
List<Row> offsets = executeAndParse("--topic-partitions", "__.*:0");
|
||||
|
||||
assertEquals(Arrays.asList(new Row("__consumer_offsets", 0, 0L)), offsets);
|
||||
assertEquals(List.of(new Row("__consumer_offsets", 0, 0L)), offsets);
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
|
@ -487,9 +486,9 @@ public class GetOffsetShellTest {
|
|||
private List<Row> expectedOffsetsWithInternal() {
|
||||
List<Row> consOffsets = IntStream.range(0, 4)
|
||||
.mapToObj(i -> new Row("__consumer_offsets", i, 0L))
|
||||
.collect(Collectors.toList());
|
||||
.toList();
|
||||
|
||||
return Stream.concat(consOffsets.stream(), expectedTestTopicOffsets().stream()).collect(Collectors.toList());
|
||||
return Stream.concat(consOffsets.stream(), expectedTestTopicOffsets().stream()).toList();
|
||||
}
|
||||
|
||||
private List<Row> expectedTestTopicOffsets() {
|
||||
|
@ -505,7 +504,7 @@ public class GetOffsetShellTest {
|
|||
private List<Row> expectedOffsetsForTopic(int i) {
|
||||
String name = getTopicName(i);
|
||||
|
||||
return IntStream.range(0, i).mapToObj(p -> new Row(name, p, (long) i)).collect(Collectors.toList());
|
||||
return IntStream.range(0, i).mapToObj(p -> new Row(name, p, (long) i)).toList();
|
||||
}
|
||||
|
||||
private List<Row> executeAndParse(String... args) {
|
||||
|
@ -515,11 +514,11 @@ public class GetOffsetShellTest {
|
|||
.map(i -> i.split(":"))
|
||||
.filter(i -> i.length >= 2)
|
||||
.map(line -> new Row(line[0], Integer.parseInt(line[1]), (line.length == 2 || line[2].isEmpty()) ? null : Long.parseLong(line[2])))
|
||||
.collect(Collectors.toList());
|
||||
.toList();
|
||||
}
|
||||
|
||||
private String[] addBootstrapServer(String... args) {
|
||||
ArrayList<String> newArgs = new ArrayList<>(Arrays.asList(args));
|
||||
ArrayList<String> newArgs = new ArrayList<>(List.of(args));
|
||||
newArgs.add("--bootstrap-server");
|
||||
newArgs.add(cluster.bootstrapServers());
|
||||
|
||||
|
|
|
@ -139,9 +139,9 @@ public class JmxToolTest {
|
|||
"--one-time"
|
||||
};
|
||||
String out = executeAndGetOut(args);
|
||||
Arrays.stream(out.split("\\r?\\n")).forEach(line -> {
|
||||
assertTrue(line.matches("([a-zA-Z0-9=:,.]+),\"([ -~]+)\""), line);
|
||||
});
|
||||
Arrays.stream(out.split("\\r?\\n")).forEach(line ->
|
||||
assertTrue(line.matches("([a-zA-Z0-9=:,.]+),\"([ -~]+)\""), line)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -153,9 +153,9 @@ public class JmxToolTest {
|
|||
"--one-time"
|
||||
};
|
||||
String out = executeAndGetOut(args);
|
||||
Arrays.stream(out.split("\\r?\\n")).forEach(line -> {
|
||||
assertTrue(line.matches("([a-zA-Z0-9=:,.]+)\\t([ -~]+)"), line);
|
||||
});
|
||||
Arrays.stream(out.split("\\r?\\n")).forEach(line ->
|
||||
assertTrue(line.matches("([a-zA-Z0-9=:,.]+)\\t([ -~]+)"), line)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -37,7 +37,6 @@ import java.nio.file.Files;
|
|||
import java.nio.file.Path;
|
||||
import java.time.Duration;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
|
@ -49,8 +48,6 @@ import java.util.concurrent.ExecutionException;
|
|||
|
||||
import scala.jdk.javaapi.CollectionConverters;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
@ -76,20 +73,20 @@ public class LeaderElectionCommandTest {
|
|||
public void testAllTopicPartition() throws InterruptedException, ExecutionException {
|
||||
String topic = "unclean-topic";
|
||||
int partition = 0;
|
||||
List<Integer> assignment = asList(broker2, broker3);
|
||||
List<Integer> assignment = List.of(broker2, broker3);
|
||||
|
||||
cluster.waitForReadyBrokers();
|
||||
try (Admin client = cluster.admin()) {
|
||||
|
||||
createTopic(client, topic, Collections.singletonMap(partition, assignment));
|
||||
createTopic(client, topic, Map.of(partition, assignment));
|
||||
|
||||
TopicPartition topicPartition = new TopicPartition(topic, partition);
|
||||
|
||||
TestUtils.assertLeader(client, topicPartition, broker2);
|
||||
cluster.shutdownBroker(broker3);
|
||||
TestUtils.waitForBrokersOutOfIsr(client,
|
||||
CollectionConverters.asScala(singletonList(topicPartition)).toSet(),
|
||||
CollectionConverters.asScala(singletonList(broker3)).toSet()
|
||||
CollectionConverters.asScala(List.of(topicPartition)).toSet(),
|
||||
CollectionConverters.asScala(List.of(broker3)).toSet()
|
||||
);
|
||||
cluster.shutdownBroker(broker2);
|
||||
TestUtils.assertNoLeader(client, topicPartition);
|
||||
|
@ -133,11 +130,11 @@ public class LeaderElectionCommandTest {
|
|||
public void testTopicPartition() throws InterruptedException, ExecutionException {
|
||||
String topic = "unclean-topic";
|
||||
int partition = 0;
|
||||
List<Integer> assignment = asList(broker2, broker3);
|
||||
List<Integer> assignment = List.of(broker2, broker3);
|
||||
|
||||
cluster.waitForReadyBrokers();
|
||||
try (Admin client = cluster.admin()) {
|
||||
createTopic(client, topic, Collections.singletonMap(partition, assignment));
|
||||
createTopic(client, topic, Map.of(partition, assignment));
|
||||
|
||||
TopicPartition topicPartition = new TopicPartition(topic, partition);
|
||||
|
||||
|
@ -145,8 +142,8 @@ public class LeaderElectionCommandTest {
|
|||
|
||||
cluster.shutdownBroker(broker3);
|
||||
TestUtils.waitForBrokersOutOfIsr(client,
|
||||
CollectionConverters.asScala(singletonList(topicPartition)).toSet(),
|
||||
CollectionConverters.asScala(singletonList(broker3)).toSet()
|
||||
CollectionConverters.asScala(List.of(topicPartition)).toSet(),
|
||||
CollectionConverters.asScala(List.of(broker3)).toSet()
|
||||
);
|
||||
cluster.shutdownBroker(broker2);
|
||||
TestUtils.assertNoLeader(client, topicPartition);
|
||||
|
@ -168,7 +165,7 @@ public class LeaderElectionCommandTest {
|
|||
public void testPathToJsonFile() throws Exception {
|
||||
String topic = "unclean-topic";
|
||||
int partition = 0;
|
||||
List<Integer> assignment = asList(broker2, broker3);
|
||||
List<Integer> assignment = List.of(broker2, broker3);
|
||||
|
||||
cluster.waitForReadyBrokers();
|
||||
Map<Integer, List<Integer>> partitionAssignment = new HashMap<>();
|
||||
|
@ -183,15 +180,15 @@ public class LeaderElectionCommandTest {
|
|||
|
||||
cluster.shutdownBroker(broker3);
|
||||
TestUtils.waitForBrokersOutOfIsr(client,
|
||||
CollectionConverters.asScala(singletonList(topicPartition)).toSet(),
|
||||
CollectionConverters.asScala(singletonList(broker3)).toSet()
|
||||
CollectionConverters.asScala(List.of(topicPartition)).toSet(),
|
||||
CollectionConverters.asScala(List.of(broker3)).toSet()
|
||||
);
|
||||
cluster.shutdownBroker(broker2);
|
||||
TestUtils.assertNoLeader(client, topicPartition);
|
||||
cluster.startBroker(broker3);
|
||||
TestUtils.waitForOnlineBroker(client, broker3);
|
||||
|
||||
Path topicPartitionPath = tempTopicPartitionFile(singletonList(topicPartition));
|
||||
Path topicPartitionPath = tempTopicPartitionFile(List.of(topicPartition));
|
||||
|
||||
assertEquals(0, LeaderElectionCommand.mainNoExit(
|
||||
"--bootstrap-server", cluster.bootstrapServers(),
|
||||
|
@ -207,7 +204,7 @@ public class LeaderElectionCommandTest {
|
|||
public void testPreferredReplicaElection() throws InterruptedException, ExecutionException {
|
||||
String topic = "preferred-topic";
|
||||
int partition = 0;
|
||||
List<Integer> assignment = asList(broker2, broker3);
|
||||
List<Integer> assignment = List.of(broker2, broker3);
|
||||
|
||||
cluster.waitForReadyBrokers();
|
||||
try (Admin client = cluster.admin()) {
|
||||
|
@ -224,7 +221,7 @@ public class LeaderElectionCommandTest {
|
|||
TestUtils.assertLeader(client, topicPartition, broker3);
|
||||
cluster.startBroker(broker2);
|
||||
TestUtils.waitForBrokersInIsr(client, topicPartition,
|
||||
CollectionConverters.asScala(singletonList(broker2)).toSet()
|
||||
CollectionConverters.asScala(List.of(broker2)).toSet()
|
||||
);
|
||||
|
||||
assertEquals(0, LeaderElectionCommand.mainNoExit(
|
||||
|
@ -254,8 +251,8 @@ public class LeaderElectionCommandTest {
|
|||
String topic = "non-preferred-topic";
|
||||
int partition0 = 0;
|
||||
int partition1 = 1;
|
||||
List<Integer> assignment0 = asList(broker2, broker3);
|
||||
List<Integer> assignment1 = asList(broker3, broker2);
|
||||
List<Integer> assignment0 = List.of(broker2, broker3);
|
||||
List<Integer> assignment1 = List.of(broker3, broker2);
|
||||
|
||||
cluster.waitForReadyBrokers();
|
||||
TopicPartition topicPartition0;
|
||||
|
@ -277,14 +274,14 @@ public class LeaderElectionCommandTest {
|
|||
TestUtils.assertLeader(client, topicPartition0, broker3);
|
||||
cluster.startBroker(broker2);
|
||||
TestUtils.waitForBrokersInIsr(client, topicPartition0,
|
||||
CollectionConverters.asScala(singletonList(broker2)).toSet()
|
||||
CollectionConverters.asScala(List.of(broker2)).toSet()
|
||||
);
|
||||
TestUtils.waitForBrokersInIsr(client, topicPartition1,
|
||||
CollectionConverters.asScala(singletonList(broker2)).toSet()
|
||||
CollectionConverters.asScala(List.of(broker2)).toSet()
|
||||
);
|
||||
}
|
||||
|
||||
Path topicPartitionPath = tempTopicPartitionFile(asList(topicPartition0, topicPartition1));
|
||||
Path topicPartitionPath = tempTopicPartitionFile(List.of(topicPartition0, topicPartition1));
|
||||
String output = ToolsTestUtils.captureStandardOut(() ->
|
||||
LeaderElectionCommand.mainNoExit(
|
||||
"--bootstrap-server", cluster.bootstrapServers(),
|
||||
|
@ -308,7 +305,7 @@ public class LeaderElectionCommandTest {
|
|||
|
||||
private void createTopic(Admin admin, String topic, Map<Integer, List<Integer>> replicaAssignment) throws ExecutionException, InterruptedException {
|
||||
NewTopic newTopic = new NewTopic(topic, replicaAssignment);
|
||||
List<NewTopic> newTopics = singletonList(newTopic);
|
||||
List<NewTopic> newTopics = List.of(newTopic);
|
||||
CreateTopicsResult createTopicResult = admin.createTopics(newTopics);
|
||||
createTopicResult.all().get();
|
||||
}
|
||||
|
|
|
@ -25,15 +25,12 @@ import org.apache.kafka.tools.api.RecordReader;
|
|||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.apache.kafka.common.utils.Utils.propsToStringMap;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -57,7 +54,7 @@ public class LineMessageReaderTest {
|
|||
ProducerRecord<String, String> expected = record(
|
||||
"key0",
|
||||
"value0",
|
||||
asList(
|
||||
List.of(
|
||||
new RecordHeader("headerKey0", "headerValue0".getBytes(UTF_8)),
|
||||
new RecordHeader("headerKey1", "headerValue1".getBytes(UTF_8))
|
||||
)
|
||||
|
@ -67,7 +64,7 @@ public class LineMessageReaderTest {
|
|||
|
||||
@Test
|
||||
public void testMinimalValidInputWithHeaderKeyAndValue() {
|
||||
runTest(defaultTestProps(), ":\t\t", record("", "", singletonList(new RecordHeader("", "".getBytes(UTF_8)))));
|
||||
runTest(defaultTestProps(), ":\t\t", record("", "", List.of(new RecordHeader("", "".getBytes(UTF_8)))));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -90,7 +87,7 @@ public class LineMessageReaderTest {
|
|||
"headerKey0.0::::headerValue0.0---headerKey1.0::::\t\tkey\t\tvalue",
|
||||
record("key",
|
||||
"value",
|
||||
asList(
|
||||
List.of(
|
||||
new RecordHeader("headerKey0.0", "headerValue0.0".getBytes(UTF_8)),
|
||||
new RecordHeader("headerKey1.0", "".getBytes(UTF_8))
|
||||
)
|
||||
|
@ -105,7 +102,7 @@ public class LineMessageReaderTest {
|
|||
Properties props = defaultTestProps();
|
||||
props.put("parse.key", "false");
|
||||
|
||||
runTest(props, input, record(null, "value", singletonList(new RecordHeader("headerKey", "headerValue".getBytes(UTF_8)))));
|
||||
runTest(props, input, record(null, "value", List.of(new RecordHeader("headerKey", "headerValue".getBytes(UTF_8)))));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -132,7 +129,7 @@ public class LineMessageReaderTest {
|
|||
ProducerRecord<String, String> record0 = record(
|
||||
"key0",
|
||||
"value0",
|
||||
asList(
|
||||
List.of(
|
||||
new RecordHeader("headerKey0.0", "headerValue0.0".getBytes(UTF_8)),
|
||||
new RecordHeader("headerKey0.1", "headerValue0.1".getBytes(UTF_8))
|
||||
)
|
||||
|
@ -140,7 +137,7 @@ public class LineMessageReaderTest {
|
|||
ProducerRecord<String, String> record1 = record(
|
||||
"key1",
|
||||
"value1",
|
||||
singletonList(new RecordHeader("headerKey1.0", "headerValue1.0".getBytes(UTF_8)))
|
||||
List.of(new RecordHeader("headerKey1.0", "headerValue1.0".getBytes(UTF_8)))
|
||||
);
|
||||
|
||||
runTest(props, input, record0, record1);
|
||||
|
@ -208,12 +205,12 @@ public class LineMessageReaderTest {
|
|||
props.put("ignore.error", "true");
|
||||
|
||||
ProducerRecord<String, String> validRecord = record("key0", "value0",
|
||||
singletonList(new RecordHeader("headerKey0.0", "headerValue0.0".getBytes(UTF_8))));
|
||||
List.of(new RecordHeader("headerKey0.0", "headerValue0.0".getBytes(UTF_8))));
|
||||
|
||||
ProducerRecord<String, String> missingHeaderDelimiter = record(
|
||||
null,
|
||||
"value1",
|
||||
asList(
|
||||
List.of(
|
||||
new RecordHeader("headerKey1.0", "headerValue1.0".getBytes(UTF_8)),
|
||||
new RecordHeader("headerKey1.1", "headerValue1.1[MISSING-HEADER-DELIMITER]key1".getBytes(UTF_8))
|
||||
)
|
||||
|
@ -222,13 +219,13 @@ public class LineMessageReaderTest {
|
|||
ProducerRecord<String, String> missingKeyDelimiter = record(
|
||||
null,
|
||||
"key2[MISSING-KEY-DELIMITER]value2",
|
||||
singletonList(new RecordHeader("headerKey2.0", "headerValue2.0".getBytes(UTF_8)))
|
||||
List.of(new RecordHeader("headerKey2.0", "headerValue2.0".getBytes(UTF_8)))
|
||||
);
|
||||
|
||||
ProducerRecord<String, String> missingKeyHeaderDelimiter = record(
|
||||
null,
|
||||
"headerKey3.0:headerValue3.0[MISSING-HEADER-DELIMITER]key3[MISSING-KEY-DELIMITER]value3",
|
||||
Collections.emptyList()
|
||||
List.of()
|
||||
);
|
||||
|
||||
runTest(props, input, validRecord, missingHeaderDelimiter, missingKeyDelimiter, missingKeyHeaderDelimiter);
|
||||
|
@ -241,7 +238,7 @@ public class LineMessageReaderTest {
|
|||
Properties props = defaultTestProps();
|
||||
props.put("ignore.error", "true");
|
||||
|
||||
ProducerRecord<String, String> expected = record("key0", "value0", singletonList(new RecordHeader("key-val", null)));
|
||||
ProducerRecord<String, String> expected = record("key0", "value0", List.of(new RecordHeader("key-val", null)));
|
||||
|
||||
runTest(props, input, expected);
|
||||
}
|
||||
|
@ -288,11 +285,11 @@ public class LineMessageReaderTest {
|
|||
Properties props = defaultTestProps();
|
||||
props.put("null.marker", "<NULL>");
|
||||
runTest(props, input,
|
||||
record(null, "value", asList(new RecordHeader("h0", "v0".getBytes(UTF_8)), header)),
|
||||
record(null, "value", List.of(new RecordHeader("h0", "v0".getBytes(UTF_8)), header)),
|
||||
record("key", null),
|
||||
record(null, null, asList(new RecordHeader("h0", "".getBytes(UTF_8)), header)),
|
||||
record("key", null, asList(new RecordHeader("h0", null), header)),
|
||||
record("key", null, asList(new RecordHeader("h0", null), new RecordHeader("h1", "<NULL>value".getBytes(UTF_8))))
|
||||
record(null, null, List.of(new RecordHeader("h0", "".getBytes(UTF_8)), header)),
|
||||
record("key", null, List.of(new RecordHeader("h0", null), header)),
|
||||
record("key", null, List.of(new RecordHeader("h0", null), new RecordHeader("h1", "<NULL>value".getBytes(UTF_8))))
|
||||
);
|
||||
|
||||
// If the null marker is not set
|
||||
|
@ -300,16 +297,16 @@ public class LineMessageReaderTest {
|
|||
props.remove("null.marker");
|
||||
lineReader.configure(propsToStringMap(props));
|
||||
Iterator<ProducerRecord<byte[], byte[]>> iter = lineReader.readRecords(new ByteArrayInputStream(input.getBytes()));
|
||||
assertRecordEquals(record("<NULL>", "value", asList(new RecordHeader("h0", "v0".getBytes(UTF_8)), header)), iter.next());
|
||||
assertRecordEquals(record("<NULL>", "value", List.of(new RecordHeader("h0", "v0".getBytes(UTF_8)), header)), iter.next());
|
||||
// line 2 is not valid anymore
|
||||
KafkaException expectedException = assertThrows(KafkaException.class, iter::next);
|
||||
assertEquals(
|
||||
"No header key separator found in pair '<NULL>' on line number 2",
|
||||
expectedException.getMessage()
|
||||
);
|
||||
assertRecordEquals(record("<NULL>", "<NULL>", asList(new RecordHeader("h0", "".getBytes(UTF_8)), header)), iter.next());
|
||||
assertRecordEquals(record("key", "<NULL>", asList(new RecordHeader("h0", "<NULL>".getBytes(UTF_8)), header)), iter.next());
|
||||
assertRecordEquals(record("key", "<NULL>", asList(
|
||||
assertRecordEquals(record("<NULL>", "<NULL>", List.of(new RecordHeader("h0", "".getBytes(UTF_8)), header)), iter.next());
|
||||
assertRecordEquals(record("key", "<NULL>", List.of(new RecordHeader("h0", "<NULL>".getBytes(UTF_8)), header)), iter.next());
|
||||
assertRecordEquals(record("key", "<NULL>", List.of(
|
||||
new RecordHeader("h0", "<NULL>".getBytes(UTF_8)),
|
||||
new RecordHeader("h1", "<NULL>value".getBytes(UTF_8)))), iter.next()
|
||||
);
|
||||
|
@ -332,7 +329,7 @@ public class LineMessageReaderTest {
|
|||
|
||||
// If the null marker is not set
|
||||
props.remove("null.marker");
|
||||
runTest(props, input, record("key", "value", asList(
|
||||
runTest(props, input, record("key", "value", List.of(
|
||||
new RecordHeader("<NULL>", "v0".getBytes(UTF_8)),
|
||||
new RecordHeader("h1", "v1".getBytes(UTF_8))))
|
||||
);
|
||||
|
|
|
@ -31,8 +31,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -53,7 +51,7 @@ public class LogDirsCommandTest {
|
|||
@ClusterTest(brokers = 3)
|
||||
public void testLogDirsWithoutBrokers(ClusterInstance clusterInstance) {
|
||||
createTopic(clusterInstance, TOPIC);
|
||||
try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
String output = assertDoesNotThrow(() -> execute(fromArgsToOptions("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe"), admin));
|
||||
|
||||
// check all brokers are present
|
||||
|
@ -73,7 +71,7 @@ public class LogDirsCommandTest {
|
|||
@ClusterTest(brokers = 3)
|
||||
public void testLogDirsWithBrokers(ClusterInstance clusterInstance) {
|
||||
createTopic(clusterInstance, TOPIC);
|
||||
try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
int brokerId = 0;
|
||||
String output = assertDoesNotThrow(() -> execute(fromArgsToOptions("--bootstrap-server", clusterInstance.bootstrapServers(), "--broker-list", String.valueOf(brokerId), "--describe"), admin));
|
||||
|
||||
|
@ -82,7 +80,7 @@ public class LogDirsCommandTest {
|
|||
clusterInstance.brokerIds().stream().filter(id -> id != brokerId).forEach(id -> assertFalse(output.contains("\"broker\":" + id)));
|
||||
|
||||
// check log dir and topic partition are present
|
||||
Map<Integer, Map<String, LogDirDescription>> logDirs = assertDoesNotThrow(() -> admin.describeLogDirs(Collections.singleton(brokerId)).allDescriptions().get());
|
||||
Map<Integer, Map<String, LogDirDescription>> logDirs = assertDoesNotThrow(() -> admin.describeLogDirs(Set.of(brokerId)).allDescriptions().get());
|
||||
assertEquals(1, logDirs.size());
|
||||
logDirs.forEach((brokerIdValue, logDirInfo) -> {
|
||||
assertFalse(logDirInfo.isEmpty());
|
||||
|
@ -98,7 +96,7 @@ public class LogDirsCommandTest {
|
|||
|
||||
@ClusterTest
|
||||
public void testLogDirsWithNonExistentTopic(ClusterInstance clusterInstance) {
|
||||
try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
String output = assertDoesNotThrow(() -> execute(fromArgsToOptions("--bootstrap-server", clusterInstance.bootstrapServers(), "--topic-list", TOPIC, "--describe"), admin));
|
||||
// check all brokers are present
|
||||
clusterInstance.brokerIds().forEach(brokerId -> assertTrue(output.contains("\"broker\":" + brokerId)));
|
||||
|
@ -109,9 +107,9 @@ public class LogDirsCommandTest {
|
|||
logDirs.forEach((brokerId, logDirInfo) ->
|
||||
logDirInfo.forEach((logDir, logDirInfoValue) -> {
|
||||
assertTrue(output.contains("\"logDir\":\"" + logDir + "\""));
|
||||
logDirInfoValue.replicaInfos().forEach((topicPartition, replicaInfo) -> {
|
||||
assertFalse(output.contains("\"partition\":\"" + topicPartition + "\""));
|
||||
});
|
||||
logDirInfoValue.replicaInfos().forEach((topicPartition, replicaInfo) ->
|
||||
assertFalse(output.contains("\"partition\":\"" + topicPartition + "\""))
|
||||
);
|
||||
}));
|
||||
}
|
||||
}
|
||||
|
@ -120,7 +118,7 @@ public class LogDirsCommandTest {
|
|||
public void testLogDirsWithSpecificTopic(ClusterInstance clusterInstance) {
|
||||
createTopic(clusterInstance, TOPIC);
|
||||
createTopic(clusterInstance, "other-topic");
|
||||
try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
String output = assertDoesNotThrow(() -> execute(fromArgsToOptions("--bootstrap-server", clusterInstance.bootstrapServers(), "--topic-list", TOPIC, "--describe"), admin));
|
||||
// check all brokers are present
|
||||
clusterInstance.brokerIds().forEach(brokerId -> assertTrue(output.contains("\"broker\":" + brokerId)));
|
||||
|
@ -133,9 +131,9 @@ public class LogDirsCommandTest {
|
|||
logDirs.forEach((brokerId, logDirInfo) ->
|
||||
logDirInfo.forEach((logDir, logDirInfoValue) -> {
|
||||
assertTrue(output.contains("\"logDir\":\"" + logDir + "\""));
|
||||
logDirInfoValue.replicaInfos().keySet().stream().filter(tp -> !tp.topic().equals(TOPIC)).forEach(tp -> {
|
||||
assertFalse(output.contains("\"partition\":\"" + tp + "\""));
|
||||
});
|
||||
logDirInfoValue.replicaInfos().keySet().stream().filter(tp -> !tp.topic().equals(TOPIC)).forEach(tp ->
|
||||
assertFalse(output.contains("\"partition\":\"" + tp + "\""))
|
||||
);
|
||||
}));
|
||||
}
|
||||
}
|
||||
|
@ -143,7 +141,7 @@ public class LogDirsCommandTest {
|
|||
@Test
|
||||
public void shouldThrowWhenQueryingNonExistentBrokers() {
|
||||
Node broker = new Node(1, "hostname", 9092);
|
||||
try (MockAdminClient adminClient = new MockAdminClient(Collections.singletonList(broker), broker)) {
|
||||
try (MockAdminClient adminClient = new MockAdminClient(List.of(broker), broker)) {
|
||||
RuntimeException exception = assertThrows(RuntimeException.class, () -> execute(fromArgsToOptions("--bootstrap-server", "EMPTY", "--broker-list", "0,1,2", "--describe"), adminClient));
|
||||
assertNotNull(exception.getCause());
|
||||
assertEquals(TerseException.class, exception.getCause().getClass());
|
||||
|
@ -155,7 +153,7 @@ public class LogDirsCommandTest {
|
|||
@SuppressWarnings("unchecked")
|
||||
public void shouldNotThrowWhenDuplicatedBrokers() throws JsonProcessingException {
|
||||
Node broker = new Node(1, "hostname", 9092);
|
||||
try (MockAdminClient adminClient = new MockAdminClient(Collections.singletonList(broker), broker)) {
|
||||
try (MockAdminClient adminClient = new MockAdminClient(List.of(broker), broker)) {
|
||||
String standardOutput = execute(fromArgsToOptions("--bootstrap-server", "EMPTY", "--broker-list", "1,1", "--describe"), adminClient);
|
||||
String[] standardOutputLines = standardOutput.split("\n");
|
||||
assertEquals(3, standardOutputLines.length);
|
||||
|
@ -172,13 +170,13 @@ public class LogDirsCommandTest {
|
|||
public void shouldQueryAllBrokersIfNonSpecified() throws JsonProcessingException {
|
||||
Node brokerOne = new Node(1, "hostname", 9092);
|
||||
Node brokerTwo = new Node(2, "hostname", 9092);
|
||||
try (MockAdminClient adminClient = new MockAdminClient(Arrays.asList(brokerTwo, brokerOne), brokerOne)) {
|
||||
try (MockAdminClient adminClient = new MockAdminClient(List.of(brokerTwo, brokerOne), brokerOne)) {
|
||||
String standardOutput = execute(fromArgsToOptions("--bootstrap-server", "EMPTY", "--describe"), adminClient);
|
||||
String[] standardOutputLines = standardOutput.split("\n");
|
||||
assertEquals(3, standardOutputLines.length);
|
||||
Map<String, Object> information = new ObjectMapper().readValue(standardOutputLines[2], HashMap.class);
|
||||
List<Object> brokersInformation = (List<Object>) information.get("brokers");
|
||||
Set<Integer> brokerIds = new HashSet<Integer>() {{
|
||||
Set<Integer> brokerIds = new HashSet<>() {{
|
||||
add((Integer) ((HashMap<String, Object>) brokersInformation.get(0)).get("broker"));
|
||||
add((Integer) ((HashMap<String, Object>) brokersInformation.get(1)).get("broker"));
|
||||
}};
|
||||
|
@ -192,7 +190,7 @@ public class LogDirsCommandTest {
|
|||
public void shouldQuerySpecifiedBroker() throws JsonProcessingException {
|
||||
Node brokerOne = new Node(1, "hostname", 9092);
|
||||
Node brokerTwo = new Node(2, "hostname", 9092);
|
||||
try (MockAdminClient adminClient = new MockAdminClient(Arrays.asList(brokerOne, brokerTwo), brokerOne)) {
|
||||
try (MockAdminClient adminClient = new MockAdminClient(List.of(brokerOne, brokerTwo), brokerOne)) {
|
||||
String standardOutput = execute(fromArgsToOptions("--bootstrap-server", "EMPTY", "--broker-list", "1", "--describe"), adminClient);
|
||||
String[] standardOutputLines = standardOutput.split("\n");
|
||||
assertEquals(3, standardOutputLines.length);
|
||||
|
@ -220,8 +218,8 @@ public class LogDirsCommandTest {
|
|||
}
|
||||
|
||||
private void createTopic(ClusterInstance clusterInstance, String topic) {
|
||||
try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
assertDoesNotThrow(() -> admin.createTopics(Collections.singletonList(new NewTopic(topic, Collections.singletonMap(0, Collections.singletonList(0))))).topicId(topic).get());
|
||||
try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) {
|
||||
assertDoesNotThrow(() -> admin.createTopics(List.of(new NewTopic(topic, Map.of(0, List.of(0))))).topicId(topic).get());
|
||||
assertDoesNotThrow(() -> clusterInstance.waitTopicCreation(topic, 1));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -54,7 +53,7 @@ class MetadataQuorumCommandTest {
|
|||
MetadataQuorumCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication")
|
||||
);
|
||||
|
||||
List<String> outputs = Arrays.stream(describeOutput.split("\n")).collect(Collectors.toList());
|
||||
List<String> outputs = Arrays.stream(describeOutput.split("\n")).toList();
|
||||
String header = outputs.get(0);
|
||||
List<String> data = outputs.subList(1, outputs.size());
|
||||
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.junit.jupiter.api.Test;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
@ -41,7 +40,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
|||
public class MetadataQuorumCommandUnitTest {
|
||||
@Test
|
||||
public void testRemoveControllerDryRun() {
|
||||
List<String> outputs = Arrays.asList(
|
||||
List<String> outputs = List.of(
|
||||
ToolsTestUtils.captureStandardOut(() ->
|
||||
assertEquals(0, MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092",
|
||||
"remove-controller",
|
||||
|
@ -250,7 +249,7 @@ public class MetadataQuorumCommandUnitTest {
|
|||
new MetadataQuorumCommandUnitTestEnv(Optional.
|
||||
of(Uuid.fromString("wZoXPqWoSu6F6c8MkmdyAg")))) {
|
||||
File propsFile = testEnv.writePropertiesFile();
|
||||
List<String> outputs = Arrays.asList(
|
||||
List<String> outputs = List.of(
|
||||
ToolsTestUtils.captureStandardOut(() ->
|
||||
assertEquals(0, MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092",
|
||||
"--command-config", propsFile.getAbsolutePath(),
|
||||
|
|
|
@ -38,7 +38,6 @@ import java.io.IOException;
|
|||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.SplittableRandom;
|
||||
|
@ -94,7 +93,7 @@ public class ProducerPerformanceTest {
|
|||
|
||||
@Test
|
||||
public void testReadProps() throws Exception {
|
||||
List<String> producerProps = Collections.singletonList("bootstrap.servers=localhost:9000");
|
||||
List<String> producerProps = List.of("bootstrap.servers=localhost:9000");
|
||||
File producerConfig = createTempFile("acks=1");
|
||||
|
||||
Properties prop = ProducerPerformance.readProps(producerProps, producerConfig.getAbsolutePath());
|
||||
|
@ -360,7 +359,7 @@ public class ProducerPerformanceTest {
|
|||
|
||||
@Test
|
||||
public void testClientIdOverride() throws Exception {
|
||||
List<String> producerProps = Collections.singletonList("client.id=producer-1");
|
||||
List<String> producerProps = List.of("client.id=producer-1");
|
||||
|
||||
Properties prop = ProducerPerformance.readProps(producerProps, null);
|
||||
|
||||
|
@ -370,7 +369,7 @@ public class ProducerPerformanceTest {
|
|||
|
||||
@Test
|
||||
public void testDefaultClientId() throws Exception {
|
||||
List<String> producerProps = Collections.singletonList("acks=1");
|
||||
List<String> producerProps = List.of("acks=1");
|
||||
|
||||
Properties prop = ProducerPerformance.readProps(producerProps, null);
|
||||
|
||||
|
@ -391,9 +390,7 @@ public class ProducerPerformanceTest {
|
|||
ProducerPerformance.Stats stats = new ProducerPerformance.Stats(numRecords, false);
|
||||
for (long i = 0; i < numRecords; i++) {
|
||||
final Callback callback = new ProducerPerformance.PerfCallback(0, 100, stats, null);
|
||||
CompletableFuture.runAsync(() -> {
|
||||
callback.onCompletion(null, null);
|
||||
}, singleThreaded);
|
||||
CompletableFuture.runAsync(() -> callback.onCompletion(null, null), singleThreaded);
|
||||
}
|
||||
|
||||
singleThreaded.shutdown();
|
||||
|
|
|
@ -43,8 +43,7 @@ import java.net.InetAddress;
|
|||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -186,41 +185,41 @@ public class PushHttpMetricsReporterTest {
|
|||
verifyConfigure();
|
||||
KafkaMetric metric1 = new KafkaMetric(
|
||||
new Object(),
|
||||
new MetricName("name1", "group1", "desc1", Collections.singletonMap("key1", "value1")),
|
||||
new MetricName("name1", "group1", "desc1", Map.of("key1", "value1")),
|
||||
new ImmutableValue<>(1.0),
|
||||
null,
|
||||
time
|
||||
);
|
||||
KafkaMetric newMetric1 = new KafkaMetric(
|
||||
new Object(),
|
||||
new MetricName("name1", "group1", "desc1", Collections.singletonMap("key1", "value1")),
|
||||
new MetricName("name1", "group1", "desc1", Map.of("key1", "value1")),
|
||||
new ImmutableValue<>(-1.0),
|
||||
null,
|
||||
time
|
||||
);
|
||||
KafkaMetric metric2 = new KafkaMetric(
|
||||
new Object(),
|
||||
new MetricName("name2", "group2", "desc2", Collections.singletonMap("key2", "value2")),
|
||||
new MetricName("name2", "group2", "desc2", Map.of("key2", "value2")),
|
||||
new ImmutableValue<>(2.0),
|
||||
null,
|
||||
time
|
||||
);
|
||||
KafkaMetric metric3 = new KafkaMetric(
|
||||
new Object(),
|
||||
new MetricName("name3", "group3", "desc3", Collections.singletonMap("key3", "value3")),
|
||||
new MetricName("name3", "group3", "desc3", Map.of("key3", "value3")),
|
||||
new ImmutableValue<>(3.0),
|
||||
null,
|
||||
time
|
||||
);
|
||||
KafkaMetric metric4 = new KafkaMetric(
|
||||
new Object(),
|
||||
new MetricName("name4", "group4", "desc4", Collections.singletonMap("key4", "value4")),
|
||||
new MetricName("name4", "group4", "desc4", Map.of("key4", "value4")),
|
||||
new ImmutableValue<>("value4"),
|
||||
null,
|
||||
time
|
||||
);
|
||||
|
||||
reporter.init(Arrays.asList(metric1, metric2, metric4));
|
||||
reporter.init(List.of(metric1, metric2, metric4));
|
||||
reporter.metricChange(newMetric1); // added in init, modified
|
||||
reporter.metricChange(metric3); // added by change
|
||||
reporter.metricRemoval(metric2); // added in init, deleted by removal
|
||||
|
@ -236,7 +235,7 @@ public class PushHttpMetricsReporterTest {
|
|||
JsonNode metrics = payload.get("metrics");
|
||||
assertTrue(metrics.isArray());
|
||||
assertEquals(3, metrics.size());
|
||||
List<JsonNode> metricsList = Arrays.asList(metrics.get(0), metrics.get(1), metrics.get(2));
|
||||
List<JsonNode> metricsList = new ArrayList<>(List.of(metrics.get(0), metrics.get(1), metrics.get(2)));
|
||||
// Sort metrics based on name so that we can verify the value for each metric below
|
||||
metricsList.sort(Comparator.comparing(m -> m.get("name").textValue()));
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.kafka.common.record.SimpleRecord;
|
|||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.stream.IntStream;
|
||||
|
@ -43,8 +42,8 @@ public class ReplicaVerificationToolTest {
|
|||
}};
|
||||
|
||||
ReplicaVerificationTool.ReplicaBuffer replicaBuffer =
|
||||
new ReplicaVerificationTool.ReplicaBuffer(expectedReplicasPerTopicAndPartition, Collections.emptyMap(), 2, 0);
|
||||
expectedReplicasPerTopicAndPartition.forEach((tp, numReplicas) -> {
|
||||
new ReplicaVerificationTool.ReplicaBuffer(expectedReplicasPerTopicAndPartition, Map.of(), 2, 0);
|
||||
expectedReplicasPerTopicAndPartition.forEach((tp, numReplicas) ->
|
||||
IntStream.range(0, numReplicas).forEach(replicaId -> {
|
||||
SimpleRecord[] records = IntStream.rangeClosed(0, 5)
|
||||
.mapToObj(index -> new SimpleRecord(("key " + index).getBytes(), ("value " + index).getBytes()))
|
||||
|
@ -60,8 +59,8 @@ public class ReplicaVerificationToolTest {
|
|||
.setRecords(memoryRecords);
|
||||
|
||||
replicaBuffer.addFetchedData(tp, replicaId, partitionData);
|
||||
});
|
||||
});
|
||||
})
|
||||
);
|
||||
|
||||
replicaBuffer.verifyCheckSum(line -> sb.append(format("%s%n", line)));
|
||||
String output = sb.toString().trim();
|
||||
|
|
|
@ -33,9 +33,9 @@ import org.junit.jupiter.api.Timeout;
|
|||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -53,7 +53,7 @@ public class StreamsResetterTest {
|
|||
|
||||
@BeforeEach
|
||||
public void beforeEach() {
|
||||
consumer.assign(Collections.singletonList(topicPartition));
|
||||
consumer.assign(List.of(topicPartition));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0L, new byte[] {}, new byte[] {}));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1L, new byte[] {}, new byte[] {}));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 2L, new byte[] {}, new byte[] {}));
|
||||
|
@ -82,7 +82,7 @@ public class StreamsResetterTest {
|
|||
final long beginningOffset = 5L;
|
||||
final long endOffset = 10L;
|
||||
final MockConsumer<byte[], byte[]> emptyConsumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name());
|
||||
emptyConsumer.assign(Collections.singletonList(topicPartition));
|
||||
emptyConsumer.assign(List.of(topicPartition));
|
||||
|
||||
final Map<TopicPartition, Long> beginningOffsetsMap = new HashMap<>();
|
||||
beginningOffsetsMap.put(topicPartition, beginningOffset);
|
||||
|
@ -255,10 +255,10 @@ public class StreamsResetterTest {
|
|||
public void shouldDeleteTopic() throws InterruptedException, ExecutionException {
|
||||
final Cluster cluster = createCluster(1);
|
||||
try (final MockAdminClient adminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
|
||||
final TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList());
|
||||
adminClient.addTopic(false, TOPIC, Collections.singletonList(topicPartitionInfo), null);
|
||||
streamsResetter.doDelete(Collections.singletonList(TOPIC), adminClient);
|
||||
assertEquals(Collections.emptySet(), adminClient.listTopics().names().get());
|
||||
final TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
|
||||
adminClient.addTopic(false, TOPIC, List.of(topicPartitionInfo), null);
|
||||
streamsResetter.doDelete(List.of(TOPIC), adminClient);
|
||||
assertEquals(Set.of(), adminClient.listTopics().names().get());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -274,7 +274,7 @@ public class StreamsResetterTest {
|
|||
public void testResetToDatetimeWhenPartitionIsEmptyResetsToLatestOffset() {
|
||||
final long beginningAndEndOffset = 5L; // Empty partition implies beginning offset == end offset
|
||||
final MockConsumer<byte[], byte[]> emptyConsumer = new EmptyPartitionConsumer<>(AutoOffsetResetStrategy.EARLIEST.name());
|
||||
emptyConsumer.assign(Collections.singletonList(topicPartition));
|
||||
emptyConsumer.assign(List.of(topicPartition));
|
||||
|
||||
final Map<TopicPartition, Long> beginningOffsetsMap = new HashMap<>();
|
||||
beginningOffsetsMap.put(topicPartition, beginningAndEndOffset);
|
||||
|
@ -299,8 +299,8 @@ public class StreamsResetterTest {
|
|||
nodes.put(i, new Node(i, "localhost", 8121 + i));
|
||||
}
|
||||
return new Cluster("mockClusterId", nodes.values(),
|
||||
Collections.emptySet(), Collections.emptySet(),
|
||||
Collections.emptySet(), nodes.get(0));
|
||||
Set.of(), Set.of(),
|
||||
Set.of(), nodes.get(0));
|
||||
}
|
||||
|
||||
private static class EmptyPartitionConsumer<K, V> extends MockConsumer<K, V> {
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.io.IOException;
|
|||
import java.io.PrintStream;
|
||||
import java.util.AbstractMap.SimpleImmutableEntry;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -146,7 +145,7 @@ public class ToolsTestUtils {
|
|||
public static void removePartitionReplicaThrottles(Admin adminClient, Set<TopicPartition> partitions) throws ExecutionException, InterruptedException {
|
||||
Map<ConfigResource, Collection<AlterConfigOp>> throttles = partitions.stream().collect(Collectors.toMap(
|
||||
tp -> new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()),
|
||||
tp -> Arrays.asList(
|
||||
tp -> List.of(
|
||||
new AlterConfigOp(new ConfigEntry(QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""),
|
||||
AlterConfigOp.OpType.DELETE),
|
||||
new AlterConfigOp(new ConfigEntry(QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""),
|
||||
|
|
|
@ -110,7 +110,7 @@ public class TopicCommandTest {
|
|||
|
||||
@Test
|
||||
public void testIsNotUnderReplicatedWhenAdding() {
|
||||
List<Integer> replicaIds = Arrays.asList(1, 2);
|
||||
List<Integer> replicaIds = List.of(1, 2);
|
||||
List<Node> replicas = new ArrayList<>();
|
||||
for (int id : replicaIds) {
|
||||
replicas.add(new Node(id, "localhost", 9090 + id));
|
||||
|
@ -118,9 +118,9 @@ public class TopicCommandTest {
|
|||
|
||||
TopicCommand.PartitionDescription partitionDescription = new TopicCommand.PartitionDescription("test-topic",
|
||||
new TopicPartitionInfo(0, new Node(1, "localhost", 9091), replicas,
|
||||
Collections.singletonList(new Node(1, "localhost", 9091))),
|
||||
List.of(new Node(1, "localhost", 9091))),
|
||||
null, false,
|
||||
new PartitionReassignment(replicaIds, Arrays.asList(2), Collections.emptyList())
|
||||
new PartitionReassignment(replicaIds, List.of(2), List.of())
|
||||
);
|
||||
|
||||
assertFalse(partitionDescription.isUnderReplicated());
|
||||
|
@ -236,9 +236,9 @@ public class TopicCommandTest {
|
|||
public void testParseAssignment() {
|
||||
Map<Integer, List<Integer>> actualAssignment = TopicCommand.parseReplicaAssignment("5:4,3:2,1:0");
|
||||
Map<Integer, List<Integer>> expectedAssignment = new HashMap<>();
|
||||
expectedAssignment.put(0, Arrays.asList(5, 4));
|
||||
expectedAssignment.put(1, Arrays.asList(3, 2));
|
||||
expectedAssignment.put(2, Arrays.asList(1, 0));
|
||||
expectedAssignment.put(0, List.of(5, 4));
|
||||
expectedAssignment.put(1, List.of(3, 2));
|
||||
expectedAssignment.put(2, List.of(1, 0));
|
||||
assertEquals(expectedAssignment, actualAssignment);
|
||||
}
|
||||
|
||||
|
@ -257,7 +257,7 @@ public class TopicCommandTest {
|
|||
})));
|
||||
|
||||
NewTopic expectedNewTopic = new NewTopic(topicName, Optional.empty(), Optional.empty())
|
||||
.configs(Collections.emptyMap());
|
||||
.configs(Map.of());
|
||||
|
||||
verify(adminClient, times(1)).createTopics(
|
||||
eq(Set.of(expectedNewTopic)),
|
||||
|
@ -285,7 +285,7 @@ public class TopicCommandTest {
|
|||
assertInstanceOf(ThrottlingQuotaExceededException.class, exception.getCause());
|
||||
|
||||
verify(adminClient).deleteTopics(
|
||||
argThat((Collection<String> topics) -> topics.equals(Arrays.asList(topicName))),
|
||||
argThat((Collection<String> topics) -> topics.equals(List.of(topicName))),
|
||||
argThat((DeleteTopicsOptions options) -> !options.shouldRetryOnQuotaViolation()));
|
||||
}
|
||||
|
||||
|
@ -298,9 +298,9 @@ public class TopicCommandTest {
|
|||
when(adminClient.listTopics(any())).thenReturn(listResult);
|
||||
|
||||
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, new Node(0, "", 0),
|
||||
Collections.emptyList(), Collections.emptyList());
|
||||
List.of(), List.of());
|
||||
DescribeTopicsResult describeResult = AdminClientTestUtils.describeTopicsResult(topicName,
|
||||
new TopicDescription(topicName, false, Collections.singletonList(topicPartitionInfo)));
|
||||
new TopicDescription(topicName, false, List.of(topicPartitionInfo)));
|
||||
when(adminClient.describeTopics(anyCollection())).thenReturn(describeResult);
|
||||
|
||||
CreatePartitionsResult result = AdminClientTestUtils.createPartitionsResult(topicName, Errors.THROTTLING_QUOTA_EXCEEDED.exception());
|
||||
|
@ -365,7 +365,7 @@ public class TopicCommandTest {
|
|||
rackInfo.put(4, infoPerBroker5);
|
||||
rackInfo.put(5, infoPerBroker6);
|
||||
|
||||
return Collections.singletonList(ClusterConfig.defaultBuilder()
|
||||
return List.of(ClusterConfig.defaultBuilder()
|
||||
.setBrokers(6)
|
||||
.setServerProperties(serverProp)
|
||||
.setPerServerProperties(rackInfo)
|
||||
|
@ -385,13 +385,13 @@ public class TopicCommandTest {
|
|||
String testTopicName = TestUtils.randomString(10);
|
||||
|
||||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
Assertions.assertTrue(adminClient.listTopics().names().get().contains(testTopicName),
|
||||
"Admin client didn't see the created topic. It saw: " + adminClient.listTopics().names().get());
|
||||
|
||||
adminClient.deleteTopics(Collections.singletonList(testTopicName));
|
||||
adminClient.deleteTopics(List.of(testTopicName));
|
||||
clusterInstance.waitTopicDeletion(testTopicName);
|
||||
Assertions.assertTrue(adminClient.listTopics().names().get().isEmpty(),
|
||||
"Admin client see the created topic. It saw: " + adminClient.listTopics().names().get());
|
||||
|
@ -409,14 +409,14 @@ public class TopicCommandTest {
|
|||
String testTopicName = TestUtils.randomString(10);
|
||||
|
||||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
Assertions.assertTrue(adminClient.listTopics().names().get().contains(testTopicName),
|
||||
"Admin client didn't see the created topic. It saw: " + adminClient.listTopics().names().get());
|
||||
|
||||
List<TopicPartitionInfo> partitions = adminClient
|
||||
.describeTopics(Collections.singletonList(testTopicName))
|
||||
.describeTopics(List.of(testTopicName))
|
||||
.allTopicNames()
|
||||
.get()
|
||||
.get(testTopicName)
|
||||
|
@ -424,7 +424,7 @@ public class TopicCommandTest {
|
|||
Assertions.assertEquals(defaultNumPartitions, partitions.size(), "Unequal partition size: " + partitions.size());
|
||||
Assertions.assertEquals(defaultReplicationFactor, (short) partitions.get(0).replicas().size(), "Unequal replication factor: " + partitions.get(0).replicas().size());
|
||||
|
||||
adminClient.deleteTopics(Collections.singletonList(testTopicName));
|
||||
adminClient.deleteTopics(List.of(testTopicName));
|
||||
clusterInstance.waitTopicDeletion(testTopicName);
|
||||
Assertions.assertTrue(adminClient.listTopics().names().get().isEmpty(),
|
||||
"Admin client see the created topic. It saw: " + adminClient.listTopics().names().get());
|
||||
|
@ -442,10 +442,10 @@ public class TopicCommandTest {
|
|||
String testTopicName = TestUtils.randomString(10);
|
||||
|
||||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, 2, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, 2, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, 2);
|
||||
List<TopicPartitionInfo> partitions = adminClient
|
||||
.describeTopics(Collections.singletonList(testTopicName))
|
||||
.describeTopics(List.of(testTopicName))
|
||||
.allTopicNames()
|
||||
.get()
|
||||
.get(testTopicName)
|
||||
|
@ -460,10 +460,10 @@ public class TopicCommandTest {
|
|||
String testTopicName = TestUtils.randomString(10);
|
||||
|
||||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, (short) 2)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, (short) 2)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
List<TopicPartitionInfo> partitions = adminClient
|
||||
.describeTopics(Collections.singletonList(testTopicName))
|
||||
.describeTopics(List.of(testTopicName))
|
||||
.allTopicNames()
|
||||
.get()
|
||||
.get(testTopicName)
|
||||
|
@ -483,11 +483,11 @@ public class TopicCommandTest {
|
|||
Map<String, String> topicConfig = new HashMap<>();
|
||||
topicConfig.put(TopicConfig.DELETE_RETENTION_MS_CONFIG, "1000");
|
||||
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2).configs(topicConfig)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, 2, (short) 2).configs(topicConfig)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, 2);
|
||||
|
||||
|
||||
Config configs = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
|
||||
Config configs = adminClient.describeConfigs(Set.of(configResource)).all().get().get(configResource);
|
||||
assertEquals(1000, Integer.valueOf(configs.get("delete.retention.ms").value()),
|
||||
"Config not set correctly: " + configs.get("delete.retention.ms").value());
|
||||
}
|
||||
|
@ -502,7 +502,7 @@ public class TopicCommandTest {
|
|||
clusterInstance, "--create", "--partitions", Integer.toString(defaultNumPartitions), "--replication-factor", "1",
|
||||
"--topic", testTopicName);
|
||||
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
// try to re-create the topic
|
||||
|
@ -516,7 +516,7 @@ public class TopicCommandTest {
|
|||
String testTopicName = TestUtils.randomString(10);
|
||||
try (Admin adminClient = clusterInstance.admin();
|
||||
TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
TopicCommand.TopicCommandOptions createOpts =
|
||||
|
@ -526,7 +526,7 @@ public class TopicCommandTest {
|
|||
}
|
||||
|
||||
private List<Integer> getPartitionReplicas(List<TopicPartitionInfo> partitions, int partitionNumber) {
|
||||
return partitions.get(partitionNumber).replicas().stream().map(Node::id).collect(Collectors.toList());
|
||||
return partitions.get(partitionNumber).replicas().stream().map(Node::id).toList();
|
||||
}
|
||||
|
||||
@ClusterTemplate("generate")
|
||||
|
@ -535,15 +535,15 @@ public class TopicCommandTest {
|
|||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
String testTopicName = TestUtils.randomString(10);
|
||||
|
||||
replicaAssignmentMap.put(0, Arrays.asList(5, 4));
|
||||
replicaAssignmentMap.put(1, Arrays.asList(3, 2));
|
||||
replicaAssignmentMap.put(2, Arrays.asList(1, 0));
|
||||
replicaAssignmentMap.put(0, List.of(5, 4));
|
||||
replicaAssignmentMap.put(1, List.of(3, 2));
|
||||
replicaAssignmentMap.put(2, List.of(1, 0));
|
||||
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, replicaAssignmentMap)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, replicaAssignmentMap)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, 3);
|
||||
|
||||
List<TopicPartitionInfo> partitions = adminClient
|
||||
.describeTopics(Collections.singletonList(testTopicName))
|
||||
.describeTopics(List.of(testTopicName))
|
||||
.allTopicNames()
|
||||
.get()
|
||||
.get(testTopicName)
|
||||
|
@ -551,11 +551,11 @@ public class TopicCommandTest {
|
|||
|
||||
assertEquals(3, partitions.size(),
|
||||
"Unequal partition size: " + partitions.size());
|
||||
assertEquals(Arrays.asList(5, 4), getPartitionReplicas(partitions, 0),
|
||||
assertEquals(List.of(5, 4), getPartitionReplicas(partitions, 0),
|
||||
"Unexpected replica assignment: " + getPartitionReplicas(partitions, 0));
|
||||
assertEquals(Arrays.asList(3, 2), getPartitionReplicas(partitions, 1),
|
||||
assertEquals(List.of(3, 2), getPartitionReplicas(partitions, 1),
|
||||
"Unexpected replica assignment: " + getPartitionReplicas(partitions, 1));
|
||||
assertEquals(Arrays.asList(1, 0), getPartitionReplicas(partitions, 2),
|
||||
assertEquals(List.of(1, 0), getPartitionReplicas(partitions, 2),
|
||||
"Unexpected replica assignment: " + getPartitionReplicas(partitions, 2));
|
||||
}
|
||||
}
|
||||
|
@ -610,7 +610,7 @@ public class TopicCommandTest {
|
|||
public void testListTopics(ClusterInstance clusterInstance) throws InterruptedException {
|
||||
String testTopicName = TestUtils.randomString(10);
|
||||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
String output = captureListTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--list"));
|
||||
|
@ -626,9 +626,9 @@ public class TopicCommandTest {
|
|||
String topic3 = "oooof.testTopic1";
|
||||
int partition = 2;
|
||||
short replicationFactor = 2;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(topic1, partition, replicationFactor)));
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(topic2, partition, replicationFactor)));
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(topic3, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(topic1, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(topic2, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(topic3, partition, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(topic1, partition);
|
||||
clusterInstance.waitTopicCreation(topic2, partition);
|
||||
clusterInstance.waitTopicCreation(topic3, partition);
|
||||
|
@ -647,7 +647,7 @@ public class TopicCommandTest {
|
|||
String hiddenConsumerTopic = Topic.GROUP_METADATA_TOPIC_NAME;
|
||||
int partition = 2;
|
||||
short replicationFactor = 2;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(topic1, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(topic1, partition, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(topic1, partition);
|
||||
|
||||
String output = captureListTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--list", "--exclude-internal"));
|
||||
|
@ -663,7 +663,7 @@ public class TopicCommandTest {
|
|||
TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) {
|
||||
int partition = 2;
|
||||
short replicationFactor = 2;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partition);
|
||||
topicService.alterTopic(buildTopicCommandOptionsWithBootstrap(clusterInstance, "--alter", "--topic", testTopicName, "--partitions", "3"));
|
||||
|
||||
|
@ -676,7 +676,7 @@ public class TopicCommandTest {
|
|||
() -> clusterInstance.brokers().values().stream().allMatch(
|
||||
b -> b.metadataCache().numPartitions(testTopicName).orElse(0) == 3),
|
||||
TestUtils.DEFAULT_MAX_WAIT_MS, "Timeout waiting for new assignment propagating to broker");
|
||||
TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
|
||||
TopicDescription topicDescription = adminClient.describeTopics(List.of(testTopicName)).topicNameValues().get(testTopicName).get();
|
||||
assertEquals(3, topicDescription.partitions().size(), "Expected partition count to be 3. Got: " + topicDescription.partitions().size());
|
||||
}
|
||||
}
|
||||
|
@ -689,7 +689,7 @@ public class TopicCommandTest {
|
|||
int partition = 2;
|
||||
short replicationFactor = 2;
|
||||
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partition);
|
||||
|
||||
topicService.alterTopic(buildTopicCommandOptionsWithBootstrap(clusterInstance, "--alter",
|
||||
|
@ -705,10 +705,10 @@ public class TopicCommandTest {
|
|||
b -> b.metadataCache().numPartitions(testTopicName).orElse(0) == 3),
|
||||
TestUtils.DEFAULT_MAX_WAIT_MS, "Timeout waiting for new assignment propagating to broker");
|
||||
|
||||
TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
|
||||
TopicDescription topicDescription = adminClient.describeTopics(List.of(testTopicName)).topicNameValues().get(testTopicName).get();
|
||||
assertEquals(3, topicDescription.partitions().size(), "Expected partition count to be 3. Got: " + topicDescription.partitions().size());
|
||||
List<Integer> partitionReplicas = getPartitionReplicas(topicDescription.partitions(), 2);
|
||||
assertEquals(Arrays.asList(4, 2), partitionReplicas, "Expected to have replicas 4,2. Got: " + partitionReplicas);
|
||||
assertEquals(List.of(4, 2), partitionReplicas, "Expected to have replicas 4,2. Got: " + partitionReplicas);
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -721,7 +721,7 @@ public class TopicCommandTest {
|
|||
|
||||
int partition = 2;
|
||||
short replicationFactor = 2;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partition);
|
||||
|
||||
assertThrows(ExecutionException.class,
|
||||
|
@ -739,7 +739,7 @@ public class TopicCommandTest {
|
|||
TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) {
|
||||
int partition = 2;
|
||||
short replicationFactor = 2;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partition);
|
||||
|
||||
assertThrows(ExecutionException.class,
|
||||
|
@ -756,7 +756,7 @@ public class TopicCommandTest {
|
|||
|
||||
try (Admin adminClient = clusterInstance.admin();
|
||||
TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
assertThrows(ExecutionException.class,
|
||||
|
@ -805,15 +805,15 @@ public class TopicCommandTest {
|
|||
|
||||
int numPartitions = 18;
|
||||
int replicationFactor = 3;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, numPartitions, (short) replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, numPartitions, (short) replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, numPartitions);
|
||||
|
||||
Map<Integer, List<Integer>> assignment = adminClient.describeTopics(Collections.singletonList(testTopicName))
|
||||
Map<Integer, List<Integer>> assignment = adminClient.describeTopics(List.of(testTopicName))
|
||||
.allTopicNames().get().get(testTopicName).partitions()
|
||||
.stream()
|
||||
.collect(Collectors.toMap(
|
||||
info -> info.partition(),
|
||||
info -> info.replicas().stream().map(Node::id).collect(Collectors.toList())));
|
||||
TopicPartitionInfo::partition,
|
||||
info -> info.replicas().stream().map(Node::id).toList()));
|
||||
checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), numPartitions,
|
||||
replicationFactor, true, true, true);
|
||||
|
||||
|
@ -832,9 +832,9 @@ public class TopicCommandTest {
|
|||
() -> clusterInstance.brokers().values().stream().allMatch(p -> p.metadataCache().numPartitions(testTopicName).orElse(0) == alteredNumPartitions),
|
||||
TestUtils.DEFAULT_MAX_WAIT_MS, "Timeout waiting for new assignment propagating to broker");
|
||||
|
||||
assignment = adminClient.describeTopics(Collections.singletonList(testTopicName))
|
||||
assignment = adminClient.describeTopics(List.of(testTopicName))
|
||||
.allTopicNames().get().get(testTopicName).partitions().stream()
|
||||
.collect(Collectors.toMap(info -> info.partition(), info -> info.replicas().stream().map(Node::id).collect(Collectors.toList())));
|
||||
.collect(Collectors.toMap(TopicPartitionInfo::partition, info -> info.replicas().stream().map(Node::id).toList()));
|
||||
checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), alteredNumPartitions, replicationFactor,
|
||||
true, true, true);
|
||||
|
||||
|
@ -850,11 +850,11 @@ public class TopicCommandTest {
|
|||
String cleanUpPolicy = "compact";
|
||||
HashMap<String, String> topicConfig = new HashMap<>();
|
||||
topicConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, cleanUpPolicy);
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor).configs(topicConfig)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor).configs(topicConfig)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
|
||||
Config props = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
|
||||
Config props = adminClient.describeConfigs(Set.of(configResource)).all().get().get(configResource);
|
||||
assertNotNull(props.get(TopicConfig.CLEANUP_POLICY_CONFIG), "Properties after creation don't contain " + cleanUpPolicy);
|
||||
assertEquals(cleanUpPolicy, props.get(TopicConfig.CLEANUP_POLICY_CONFIG).value(), "Properties after creation have incorrect value");
|
||||
|
||||
|
@ -868,7 +868,7 @@ public class TopicCommandTest {
|
|||
() -> clusterInstance.brokers().values().stream().allMatch(p -> p.metadataCache().numPartitions(testTopicName).orElse(0) == numPartitionsModified),
|
||||
TestUtils.DEFAULT_MAX_WAIT_MS, "Timeout waiting for new assignment propagating to broker");
|
||||
|
||||
Config newProps = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
|
||||
Config newProps = adminClient.describeConfigs(Set.of(configResource)).all().get().get(configResource);
|
||||
assertNotNull(newProps.get(TopicConfig.CLEANUP_POLICY_CONFIG), "Updated properties do not contain " + TopicConfig.CLEANUP_POLICY_CONFIG);
|
||||
assertEquals(cleanUpPolicy, newProps.get(TopicConfig.CLEANUP_POLICY_CONFIG).value(), "Updated properties have incorrect value");
|
||||
|
||||
|
@ -887,7 +887,7 @@ public class TopicCommandTest {
|
|||
TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) {
|
||||
String testTopicName = TestUtils.randomString(10);
|
||||
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
// delete the NormalTopic
|
||||
|
@ -913,7 +913,7 @@ public class TopicCommandTest {
|
|||
TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) {
|
||||
// create the topic with colliding chars
|
||||
String topicWithCollidingChar = "test.a";
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(topicWithCollidingChar, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(topicWithCollidingChar, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(topicWithCollidingChar, defaultNumPartitions);
|
||||
|
||||
// delete the topic
|
||||
|
@ -927,7 +927,7 @@ public class TopicCommandTest {
|
|||
clusterInstance.waitTopicDeletion(topicWithCollidingChar);
|
||||
|
||||
// recreate same topic
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(topicWithCollidingChar, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(topicWithCollidingChar, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(topicWithCollidingChar, defaultNumPartitions);
|
||||
}
|
||||
}
|
||||
|
@ -944,7 +944,7 @@ public class TopicCommandTest {
|
|||
TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) {
|
||||
|
||||
// create the offset topic
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(Topic.GROUP_METADATA_TOPIC_NAME, defaultNumPartitions);
|
||||
|
||||
// Try to delete the Topic.GROUP_METADATA_TOPIC_NAME which is allowed by default.
|
||||
|
@ -1001,7 +1001,7 @@ public class TopicCommandTest {
|
|||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
int partition = 2;
|
||||
short replicationFactor = 2;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partition);
|
||||
|
||||
String output = captureDescribeTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--describe", "--topic", testTopicName));
|
||||
|
@ -1075,7 +1075,7 @@ public class TopicCommandTest {
|
|||
int partitions = 3;
|
||||
short replicationFactor = 1;
|
||||
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partitions);
|
||||
|
||||
// check which partition is on broker 0 which we'll kill
|
||||
|
@ -1102,11 +1102,11 @@ public class TopicCommandTest {
|
|||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
int partitions = 1;
|
||||
short replicationFactor = 3;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partitions);
|
||||
|
||||
clusterInstance.shutdownBroker(0);
|
||||
Assertions.assertEquals(clusterInstance.aliveBrokers().size(), 2);
|
||||
Assertions.assertEquals(2, clusterInstance.aliveBrokers().size());
|
||||
|
||||
TestUtils.waitForCondition(
|
||||
() -> clusterInstance.aliveBrokers().values().stream().allMatch(
|
||||
|
@ -1133,7 +1133,7 @@ public class TopicCommandTest {
|
|||
topicConfig.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3");
|
||||
int partitions = 1;
|
||||
short replicationFactor = 3;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partitions);
|
||||
|
||||
clusterInstance.shutdownBroker(0);
|
||||
|
@ -1157,7 +1157,7 @@ public class TopicCommandTest {
|
|||
|
||||
try (Admin adminClient = clusterInstance.admin();
|
||||
KafkaProducer<String, String> producer = createProducer(clusterInstance)) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
TopicPartition tp = new TopicPartition(testTopicName, 0);
|
||||
|
@ -1170,22 +1170,22 @@ public class TopicCommandTest {
|
|||
// throughput so the reassignment doesn't complete quickly.
|
||||
List<Integer> brokerIds = new ArrayList<>(clusterInstance.brokerIds());
|
||||
|
||||
ToolsTestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, Collections.singleton(tp), 1);
|
||||
ToolsTestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, Set.of(tp), 1);
|
||||
|
||||
TopicDescription testTopicDesc = adminClient.describeTopics(Collections.singleton(testTopicName)).allTopicNames().get().get(testTopicName);
|
||||
TopicDescription testTopicDesc = adminClient.describeTopics(Set.of(testTopicName)).allTopicNames().get().get(testTopicName);
|
||||
TopicPartitionInfo firstPartition = testTopicDesc.partitions().get(0);
|
||||
|
||||
List<Integer> replicasOfFirstPartition = firstPartition.replicas().stream().map(Node::id).collect(Collectors.toList());
|
||||
List<Integer> replicasOfFirstPartition = firstPartition.replicas().stream().map(Node::id).toList();
|
||||
List<Integer> replicasDiff = new ArrayList<>(brokerIds);
|
||||
replicasDiff.removeAll(replicasOfFirstPartition);
|
||||
Integer targetReplica = replicasDiff.get(0);
|
||||
|
||||
adminClient.alterPartitionReassignments(Collections.singletonMap(tp,
|
||||
Optional.of(new NewPartitionReassignment(Collections.singletonList(targetReplica))))).all().get();
|
||||
adminClient.alterPartitionReassignments(Map.of(tp,
|
||||
Optional.of(new NewPartitionReassignment(List.of(targetReplica))))).all().get();
|
||||
|
||||
// let's wait until the LAIR is propagated
|
||||
TestUtils.waitForCondition(
|
||||
() -> !adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get()
|
||||
() -> !adminClient.listPartitionReassignments(Set.of(tp)).reassignments().get()
|
||||
.get(tp).addingReplicas().isEmpty(), CLUSTER_WAIT_MS, "Reassignment didn't add the second node"
|
||||
);
|
||||
|
||||
|
@ -1208,7 +1208,7 @@ public class TopicCommandTest {
|
|||
|
||||
TestUtils.waitForCondition(
|
||||
() -> {
|
||||
PartitionReassignment tempReassignments = adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get().get(tp);
|
||||
PartitionReassignment tempReassignments = adminClient.listPartitionReassignments(Set.of(tp)).reassignments().get().get(tp);
|
||||
reassignmentsRef.set(tempReassignments);
|
||||
return reassignmentsRef.get() != null;
|
||||
}, waitTimeMs, "Reassignments did not become non-null within the specified time"
|
||||
|
@ -1216,7 +1216,7 @@ public class TopicCommandTest {
|
|||
|
||||
assertFalse(reassignmentsRef.get().addingReplicas().isEmpty());
|
||||
|
||||
ToolsTestUtils.removeReplicationThrottleForPartitions(adminClient, brokerIds, Collections.singleton(tp));
|
||||
ToolsTestUtils.removeReplicationThrottleForPartitions(adminClient, brokerIds, Set.of(tp));
|
||||
TestUtils.waitForCondition(
|
||||
() -> adminClient.listPartitionReassignments().reassignments().get().isEmpty(),
|
||||
CLUSTER_WAIT_MS, String.format("reassignmet not finished after %s ms", CLUSTER_WAIT_MS)
|
||||
|
@ -1235,7 +1235,7 @@ public class TopicCommandTest {
|
|||
int partitions = 1;
|
||||
short replicationFactor = 6;
|
||||
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partitions);
|
||||
|
||||
clusterInstance.shutdownBroker(0);
|
||||
|
@ -1278,10 +1278,10 @@ public class TopicCommandTest {
|
|||
List<NewTopic> newTopics = new ArrayList<>();
|
||||
|
||||
Map<Integer, List<Integer>> fullyReplicatedReplicaAssignmentMap = new HashMap<>();
|
||||
fullyReplicatedReplicaAssignmentMap.put(0, Arrays.asList(1, 2, 3));
|
||||
fullyReplicatedReplicaAssignmentMap.put(0, List.of(1, 2, 3));
|
||||
|
||||
Map<Integer, List<Integer>> offlineReplicaAssignmentMap = new HashMap<>();
|
||||
offlineReplicaAssignmentMap.put(0, Arrays.asList(0));
|
||||
offlineReplicaAssignmentMap.put(0, List.of(0));
|
||||
|
||||
Map<String, String> topicConfig = new HashMap<>();
|
||||
topicConfig.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "6");
|
||||
|
@ -1333,7 +1333,7 @@ public class TopicCommandTest {
|
|||
int partitions = 2;
|
||||
short replicationFactor = 2;
|
||||
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, partitions);
|
||||
|
||||
String output = captureDescribeTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--describe"));
|
||||
|
@ -1345,7 +1345,7 @@ public class TopicCommandTest {
|
|||
public void testDescribeAndListTopicsWithoutInternalTopics(ClusterInstance clusterInstance) throws InterruptedException {
|
||||
String testTopicName = TestUtils.randomString(10);
|
||||
try (Admin adminClient = clusterInstance.admin()) {
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
// test describe
|
||||
|
@ -1374,9 +1374,9 @@ public class TopicCommandTest {
|
|||
new ClusterAuthorizationException("Unauthorized"));
|
||||
|
||||
doReturn(result).when(adminClient).listPartitionReassignments(
|
||||
Collections.singleton(new TopicPartition(testTopicName, 0))
|
||||
Set.of(new TopicPartition(testTopicName, 0))
|
||||
);
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor)));
|
||||
clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);
|
||||
|
||||
String output = captureDescribeTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--describe", "--topic", testTopicName));
|
||||
|
@ -1395,7 +1395,7 @@ public class TopicCommandTest {
|
|||
String topic = "foo_bar";
|
||||
int partitions = 1;
|
||||
short replicationFactor = 3;
|
||||
adminClient.createTopics(Collections.singletonList(new NewTopic(topic, partitions, replicationFactor)));
|
||||
adminClient.createTopics(List.of(new NewTopic(topic, partitions, replicationFactor)));
|
||||
clusterInstance.waitTopicCreation(topic, defaultNumPartitions);
|
||||
|
||||
assertThrows(TopicExistsException.class,
|
||||
|
@ -1444,7 +1444,7 @@ public class TopicCommandTest {
|
|||
|
||||
List<Integer> partitionRackMapValueSize = partitionRackMap.values().stream()
|
||||
.map(value -> (int) value.stream().distinct().count())
|
||||
.collect(Collectors.toList());
|
||||
.toList();
|
||||
|
||||
List<Integer> expected = Collections.nCopies(numPartitions, replicationFactor);
|
||||
assertEquals(expected, partitionRackMapValueSize, "More than one replica of the same partition is assigned to the same rack");
|
||||
|
@ -1502,9 +1502,9 @@ public class TopicCommandTest {
|
|||
String rack;
|
||||
if (brokerRackMapping.containsKey(brokerId)) {
|
||||
rack = brokerRackMapping.get(brokerId);
|
||||
List<String> partitionRackValues = Stream.of(Collections.singletonList(rack), partitionRackMap.getOrDefault(partitionId, Collections.emptyList()))
|
||||
List<String> partitionRackValues = Stream.of(List.of(rack), partitionRackMap.getOrDefault(partitionId, List.of()))
|
||||
.flatMap(List::stream)
|
||||
.collect(Collectors.toList());
|
||||
.toList();
|
||||
partitionRackMap.put(partitionId, partitionRackValues);
|
||||
} else {
|
||||
System.err.printf("No mapping found for %s in `brokerRackMapping`%n", brokerId);
|
||||
|
@ -1514,18 +1514,9 @@ public class TopicCommandTest {
|
|||
return new ReplicaDistributions(partitionRackMap, leaderCount, partitionCount);
|
||||
}
|
||||
|
||||
private static class ReplicaDistributions {
|
||||
private final Map<Integer, List<String>> partitionRacks;
|
||||
private final Map<Integer, Integer> brokerLeaderCount;
|
||||
private final Map<Integer, Integer> brokerReplicasCount;
|
||||
|
||||
public ReplicaDistributions(Map<Integer, List<String>> partitionRacks,
|
||||
Map<Integer, Integer> brokerLeaderCount,
|
||||
Map<Integer, Integer> brokerReplicasCount) {
|
||||
this.partitionRacks = partitionRacks;
|
||||
this.brokerLeaderCount = brokerLeaderCount;
|
||||
this.brokerReplicasCount = brokerReplicasCount;
|
||||
}
|
||||
private record ReplicaDistributions(Map<Integer, List<String>> partitionRacks,
|
||||
Map<Integer, Integer> brokerLeaderCount,
|
||||
Map<Integer, Integer> brokerReplicasCount) {
|
||||
}
|
||||
|
||||
private KafkaProducer<String, String> createProducer(ClusterInstance clusterInstance) {
|
||||
|
|
|
@ -57,9 +57,7 @@ import java.io.IOException;
|
|||
import java.io.InputStreamReader;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -69,11 +67,6 @@ import java.util.OptionalLong;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singleton;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.KafkaFuture.completedFuture;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
@ -161,7 +154,7 @@ public class TransactionsCommandTest {
|
|||
) throws Exception {
|
||||
DescribeProducersResult describeResult = Mockito.mock(DescribeProducersResult.class);
|
||||
KafkaFuture<PartitionProducerState> describeFuture = completedFuture(
|
||||
new PartitionProducerState(asList(
|
||||
new PartitionProducerState(List.of(
|
||||
new ProducerState(12345L, 15, 1300, 1599509565L,
|
||||
OptionalInt.of(20), OptionalLong.of(990)),
|
||||
new ProducerState(98765L, 30, 2300, 1599509599L,
|
||||
|
@ -170,7 +163,7 @@ public class TransactionsCommandTest {
|
|||
|
||||
|
||||
Mockito.when(describeResult.partitionResult(topicPartition)).thenReturn(describeFuture);
|
||||
Mockito.when(admin.describeProducers(singleton(topicPartition), expectedOptions)).thenReturn(describeResult);
|
||||
Mockito.when(admin.describeProducers(Set.of(topicPartition), expectedOptions)).thenReturn(describeResult);
|
||||
|
||||
execute(args);
|
||||
assertNormalExit();
|
||||
|
@ -182,8 +175,8 @@ public class TransactionsCommandTest {
|
|||
assertEquals(expectedHeaders, table.get(0));
|
||||
|
||||
Set<List<String>> expectedRows = Set.of(
|
||||
asList("12345", "15", "20", "1300", "1599509565", "990"),
|
||||
asList("98765", "30", "-1", "2300", "1599509599", "None")
|
||||
List.of("12345", "15", "20", "1300", "1599509565", "990"),
|
||||
List.of("98765", "30", "-1", "2300", "1599509599", "None")
|
||||
);
|
||||
assertEquals(expectedRows, new HashSet<>(table.subList(1, table.size())));
|
||||
}
|
||||
|
@ -233,9 +226,9 @@ public class TransactionsCommandTest {
|
|||
assertEquals(expectedHeaders, table.get(0));
|
||||
|
||||
Set<List<String>> expectedRows = Set.of(
|
||||
asList("foo", "0", "12345", "Ongoing"),
|
||||
asList("bar", "0", "98765", "PrepareAbort"),
|
||||
asList("baz", "1", "13579", "CompleteCommit")
|
||||
List.of("foo", "0", "12345", "Ongoing"),
|
||||
List.of("bar", "0", "98765", "PrepareAbort"),
|
||||
List.of("baz", "1", "13579", "CompleteCommit")
|
||||
);
|
||||
assertEquals(expectedRows, new HashSet<>(table.subList(1, table.size())));
|
||||
}
|
||||
|
@ -302,11 +295,11 @@ public class TransactionsCommandTest {
|
|||
15,
|
||||
10000,
|
||||
OptionalLong.of(transactionStartTime),
|
||||
singleton(new TopicPartition("bar", 0))
|
||||
Set.of(new TopicPartition("bar", 0))
|
||||
));
|
||||
|
||||
Mockito.when(describeResult.description(transactionalId)).thenReturn(describeFuture);
|
||||
Mockito.when(admin.describeTransactions(singleton(transactionalId))).thenReturn(describeResult);
|
||||
Mockito.when(admin.describeTransactions(Set.of(transactionalId))).thenReturn(describeResult);
|
||||
|
||||
// Add a little time so that we can see a positive transaction duration in the output
|
||||
time.sleep(5000);
|
||||
|
@ -320,7 +313,7 @@ public class TransactionsCommandTest {
|
|||
List<String> expectedHeaders = TransactionsCommand.DescribeTransactionsCommand.HEADERS;
|
||||
assertEquals(expectedHeaders, table.get(0));
|
||||
|
||||
List<String> expectedRow = asList(
|
||||
List<String> expectedRow = List.of(
|
||||
String.valueOf(coordinatorId),
|
||||
transactionalId,
|
||||
"12345",
|
||||
|
@ -463,7 +456,7 @@ public class TransactionsCommandTest {
|
|||
|
||||
DescribeProducersResult describeResult = Mockito.mock(DescribeProducersResult.class);
|
||||
KafkaFuture<PartitionProducerState> describeFuture = completedFuture(
|
||||
new PartitionProducerState(singletonList(
|
||||
new PartitionProducerState(List.of(
|
||||
new ProducerState(producerId, producerEpoch, 1300, 1599509565L,
|
||||
OptionalInt.of(coordinatorEpoch), OptionalLong.of(startOffset))
|
||||
)));
|
||||
|
@ -474,7 +467,7 @@ public class TransactionsCommandTest {
|
|||
topicPartition, producerId, producerEpoch, coordinatorEpoch);
|
||||
|
||||
Mockito.when(describeResult.partitionResult(topicPartition)).thenReturn(describeFuture);
|
||||
Mockito.when(admin.describeProducers(singleton(topicPartition))).thenReturn(describeResult);
|
||||
Mockito.when(admin.describeProducers(Set.of(topicPartition))).thenReturn(describeResult);
|
||||
|
||||
Mockito.when(abortTransactionResult.all()).thenReturn(abortFuture);
|
||||
Mockito.when(admin.abortTransaction(expectedAbortSpec)).thenReturn(abortTransactionResult);
|
||||
|
@ -570,7 +563,7 @@ public class TransactionsCommandTest {
|
|||
OptionalInt coordinatorEpoch,
|
||||
OptionalLong txnStartOffset
|
||||
) {
|
||||
PartitionProducerState partitionProducerState = new PartitionProducerState(singletonList(
|
||||
PartitionProducerState partitionProducerState = new PartitionProducerState(List.of(
|
||||
new ProducerState(
|
||||
producerId,
|
||||
producerEpoch,
|
||||
|
@ -583,11 +576,11 @@ public class TransactionsCommandTest {
|
|||
|
||||
DescribeProducersResult result = Mockito.mock(DescribeProducersResult.class);
|
||||
Mockito.when(result.all()).thenReturn(
|
||||
completedFuture(singletonMap(topicPartition, partitionProducerState))
|
||||
completedFuture(Map.of(topicPartition, partitionProducerState))
|
||||
);
|
||||
|
||||
Mockito.when(admin.describeProducers(
|
||||
Collections.singletonList(topicPartition),
|
||||
List.of(topicPartition),
|
||||
new DescribeProducersOptions()
|
||||
)).thenReturn(result);
|
||||
}
|
||||
|
@ -596,10 +589,10 @@ public class TransactionsCommandTest {
|
|||
Map<String, TransactionDescription> descriptions
|
||||
) {
|
||||
DescribeTransactionsResult result = Mockito.mock(DescribeTransactionsResult.class);
|
||||
descriptions.forEach((transactionalId, description) -> {
|
||||
descriptions.forEach((transactionalId, description) ->
|
||||
Mockito.when(result.description(transactionalId))
|
||||
.thenReturn(completedFuture(description));
|
||||
});
|
||||
.thenReturn(completedFuture(description))
|
||||
);
|
||||
Mockito.when(result.all()).thenReturn(completedFuture(descriptions));
|
||||
Mockito.when(admin.describeTransactions(descriptions.keySet())).thenReturn(result);
|
||||
}
|
||||
|
@ -634,7 +627,7 @@ public class TransactionsCommandTest {
|
|||
};
|
||||
|
||||
String topic = "foo";
|
||||
expectListTopics(singleton(topic));
|
||||
expectListTopics(Set.of(topic));
|
||||
|
||||
Node node0 = new Node(0, "localhost", 9092);
|
||||
Node node1 = new Node(1, "localhost", 9093);
|
||||
|
@ -643,28 +636,28 @@ public class TransactionsCommandTest {
|
|||
TopicPartitionInfo partition0 = new TopicPartitionInfo(
|
||||
0,
|
||||
node0,
|
||||
Arrays.asList(node0, node1),
|
||||
Arrays.asList(node0, node1)
|
||||
List.of(node0, node1),
|
||||
List.of(node0, node1)
|
||||
);
|
||||
TopicPartitionInfo partition1 = new TopicPartitionInfo(
|
||||
1,
|
||||
node1,
|
||||
Arrays.asList(node1, node5),
|
||||
Arrays.asList(node1, node5)
|
||||
List.of(node1, node5),
|
||||
List.of(node1, node5)
|
||||
);
|
||||
|
||||
TopicDescription description = new TopicDescription(
|
||||
topic,
|
||||
false,
|
||||
Arrays.asList(partition0, partition1)
|
||||
List.of(partition0, partition1)
|
||||
);
|
||||
expectDescribeTopics(singletonMap(topic, description));
|
||||
expectDescribeTopics(Map.of(topic, description));
|
||||
|
||||
DescribeProducersResult result = Mockito.mock(DescribeProducersResult.class);
|
||||
Mockito.when(result.all()).thenReturn(completedFuture(emptyMap()));
|
||||
Mockito.when(result.all()).thenReturn(completedFuture(Map.of()));
|
||||
|
||||
Mockito.when(admin.describeProducers(
|
||||
Collections.singletonList(new TopicPartition(topic, 1)),
|
||||
List.of(new TopicPartition(topic, 1)),
|
||||
new DescribeProducersOptions().brokerId(brokerId)
|
||||
)).thenReturn(result);
|
||||
|
||||
|
@ -695,28 +688,28 @@ public class TransactionsCommandTest {
|
|||
TopicPartitionInfo partition0 = new TopicPartitionInfo(
|
||||
0,
|
||||
node0,
|
||||
Arrays.asList(node0, node1),
|
||||
Arrays.asList(node0, node1)
|
||||
List.of(node0, node1),
|
||||
List.of(node0, node1)
|
||||
);
|
||||
TopicPartitionInfo partition1 = new TopicPartitionInfo(
|
||||
1,
|
||||
node1,
|
||||
Arrays.asList(node1, node5),
|
||||
Arrays.asList(node1, node5)
|
||||
List.of(node1, node5),
|
||||
List.of(node1, node5)
|
||||
);
|
||||
|
||||
TopicDescription description = new TopicDescription(
|
||||
topic,
|
||||
false,
|
||||
Arrays.asList(partition0, partition1)
|
||||
List.of(partition0, partition1)
|
||||
);
|
||||
expectDescribeTopics(singletonMap(topic, description));
|
||||
expectDescribeTopics(Map.of(topic, description));
|
||||
|
||||
DescribeProducersResult result = Mockito.mock(DescribeProducersResult.class);
|
||||
Mockito.when(result.all()).thenReturn(completedFuture(emptyMap()));
|
||||
Mockito.when(result.all()).thenReturn(completedFuture(Map.of()));
|
||||
|
||||
Mockito.when(admin.describeProducers(
|
||||
Collections.singletonList(new TopicPartition(topic, 1)),
|
||||
List.of(new TopicPartition(topic, 1)),
|
||||
new DescribeProducersOptions().brokerId(brokerId)
|
||||
)).thenReturn(result);
|
||||
|
||||
|
@ -744,28 +737,28 @@ public class TransactionsCommandTest {
|
|||
TopicPartitionInfo partition0 = new TopicPartitionInfo(
|
||||
0,
|
||||
node0,
|
||||
Arrays.asList(node0, node1),
|
||||
Arrays.asList(node0, node1)
|
||||
List.of(node0, node1),
|
||||
List.of(node0, node1)
|
||||
);
|
||||
TopicPartitionInfo partition1 = new TopicPartitionInfo(
|
||||
1,
|
||||
node1,
|
||||
Arrays.asList(node1, node5),
|
||||
Arrays.asList(node1, node5)
|
||||
List.of(node1, node5),
|
||||
List.of(node1, node5)
|
||||
);
|
||||
|
||||
TopicDescription description = new TopicDescription(
|
||||
topic,
|
||||
false,
|
||||
Arrays.asList(partition0, partition1)
|
||||
List.of(partition0, partition1)
|
||||
);
|
||||
expectDescribeTopics(singletonMap(topic, description));
|
||||
expectDescribeTopics(Map.of(topic, description));
|
||||
|
||||
DescribeProducersResult result = Mockito.mock(DescribeProducersResult.class);
|
||||
Mockito.when(result.all()).thenReturn(completedFuture(emptyMap()));
|
||||
Mockito.when(result.all()).thenReturn(completedFuture(Map.of()));
|
||||
|
||||
Mockito.when(admin.describeProducers(
|
||||
Arrays.asList(new TopicPartition(topic, 0), new TopicPartition(topic, 1)),
|
||||
List.of(new TopicPartition(topic, 0), new TopicPartition(topic, 1)),
|
||||
new DescribeProducersOptions()
|
||||
)).thenReturn(result);
|
||||
|
||||
|
@ -851,11 +844,11 @@ public class TransactionsCommandTest {
|
|||
);
|
||||
|
||||
expectListTransactions(
|
||||
new ListTransactionsOptions().filterProducerIds(singleton(producerId)),
|
||||
singletonMap(1, Collections.emptyList())
|
||||
new ListTransactionsOptions().filterProducerIds(Set.of(producerId)),
|
||||
Map.of(1, List.of())
|
||||
);
|
||||
|
||||
expectDescribeTransactions(Collections.emptyMap());
|
||||
expectDescribeTransactions(Map.of());
|
||||
|
||||
execute(args);
|
||||
assertNormalExit();
|
||||
|
@ -907,14 +900,14 @@ public class TransactionsCommandTest {
|
|||
);
|
||||
|
||||
expectListTransactions(
|
||||
new ListTransactionsOptions().filterProducerIds(singleton(producerId)),
|
||||
singletonMap(1, Collections.singletonList(listing))
|
||||
new ListTransactionsOptions().filterProducerIds(Set.of(producerId)),
|
||||
Map.of(1, List.of(listing))
|
||||
);
|
||||
|
||||
DescribeTransactionsResult result = Mockito.mock(DescribeTransactionsResult.class);
|
||||
Mockito.when(result.description(transactionalId))
|
||||
.thenReturn(failedFuture(new TransactionalIdNotFoundException(transactionalId + " not found")));
|
||||
Mockito.when(admin.describeTransactions(singleton(transactionalId))).thenReturn(result);
|
||||
Mockito.when(admin.describeTransactions(Set.of(transactionalId))).thenReturn(result);
|
||||
|
||||
execute(args);
|
||||
assertNormalExit();
|
||||
|
@ -972,8 +965,8 @@ public class TransactionsCommandTest {
|
|||
);
|
||||
|
||||
expectListTransactions(
|
||||
new ListTransactionsOptions().filterProducerIds(singleton(producerId)),
|
||||
singletonMap(1, Collections.singletonList(listing))
|
||||
new ListTransactionsOptions().filterProducerIds(Set.of(producerId)),
|
||||
Map.of(1, List.of(listing))
|
||||
);
|
||||
|
||||
// Although there is a transaction in progress from the same
|
||||
|
@ -986,10 +979,10 @@ public class TransactionsCommandTest {
|
|||
producerEpoch,
|
||||
60000,
|
||||
OptionalLong.of(time.milliseconds()),
|
||||
singleton(new TopicPartition("foo", 10))
|
||||
Set.of(new TopicPartition("foo", 10))
|
||||
);
|
||||
|
||||
expectDescribeTransactions(singletonMap(transactionalId, description));
|
||||
expectDescribeTransactions(Map.of(transactionalId, description));
|
||||
|
||||
execute(args);
|
||||
assertNormalExit();
|
||||
|
@ -1020,7 +1013,7 @@ public class TransactionsCommandTest {
|
|||
|
||||
long durationMinutes = TimeUnit.MILLISECONDS.toMinutes(time.milliseconds() - lastTimestamp);
|
||||
|
||||
List<String> expectedRow = asList(
|
||||
List<String> expectedRow = List.of(
|
||||
topicPartition.topic(),
|
||||
String.valueOf(topicPartition.partition()),
|
||||
String.valueOf(producerId),
|
||||
|
@ -1070,8 +1063,8 @@ public class TransactionsCommandTest {
|
|||
);
|
||||
|
||||
expectListTransactions(
|
||||
new ListTransactionsOptions().filterProducerIds(singleton(producerId)),
|
||||
singletonMap(1, Collections.singletonList(listing))
|
||||
new ListTransactionsOptions().filterProducerIds(Set.of(producerId)),
|
||||
Map.of(1, List.of(listing))
|
||||
);
|
||||
|
||||
// The coordinator shows an active transaction with the same epoch
|
||||
|
@ -1084,10 +1077,10 @@ public class TransactionsCommandTest {
|
|||
producerEpoch,
|
||||
60000,
|
||||
OptionalLong.of(lastTimestamp),
|
||||
singleton(topicPartition)
|
||||
Set.of(topicPartition)
|
||||
);
|
||||
|
||||
expectDescribeTransactions(singletonMap(transactionalId, description));
|
||||
expectDescribeTransactions(Map.of(transactionalId, description));
|
||||
|
||||
execute(args);
|
||||
assertNormalExit();
|
||||
|
@ -1118,7 +1111,7 @@ public class TransactionsCommandTest {
|
|||
if (line == null) {
|
||||
return null;
|
||||
} else {
|
||||
return asList(line.split("\\s+"));
|
||||
return List.of(line.split("\\s+"));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.kafka.test.NoRetryException;
|
|||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.OptionalInt;
|
||||
|
@ -68,8 +67,8 @@ public class UserScramCredentialsCommandTest {
|
|||
throw new RuntimeException();
|
||||
});
|
||||
|
||||
List<String> commandArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", cluster.bootstrapServers()));
|
||||
commandArgs.addAll(Arrays.asList(args));
|
||||
List<String> commandArgs = new ArrayList<>(List.of("--bootstrap-server", cluster.bootstrapServers()));
|
||||
commandArgs.addAll(List.of(args));
|
||||
try {
|
||||
String output = captureStandardOut(() -> ConfigCommand.main(commandArgs.toArray(new String[0])));
|
||||
return new ConfigCommandResult(output);
|
||||
|
@ -162,7 +161,7 @@ public class UserScramCredentialsCommandTest {
|
|||
() -> {
|
||||
try {
|
||||
String output = runConfigCommandViaBroker("--entity-type", "users", "--describe").stdout;
|
||||
return Arrays.asList(msgs).contains(output);
|
||||
return List.of(msgs).contains(output);
|
||||
} catch (Exception e) {
|
||||
throw new NoRetryException(e);
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.junit.jupiter.api.Test;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
@ -283,7 +282,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
});
|
||||
|
||||
// different in all three places
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file"));
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -295,7 +294,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args));
|
||||
|
||||
// the same in all three places
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "test-group"));
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group"));
|
||||
final String[] args1 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -309,7 +308,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
assertEquals("test-group", props.getProperty("group.id"));
|
||||
|
||||
// different via --consumer-property and --consumer.config
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file"));
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args2 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -330,7 +329,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args3));
|
||||
|
||||
// different via --group and --consumer.config
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file"));
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args4 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
|
|
@ -61,14 +61,13 @@ import java.io.ByteArrayOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.time.Duration;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static java.util.Collections.singleton;
|
||||
import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
|
||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG;
|
||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG;
|
||||
|
@ -155,7 +154,7 @@ public class ConsoleConsumerTest {
|
|||
mockConsumer
|
||||
);
|
||||
|
||||
mockConsumer.rebalance(Arrays.asList(tp1, tp2));
|
||||
mockConsumer.rebalance(List.of(tp1, tp2));
|
||||
Map<TopicPartition, Long> offsets = new HashMap<>();
|
||||
offsets.put(tp1, startOffset);
|
||||
offsets.put(tp2, startOffset);
|
||||
|
@ -234,8 +233,8 @@ public class ConsoleConsumerTest {
|
|||
mockConsumer
|
||||
);
|
||||
|
||||
verify(mockConsumer).assign(eq(Collections.singletonList(tp0)));
|
||||
verify(mockConsumer).seekToEnd(eq(Collections.singletonList(tp0)));
|
||||
verify(mockConsumer).assign(eq(List.of(tp0)));
|
||||
verify(mockConsumer).seekToEnd(eq(List.of(tp0)));
|
||||
consumer.cleanup();
|
||||
reset(mockConsumer);
|
||||
|
||||
|
@ -249,7 +248,7 @@ public class ConsoleConsumerTest {
|
|||
|
||||
consumer = new ConsoleConsumer.ConsumerWrapper(new ConsoleConsumerOptions(args), mockConsumer);
|
||||
|
||||
verify(mockConsumer).assign(eq(Collections.singletonList(tp0)));
|
||||
verify(mockConsumer).assign(eq(List.of(tp0)));
|
||||
verify(mockConsumer).seek(eq(tp0), eq(123L));
|
||||
consumer.cleanup();
|
||||
reset(mockConsumer);
|
||||
|
@ -264,8 +263,8 @@ public class ConsoleConsumerTest {
|
|||
|
||||
consumer = new ConsoleConsumer.ConsumerWrapper(new ConsoleConsumerOptions(args), mockConsumer);
|
||||
|
||||
verify(mockConsumer).assign(eq(Collections.singletonList(tp0)));
|
||||
verify(mockConsumer).seekToBeginning(eq(Collections.singletonList(tp0)));
|
||||
verify(mockConsumer).assign(eq(List.of(tp0)));
|
||||
verify(mockConsumer).seekToBeginning(eq(List.of(tp0)));
|
||||
consumer.cleanup();
|
||||
reset(mockConsumer);
|
||||
}
|
||||
|
@ -295,7 +294,7 @@ public class ConsoleConsumerTest {
|
|||
try (Admin admin = cluster.admin()) {
|
||||
|
||||
NewTopic newTopic = new NewTopic(topic, 1, (short) 1);
|
||||
admin.createTopics(singleton(newTopic));
|
||||
admin.createTopics(Set.of(newTopic));
|
||||
produceMessagesWithTxn(cluster);
|
||||
|
||||
String[] transactionLogMessageFormatter = createConsoleConsumerArgs(cluster,
|
||||
|
@ -334,7 +333,7 @@ public class ConsoleConsumerTest {
|
|||
try (Admin admin = cluster.admin()) {
|
||||
|
||||
NewTopic newTopic = new NewTopic(topic, 1, (short) 1);
|
||||
admin.createTopics(singleton(newTopic));
|
||||
admin.createTopics(Set.of(newTopic));
|
||||
produceMessages(cluster);
|
||||
|
||||
String[] offsetsMessageFormatter = createConsoleConsumerArgs(cluster,
|
||||
|
@ -376,7 +375,7 @@ public class ConsoleConsumerTest {
|
|||
try (Admin admin = cluster.admin()) {
|
||||
|
||||
NewTopic newTopic = new NewTopic(topic, 1, (short) 1);
|
||||
admin.createTopics(singleton(newTopic));
|
||||
admin.createTopics(Set.of(newTopic));
|
||||
produceMessages(cluster);
|
||||
|
||||
String[] groupMetadataMessageFormatter = createConsoleConsumerArgs(cluster,
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.junit.jupiter.api.Test;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
@ -116,7 +115,7 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
});
|
||||
|
||||
// different in all three places
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file"));
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -128,7 +127,7 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args));
|
||||
|
||||
// the same in all three places
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "test-group"));
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group"));
|
||||
final String[] args1 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -142,7 +141,7 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
assertEquals("test-group", props.getProperty("group.id"));
|
||||
|
||||
// different via --consumer-property and --consumer-config
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file"));
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args2 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -163,7 +162,7 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args3));
|
||||
|
||||
// different via --group and --consumer-config
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file"));
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args4 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
|
|
@ -28,10 +28,10 @@ import org.junit.jupiter.params.provider.MethodSource;
|
|||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
|
||||
|
@ -59,7 +59,7 @@ public abstract class CoordinatorRecordMessageFormatterTest {
|
|||
);
|
||||
|
||||
try (MessageFormatter formatter = formatter()) {
|
||||
formatter.configure(emptyMap());
|
||||
formatter.configure(Map.of());
|
||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
||||
formatter.writeTo(record, new PrintStream(out));
|
||||
assertEquals(expectedOutput.replaceAll("\\s+", ""), out.toString());
|
||||
|
|
|
@ -24,10 +24,9 @@ import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
|||
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.Collections.singletonList;
|
||||
|
||||
public class GroupMetadataMessageFormatterTest extends CoordinatorRecordMessageFormatterTest {
|
||||
|
||||
private static final OffsetCommitKey OFFSET_COMMIT_KEY = new OffsetCommitKey()
|
||||
|
@ -56,7 +55,7 @@ public class GroupMetadataMessageFormatterTest extends CoordinatorRecordMessageF
|
|||
.setGeneration(1)
|
||||
.setProtocol("range")
|
||||
.setLeader("leader")
|
||||
.setMembers(singletonList(MEMBER_METADATA))
|
||||
.setMembers(List.of(MEMBER_METADATA))
|
||||
.setCurrentStateTimestamp(1234L);
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
|||
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class OffsetMessageFormatterTest extends CoordinatorRecordMessageFormatterTest {
|
||||
|
@ -46,7 +46,7 @@ public class OffsetMessageFormatterTest extends CoordinatorRecordMessageFormatte
|
|||
.setGeneration(1)
|
||||
.setProtocol("range")
|
||||
.setLeader("leader")
|
||||
.setMembers(Collections.emptyList());
|
||||
.setMembers(List.of());
|
||||
|
||||
@Override
|
||||
protected CoordinatorRecordMessageFormatter formatter() {
|
||||
|
|
|
@ -22,10 +22,9 @@ import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue;
|
|||
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.Collections.emptyList;
|
||||
|
||||
public class TransactionLogMessageFormatterTest extends CoordinatorRecordMessageFormatterTest {
|
||||
|
||||
private static final TransactionLogKey TXN_LOG_KEY = new TransactionLogKey()
|
||||
|
@ -37,7 +36,7 @@ public class TransactionLogMessageFormatterTest extends CoordinatorRecordMessage
|
|||
.setTransactionStartTimestampMs(750L)
|
||||
.setTransactionLastUpdateTimestampMs(1000L)
|
||||
.setTransactionTimeoutMs(500)
|
||||
.setTransactionPartitions(emptyList());
|
||||
.setTransactionPartitions(List.of());
|
||||
|
||||
@Override
|
||||
protected CoordinatorRecordMessageFormatter formatter() {
|
||||
|
|
|
@ -23,7 +23,8 @@ import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
|||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
import scala.jdk.javaapi.CollectionConverters;
|
||||
|
@ -36,11 +37,11 @@ import static org.junit.jupiter.api.Assertions.fail;
|
|||
public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
||||
@Test
|
||||
public void testDescribeGroupCliWithGroupDescribe() throws Exception {
|
||||
addAndVerifyAcls(CollectionConverters.asScala(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), "*", DESCRIBE, ALLOW))).toSet(), groupResource());
|
||||
addAndVerifyAcls(CollectionConverters.asScala(Set.of(new AccessControlEntry(ClientPrincipal().toString(), "*", DESCRIBE, ALLOW))).toSet(), groupResource());
|
||||
|
||||
String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()};
|
||||
ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs);
|
||||
try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap())) {
|
||||
try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Map.of())) {
|
||||
consumerGroupService.describeGroups();
|
||||
fail("Non-existent group should throw an exception");
|
||||
} catch (ExecutionException e) {
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.kafka.common.utils.Utils;
|
|||
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -55,7 +54,7 @@ class ConsumerGroupCommandTestUtils {
|
|||
String topic,
|
||||
Supplier<KafkaConsumer<T, T>> consumerSupplier) {
|
||||
return buildConsumers(numberOfConsumers, syncCommit, consumerSupplier,
|
||||
consumer -> consumer.subscribe(Collections.singleton(topic)));
|
||||
consumer -> consumer.subscribe(Set.of(topic)));
|
||||
}
|
||||
|
||||
static <T> AutoCloseable buildConsumers(
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue