diff --git a/LICENSE-binary b/LICENSE-binary index 4d8e2c76a41c8..7669f9c341aee 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -248,9 +248,9 @@ netty-transport-native-unix-common-4.1.59.Final plexus-utils-3.2.1 rocksdbjni-5.18.4 scala-collection-compat_2.13-2.3.0 -scala-library-2.13.5 +scala-library-2.13.6 scala-logging_2.13-3.9.2 -scala-reflect-2.13.5 +scala-reflect-2.13.6 scala-java8-compat_2.13-0.9.1 snappy-java-1.1.8.1 zookeeper-3.5.9 diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 834ce6ba35694..6167583780bd5 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -48,7 +48,7 @@ should_include_file() { base_dir=$(dirname $0)/.. if [ -z "$SCALA_VERSION" ]; then - SCALA_VERSION=2.13.5 + SCALA_VERSION=2.13.6 if [[ -f "$base_dir/gradle.properties" ]]; then SCALA_VERSION=`grep "^scalaVersion=" "$base_dir/gradle.properties" | cut -d= -f 2` fi diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index 5c69c102c2286..26ef84a4f5c9a 100755 --- a/bin/windows/kafka-run-class.bat +++ b/bin/windows/kafka-run-class.bat @@ -27,7 +27,7 @@ set BASE_DIR=%CD% popd IF ["%SCALA_VERSION%"] EQU [""] ( - set SCALA_VERSION=2.13.5 + set SCALA_VERSION=2.13.6 ) IF ["%SCALA_BINARY_VERSION%"] EQU [""] ( diff --git a/build.gradle b/build.gradle index 94e521b9a3e17..0b3c570ddf5ca 100644 --- a/build.gradle +++ b/build.gradle @@ -1069,6 +1069,7 @@ project(':metadata') { dependencies { implementation project(':server-common') implementation project(':clients') + implementation project(':raft') implementation libs.jacksonDatabind implementation libs.jacksonJDK8Datatypes implementation libs.metrics @@ -1077,6 +1078,7 @@ project(':metadata') { testImplementation libs.hamcrest testImplementation libs.slf4jlog4j testImplementation project(':clients').sourceSets.test.output + testImplementation project(':raft').sourceSets.test.output } task processMessages(type:JavaExec) { @@ -1267,7 +1269,6 @@ project(':raft') { dependencies { implementation project(':server-common') implementation project(':clients') - implementation project(':metadata') implementation libs.slf4jApi implementation libs.jacksonDatabind diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index c98cfab787d9f..27d8f3e0e7979 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -217,12 +217,15 @@ + + + @@ -233,6 +236,9 @@ + + + @@ -242,6 +248,8 @@ + + @@ -291,7 +299,6 @@ - @@ -417,7 +424,6 @@ - diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 92800d2315c5b..baceaba62049b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1089,8 +1089,8 @@ public void handleSuccessfulResponse(RequestHeader requestHeader, long now, Meta if (!errors.isEmpty()) log.warn("Error while fetching metadata with correlation id {} : {}", requestHeader.correlationId(), errors); - // Don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being - // created which means we will get errors and no nodes until it exists + // When talking to the startup phase of a broker, it is possible to receive an empty metadata set, which + // we should retry later. if (response.brokers().isEmpty()) { log.trace("Ignoring empty metadata response with correlation id {}.", requestHeader.correlationId()); this.metadata.failedUpdate(now); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index e50d9f4d5f3d9..ef8a9cc4fd68b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -524,19 +524,13 @@ private TransactionManager configureTransactionState(ProducerConfig config, final String transactionalId = config.getString(ProducerConfig.TRANSACTIONAL_ID_CONFIG); final int transactionTimeoutMs = config.getInt(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); final long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG); - final boolean autoDowngradeTxnCommit = config.getBoolean(ProducerConfig.AUTO_DOWNGRADE_TXN_COMMIT); - // Only log a warning if being used outside of Streams, which we know includes "StreamThread-" in the client id - if (autoDowngradeTxnCommit && !clientId.contains("StreamThread-")) { - log.warn("The configuration parameter `{}` is internal and not intended for public use, it will be " + - "removed in 4.0", ProducerConfig.AUTO_DOWNGRADE_TXN_COMMIT); - } transactionManager = new TransactionManager( logContext, transactionalId, transactionTimeoutMs, retryBackoffMs, - apiVersions, - autoDowngradeTxnCommit); + apiVersions + ); if (transactionManager.isTransactional()) log.info("Instantiated a transactional producer."); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 811b630dcd596..4fd540dceaa8a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -179,7 +179,7 @@ public void beginTransaction() throws ProducerFencedException { this.sentOffsets = false; } - @SuppressWarnings("deprecation") + @Deprecated @Override public void sendOffsetsToTransaction(Map offsets, String consumerGroupId) throws ProducerFencedException { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java index 1baeb0f648ba7..13eaa5aaea9af 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java @@ -24,7 +24,6 @@ /** * Partitioner Interface */ - public interface Partitioner extends Configurable, Closeable { /** @@ -37,13 +36,12 @@ public interface Partitioner extends Configurable, Closeable { * @param valueBytes The serialized value to partition on or null * @param cluster The current cluster metadata */ - public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster); + int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster); /** * This is called when partitioner is closed. */ - public void close(); - + void close(); /** * Notifies the partitioner a new batch is about to be created. When using the sticky partitioner, @@ -52,6 +50,6 @@ public interface Partitioner extends Configurable, Closeable { * @param cluster The current cluster metadata * @param prevPartition The partition previously selected for the record that triggered a new batch */ - default public void onNewBatch(String topic, Cluster cluster, int prevPartition) { + default void onNewBatch(String topic, Cluster cluster, int prevPartition) { } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index dcbb6d22199b2..300f5a3bd5f64 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -200,8 +200,8 @@ public class ProducerConfig extends AbstractConfig { /** max.in.flight.requests.per.connection */ public static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "max.in.flight.requests.per.connection"; private static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC = "The maximum number of unacknowledged requests the client will send on a single connection before blocking." - + " Note that if this setting is set to be greater than 1 and there are failed sends, there is a risk of" - + " message re-ordering due to retries (i.e., if retries are enabled)."; + + " Note that if this config is set to be greater than 1 and enable.idempotence is set to false, there is a risk of" + + " message re-ordering after a failed send due to retries (i.e., if retries are enabled)."; /** retries */ public static final String RETRIES_CONFIG = CommonClientConfigs.RETRIES_CONFIG; @@ -246,10 +246,10 @@ public class ProducerConfig extends AbstractConfig { public static final String ENABLE_IDEMPOTENCE_CONFIG = "enable.idempotence"; public static final String ENABLE_IDEMPOTENCE_DOC = "When set to 'true', the producer will ensure that exactly one copy of each message is written in the stream. If 'false', producer " + "retries due to broker failures, etc., may write duplicates of the retried message in the stream. " - + "Note that enabling idempotence requires " + MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION + " to be less than or equal to 5, " - + "" + RETRIES_CONFIG + " to be greater than 0 and " + ACKS_CONFIG + " must be 'all'. If these values " - + "are not explicitly set by the user, suitable values will be chosen. If incompatible values are set, " - + "a ConfigException will be thrown."; + + "Note that enabling idempotence requires " + MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION + " to be less than or equal to 5 " + + "(with message ordering preserved for any allowable value), " + RETRIES_CONFIG + " to be greater than 0, and " + + ACKS_CONFIG + " must be 'all'. If these values are not explicitly set by the user, suitable values will be chosen. If incompatible " + + "values are set, a ConfigException will be thrown."; /** transaction.timeout.ms */ public static final String TRANSACTION_TIMEOUT_CONFIG = "transaction.timeout.ms"; @@ -269,21 +269,6 @@ public class ProducerConfig extends AbstractConfig { public static final String SECURITY_PROVIDERS_CONFIG = SecurityConfig.SECURITY_PROVIDERS_CONFIG; private static final String SECURITY_PROVIDERS_DOC = SecurityConfig.SECURITY_PROVIDERS_DOC; - /** - * internal.auto.downgrade.txn.commit - * Whether or not the producer should automatically downgrade the transactional commit request when the new group metadata - * feature is not supported by the broker. - *

- * The purpose of this flag is to make Kafka Streams being capable of working with old brokers when applying this new API. - * Non Kafka Streams users who are building their own EOS applications should be careful playing around - * with config as there is a risk of violating EOS semantics when turning on this flag. - * - *

- * Note: this is an internal configuration and could be changed in the future in a backward incompatible way - * - */ - static final String AUTO_DOWNGRADE_TXN_COMMIT = "internal.auto.downgrade.txn.commit"; - private static final AtomicInteger PRODUCER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); static { @@ -424,11 +409,7 @@ public class ProducerConfig extends AbstractConfig { null, new ConfigDef.NonEmptyString(), Importance.LOW, - TRANSACTIONAL_ID_DOC) - .defineInternal(AUTO_DOWNGRADE_TXN_COMMIT, - Type.BOOLEAN, - false, - Importance.LOW); + TRANSACTIONAL_ID_DOC); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/IncompleteBatches.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/IncompleteBatches.java index 93b95966e2b79..00f8197a12af3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/IncompleteBatches.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/IncompleteBatches.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.Set; +import java.util.stream.Collectors; /* * A thread-safe helper class to hold batches that haven't been acknowledged yet (including those @@ -51,6 +52,12 @@ public Iterable copyAll() { } } + public Iterable requestResults() { + synchronized (incomplete) { + return incomplete.stream().map(batch -> batch.produceFuture).collect(Collectors.toList()); + } + } + public boolean isEmpty() { synchronized (incomplete) { return incomplete.isEmpty(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 1bc4bb0cf88bf..e51fad722c376 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -710,8 +710,12 @@ private boolean appendsInProgress() { */ public void awaitFlushCompletion() throws InterruptedException { try { - for (ProducerBatch batch : this.incomplete.copyAll()) - batch.produceFuture.await(); + // Obtain a copy of all of the incomplete ProduceRequestResult(s) at the time of the flush. + // We must be careful not to hold a reference to the ProduceBatch(s) so that garbage + // collection can occur on the contents. + // The sender will remove ProducerBatch(s) from the original incomplete collection. + for (ProduceRequestResult result : this.incomplete.requestResults()) + result.await(); } finally { this.flushesInProgress.decrementAndGet(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index d1f6e68c6536a..0ac9ac57da120 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -97,7 +97,6 @@ public class TransactionManager { private final String transactionalId; private final int transactionTimeoutMs; private final ApiVersions apiVersions; - private final boolean autoDowngradeTxnCommit; private static class TopicPartitionBookkeeper { @@ -304,8 +303,7 @@ public TransactionManager(final LogContext logContext, final String transactionalId, final int transactionTimeoutMs, final long retryBackoffMs, - final ApiVersions apiVersions, - final boolean autoDowngradeTxnCommit) { + final ApiVersions apiVersions) { this.producerIdAndEpoch = ProducerIdAndEpoch.NONE; this.transactionalId = transactionalId; this.log = logContext.logger(TransactionManager.class); @@ -322,7 +320,6 @@ public TransactionManager(final LogContext logContext, this.retryBackoffMs = retryBackoffMs; this.topicPartitionBookkeeper = new TopicPartitionBookkeeper(); this.apiVersions = apiVersions; - this.autoDowngradeTxnCommit = autoDowngradeTxnCommit; } public synchronized TransactionalRequestResult initializeTransactions() { @@ -1179,8 +1176,7 @@ private TxnOffsetCommitHandler txnOffsetCommitHandler(TransactionalRequestResult pendingTxnOffsetCommits, groupMetadata.memberId(), groupMetadata.generationId(), - groupMetadata.groupInstanceId(), - autoDowngradeTxnCommit + groupMetadata.groupInstanceId() ); return new TxnOffsetCommitHandler(result, builder); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 156e08fd7123f..4fd695499cd10 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -802,7 +802,11 @@ else if (value instanceof Class) * The config types */ public enum Type { - BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD + BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD; + + public boolean isSensitive() { + return this == PASSWORD; + } } /** diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 8ec2d02ea3de6..bd28aa26aa977 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -107,7 +107,8 @@ public enum ApiKeys { BROKER_HEARTBEAT(ApiMessageType.BROKER_HEARTBEAT, true, RecordBatch.MAGIC_VALUE_V0, false), UNREGISTER_BROKER(ApiMessageType.UNREGISTER_BROKER, false, RecordBatch.MAGIC_VALUE_V0, true), DESCRIBE_TRANSACTIONS(ApiMessageType.DESCRIBE_TRANSACTIONS), - LIST_TRANSACTIONS(ApiMessageType.LIST_TRANSACTIONS); + LIST_TRANSACTIONS(ApiMessageType.LIST_TRANSACTIONS), + ALLOCATE_PRODUCER_IDS(ApiMessageType.ALLOCATE_PRODUCER_IDS, true, false); private static final Map> APIS_BY_LISTENER = new EnumMap<>(ApiMessageType.ListenerType.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 3137e3e500aef..0c38e998fe247 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -301,6 +301,8 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion, return DescribeTransactionsRequest.parse(buffer, apiVersion); case LIST_TRANSACTIONS: return ListTransactionsRequest.parse(buffer, apiVersion); + case ALLOCATE_PRODUCER_IDS: + return AllocateProducerIdsRequest.parse(buffer, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 5f7b88f269405..47f2b3c7f3099 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -245,6 +245,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response return DescribeTransactionsResponse.parse(responseBuffer, version); case LIST_TRANSACTIONS: return ListTransactionsResponse.parse(responseBuffer, version); + case ALLOCATE_PRODUCER_IDS: + return AllocateProducerIdsResponse.parse(responseBuffer, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsRequest.java new file mode 100644 index 0000000000000..7938f92df56d9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsRequest.java @@ -0,0 +1,72 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.message.AllocateProducerIdsRequestData; +import org.apache.kafka.common.message.AllocateProducerIdsResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; + +public class AllocateProducerIdsRequest extends AbstractRequest { + private final AllocateProducerIdsRequestData data; + + public AllocateProducerIdsRequest(AllocateProducerIdsRequestData data, short version) { + super(ApiKeys.ALLOCATE_PRODUCER_IDS, version); + this.data = data; + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + return new AllocateProducerIdsResponse(new AllocateProducerIdsResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(Errors.forException(e).code())); + } + + @Override + public AllocateProducerIdsRequestData data() { + return data; + } + + public static class Builder extends AbstractRequest.Builder { + + private final AllocateProducerIdsRequestData data; + + public Builder(AllocateProducerIdsRequestData data) { + super(ApiKeys.ALLOCATE_PRODUCER_IDS); + this.data = data; + } + + @Override + public AllocateProducerIdsRequest build(short version) { + return new AllocateProducerIdsRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + public static AllocateProducerIdsRequest parse(ByteBuffer buffer, short version) { + return new AllocateProducerIdsRequest(new AllocateProducerIdsRequestData( + new ByteBufferAccessor(buffer), version), version); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java new file mode 100644 index 0000000000000..5d48c39e8019d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java @@ -0,0 +1,63 @@ +/* + * 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.common.requests; + +import org.apache.kafka.common.message.AllocateProducerIdsResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Map; + +public class AllocateProducerIdsResponse extends AbstractResponse { + + private final AllocateProducerIdsResponseData data; + + public AllocateProducerIdsResponse(AllocateProducerIdsResponseData data) { + super(ApiKeys.ALLOCATE_PRODUCER_IDS); + this.data = data; + } + + @Override + public AllocateProducerIdsResponseData data() { + return data; + } + + /** + * The number of each type of error in the response, including {@link Errors#NONE} and top-level errors as well as + * more specifically scoped errors (such as topic or partition-level errors). + * + * @return A count of errors. + */ + @Override + public Map errorCounts() { + return Collections.singletonMap(Errors.forCode(data.errorCode()), 1); + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + public static AllocateProducerIdsResponse parse(ByteBuffer buffer, short version) { + return new AllocateProducerIdsResponse(new AllocateProducerIdsResponseData( + new ByteBufferAccessor(buffer), version)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java index e96f81ae7e2f9..27b13ba220251 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java @@ -28,8 +28,6 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.RecordBatch; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -42,22 +40,18 @@ public class TxnOffsetCommitRequest extends AbstractRequest { - private static final Logger log = LoggerFactory.getLogger(TxnOffsetCommitRequest.class); - private final TxnOffsetCommitRequestData data; public static class Builder extends AbstractRequest.Builder { public final TxnOffsetCommitRequestData data; - private final boolean autoDowngrade; public Builder(final String transactionalId, final String consumerGroupId, final long producerId, final short producerEpoch, - final Map pendingTxnOffsetCommits, - final boolean autoDowngrade) { + final Map pendingTxnOffsetCommits) { this(transactionalId, consumerGroupId, producerId, @@ -65,8 +59,7 @@ public Builder(final String transactionalId, pendingTxnOffsetCommits, JoinGroupRequest.UNKNOWN_MEMBER_ID, JoinGroupRequest.UNKNOWN_GENERATION_ID, - Optional.empty(), - autoDowngrade); + Optional.empty()); } public Builder(final String transactionalId, @@ -76,8 +69,7 @@ public Builder(final String transactionalId, final Map pendingTxnOffsetCommits, final String memberId, final int generationId, - final Optional groupInstanceId, - final boolean autoDowngrade) { + final Optional groupInstanceId) { super(ApiKeys.TXN_OFFSET_COMMIT); this.data = new TxnOffsetCommitRequestData() .setTransactionalId(transactionalId) @@ -88,26 +80,13 @@ public Builder(final String transactionalId, .setMemberId(memberId) .setGenerationId(generationId) .setGroupInstanceId(groupInstanceId.orElse(null)); - this.autoDowngrade = autoDowngrade; } @Override public TxnOffsetCommitRequest build(short version) { if (version < 3 && groupMetadataSet()) { - if (autoDowngrade) { - log.trace("Downgrade the request by resetting group metadata fields: " + - "[member.id:{}, generation.id:{}, group.instance.id:{}], because broker " + - "only supports TxnOffsetCommit version {}. Need " + - "v3 or newer to enable this feature", - data.memberId(), data.generationId(), data.groupInstanceId(), version); - - data.setGenerationId(JoinGroupRequest.UNKNOWN_GENERATION_ID) - .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID) - .setGroupInstanceId(null); - } else { - throw new UnsupportedVersionException("Broker unexpectedly " + - "doesn't support group metadata commit API on version " + version); - } + throw new UnsupportedVersionException("Broker doesn't support group metadata commit API on version " + version + + ", minimum supported request version is 3 which requires brokers to be on version 2.5 or above."); } return new TxnOffsetCommitRequest(data, version); } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ConfigUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/ConfigUtils.java index 504a1f06431d0..0f839ffa962c5 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/ConfigUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/ConfigUtils.java @@ -17,10 +17,15 @@ package org.apache.kafka.common.utils; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.ConfigKey; +import org.apache.kafka.common.config.ConfigDef.Type; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -113,4 +118,30 @@ public static Map translateDeprecatedConfigs(Map confi return newConfigs; } + + public static String configMapToRedactedString(Map map, ConfigDef configDef) { + StringBuilder bld = new StringBuilder("{"); + List keys = new ArrayList<>(map.keySet()); + Collections.sort(keys); + String prefix = ""; + for (String key : keys) { + bld.append(prefix).append(key).append("="); + ConfigKey configKey = configDef.configKeys().get(key); + if (configKey == null || configKey.type().isSensitive()) { + bld.append("(redacted)"); + } else { + Object value = map.get(key); + if (value == null) { + bld.append("null"); + } else if (configKey.type() == Type.STRING) { + bld.append("\"").append(value).append("\""); + } else { + bld.append(value); + } + } + prefix = ", "; + } + bld.append("}"); + return bld.toString(); + } } diff --git a/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json b/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json new file mode 100644 index 0000000000000..0cfa494291a36 --- /dev/null +++ b/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json @@ -0,0 +1,29 @@ +// 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 implie +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 67, + "type": "request", + "listeners": ["controller", "zkBroker"], + "name": "AllocateProducerIdsRequest", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the requesting broker" }, + { "name": "BrokerEpoch", "type": "int64", "versions": "0+", "default": "-1", + "about": "The epoch of the requesting broker" } + ] +} diff --git a/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json b/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json new file mode 100644 index 0000000000000..0d849c098568b --- /dev/null +++ b/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json @@ -0,0 +1,32 @@ +// 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. + +{ + "apiKey": 67, + "type": "response", + "name": "AllocateProducerIdsResponse", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The top level response error code" }, + { "name": "ProducerIdStart", "type": "int64", "versions": "0+", "entityType": "producerId", + "about": "The first producer ID in this range, inclusive"}, + { "name": "ProducerIdLen", "type": "int32", "versions": "0+", + "about": "The number of producer IDs in this range"} + ] +} diff --git a/clients/src/main/resources/common/message/CreateAclsRequest.json b/clients/src/main/resources/common/message/CreateAclsRequest.json index 109b444c79d66..5b3bfed78162c 100644 --- a/clients/src/main/resources/common/message/CreateAclsRequest.json +++ b/clients/src/main/resources/common/message/CreateAclsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 30, "type": "request", - "listeners": ["zkBroker"], + "listeners": ["zkBroker", "broker", "controller"], "name": "CreateAclsRequest", // Version 1 adds resource pattern type. // Version 2 enables flexible versions. diff --git a/clients/src/main/resources/common/message/DeleteAclsRequest.json b/clients/src/main/resources/common/message/DeleteAclsRequest.json index 1e7aa9a1b1421..fd7c1522b43bd 100644 --- a/clients/src/main/resources/common/message/DeleteAclsRequest.json +++ b/clients/src/main/resources/common/message/DeleteAclsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 31, "type": "request", - "listeners": ["zkBroker"], + "listeners": ["zkBroker", "broker", "controller"], "name": "DeleteAclsRequest", // Version 1 adds the pattern type. // Version 2 enables flexible versions. diff --git a/clients/src/main/resources/common/message/DescribeAclsRequest.json b/clients/src/main/resources/common/message/DescribeAclsRequest.json index e551ca45be0a8..58886da654707 100644 --- a/clients/src/main/resources/common/message/DescribeAclsRequest.json +++ b/clients/src/main/resources/common/message/DescribeAclsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 29, "type": "request", - "listeners": ["zkBroker"], + "listeners": ["zkBroker", "broker", "controller"], "name": "DescribeAclsRequest", // Version 1 adds resource pattern type. // Version 2 enables flexible versions. diff --git a/clients/src/main/resources/common/message/DescribeProducersRequest.json b/clients/src/main/resources/common/message/DescribeProducersRequest.json index 0ffd834e6bde8..0e3813bb02014 100644 --- a/clients/src/main/resources/common/message/DescribeProducersRequest.json +++ b/clients/src/main/resources/common/message/DescribeProducersRequest.json @@ -21,12 +21,11 @@ "validVersions": "0", "flexibleVersions": "0+", "fields": [ - { "name": "Topics", "type": "[]TopicRequest", "versions": "0+", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", - "about": "The topic name." }, - { "name": "PartitionIndexes", "type": "[]int32", "versions": "0+", - "about": "The indexes of the partitions to list producers for." } - ]} - ]} + { "name": "Topics", "type": "[]TopicRequest", "versions": "0+", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "PartitionIndexes", "type": "[]int32", "versions": "0+", + "about": "The indexes of the partitions to list producers for." } + ]} ] } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index e3d4e944cee78..4f44dfbf01da1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -907,20 +907,11 @@ public void testSendTxnOffsetsWithGroupId() { @Test public void testSendTxnOffsetsWithGroupMetadata() { - sendOffsetsWithGroupMetadata((short) 3); - } - - @Test - public void testSendTxnOffsetsWithGroupMetadataDowngrade() { - sendOffsetsWithGroupMetadata((short) 2); - } - - private void sendOffsetsWithGroupMetadata(final short maxVersion) { + final short maxVersion = (short) 3; Map configs = new HashMap<>(); configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "some.id"); configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); - configs.put(ProducerConfig.AUTO_DOWNGRADE_TXN_COMMIT, true); Time time = new MockTime(1); MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1)); @@ -943,23 +934,16 @@ private void sendOffsetsWithGroupMetadata(final short maxVersion) { String groupInstanceId = "instance"; client.prepareResponse(request -> { TxnOffsetCommitRequestData data = ((TxnOffsetCommitRequest) request).data(); - if (maxVersion < 3) { - return data.groupId().equals(groupId) && - data.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID) && - data.generationId() == JoinGroupRequest.UNKNOWN_GENERATION_ID && - data.groupInstanceId() == null; - } else { - return data.groupId().equals(groupId) && - data.memberId().equals(memberId) && - data.generationId() == generationId && - data.groupInstanceId().equals(groupInstanceId); - } + return data.groupId().equals(groupId) && + data.memberId().equals(memberId) && + data.generationId() == generationId && + data.groupInstanceId().equals(groupInstanceId); }, txnOffsetsCommitResponse(Collections.singletonMap( new TopicPartition("topic", 0), Errors.NONE))); client.prepareResponse(endTxnResponse(Errors.NONE)); try (Producer producer = kafkaProducer(configs, new StringSerializer(), - new StringSerializer(), metadata, client, null, time)) { + new StringSerializer(), metadata, client, null, time)) { producer.initTransactions(); producer.beginTransaction(); ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata(groupId, diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 16a3d692de12d..ca14ab0fda3da 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -417,6 +417,7 @@ public void shouldPublishConsumerGroupOffsetsOnlyAfterCommitIfTransactionsAreEna assertEquals(Collections.singletonList(expectedResult), producer.consumerGroupOffsetsHistory()); } + @Deprecated @Test public void shouldThrowOnNullConsumerGroupIdWhenSendOffsetsToTransaction() { buildMockProducer(true); @@ -433,6 +434,7 @@ public void shouldThrowOnNullConsumerGroupMetadataWhenSendOffsetsToTransaction() assertThrows(NullPointerException.class, () -> producer.sendOffsetsToTransaction(Collections.emptyMap(), new ConsumerGroupMetadata(null))); } + @Deprecated @Test public void shouldIgnoreEmptyOffsetsWhenSendOffsetsToTransactionByGroupId() { buildMockProducer(true); @@ -451,6 +453,7 @@ public void shouldIgnoreEmptyOffsetsWhenSendOffsetsToTransactionByGroupMetadata( assertFalse(producer.sentOffsets()); } + @Deprecated @Test public void shouldAddOffsetsWhenSendOffsetsToTransactionByGroupId() { buildMockProducer(true); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 9cc350abc17fb..db4454da89cc3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -706,7 +706,7 @@ public void testIdempotenceWithOldMagic() throws InterruptedException { String metricGrpName = "producer-metrics"; apiVersions.update("foobar", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2)); - TransactionManager transactionManager = new TransactionManager(new LogContext(), null, 0, 100L, new ApiVersions(), false); + TransactionManager transactionManager = new TransactionManager(new LogContext(), null, 0, 100L, new ApiVersions()); RecordAccumulator accum = new RecordAccumulator(logContext, batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, CompressionType.NONE, lingerMs, retryBackoffMs, deliveryTimeoutMs, metrics, metricGrpName, time, apiVersions, transactionManager, new BufferPool(totalSize, batchSize, metrics, time, metricGrpName)); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 52e801f87b3c6..26041ccf4694d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -569,7 +569,7 @@ public void testInitProducerIdWithMaxInFlightOne() throws Exception { // Initialize transaction manager. InitProducerId will be queued up until metadata response // is processed and FindCoordinator can be sent to `leastLoadedNode`. TransactionManager transactionManager = new TransactionManager(new LogContext(), "testInitProducerIdWithPendingMetadataRequest", - 60000, 100L, new ApiVersions(), false); + 60000, 100L, new ApiVersions()); setupWithTransactionState(transactionManager, false, null, false); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(producerId, (short) 0); transactionManager.initializeTransactions(); @@ -615,13 +615,13 @@ public void testIdempotentInitProducerIdWithMaxInFlightOne() throws Exception { * is not ready. */ @Test - public void testNodeNotReady() throws Exception { + public void testNodeNotReady() { final long producerId = 123456L; time = new MockTime(10); client = new MockClient(time, metadata); TransactionManager transactionManager = new TransactionManager(new LogContext(), "testNodeNotReady", - 60000, 100L, new ApiVersions(), false); + 60000, 100L, new ApiVersions()); setupWithTransactionState(transactionManager, false, null, true); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(producerId, (short) 0); transactionManager.initializeTransactions(); @@ -1458,7 +1458,7 @@ public void testExpiryOfFirstBatchShouldCauseEpochBumpIfFutureBatchesFail() thro public void testUnresolvedSequencesAreNotFatal() throws Exception { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); apiVersions.update("0", NodeApiVersions.create(ApiKeys.INIT_PRODUCER_ID.id, (short) 0, (short) 3)); - TransactionManager txnManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions, false); + TransactionManager txnManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -1744,7 +1744,7 @@ public void testCorrectHandlingOfDuplicateSequenceError() throws Exception { @Test public void testTransactionalUnknownProducerHandlingWhenRetentionLimitReached() throws Exception { final long producerId = 343434L; - TransactionManager transactionManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions, false); + TransactionManager transactionManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions); setupWithTransactionState(transactionManager); doInitTransactions(transactionManager, new ProducerIdAndEpoch(producerId, (short) 0)); @@ -2300,7 +2300,7 @@ public void testIdempotentSplitBatchAndSend() throws Exception { public void testTransactionalSplitBatchAndSend() throws Exception { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); TopicPartition tp = new TopicPartition("testSplitBatchAndSend", 1); - TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions, false); + TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -2642,7 +2642,7 @@ public void testTransactionalRequestsSentOnShutdown() { Metrics m = new Metrics(); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); try { - TransactionManager txnManager = new TransactionManager(logContext, "testTransactionalRequestsSentOnShutdown", 6000, 100, apiVersions, false); + TransactionManager txnManager = new TransactionManager(logContext, "testTransactionalRequestsSentOnShutdown", 6000, 100, apiVersions); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, txnManager, apiVersions); @@ -2677,7 +2677,7 @@ public void testIncompleteTransactionAbortOnShutdown() { Metrics m = new Metrics(); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); try { - TransactionManager txnManager = new TransactionManager(logContext, "testIncompleteTransactionAbortOnShutdown", 6000, 100, apiVersions, false); + TransactionManager txnManager = new TransactionManager(logContext, "testIncompleteTransactionAbortOnShutdown", 6000, 100, apiVersions); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, txnManager, apiVersions); @@ -2712,7 +2712,7 @@ public void testForceShutdownWithIncompleteTransaction() { Metrics m = new Metrics(); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); try { - TransactionManager txnManager = new TransactionManager(logContext, "testForceShutdownWithIncompleteTransaction", 6000, 100, apiVersions, false); + TransactionManager txnManager = new TransactionManager(logContext, "testForceShutdownWithIncompleteTransaction", 6000, 100, apiVersions); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, txnManager, apiVersions); @@ -2743,7 +2743,7 @@ public void testForceShutdownWithIncompleteTransaction() { @Test public void testTransactionAbortedExceptionOnAbortWithoutError() throws InterruptedException, ExecutionException { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); - TransactionManager txnManager = new TransactionManager(logContext, "testTransactionAbortedExceptionOnAbortWithoutError", 60000, 100, apiVersions, false); + TransactionManager txnManager = new TransactionManager(logContext, "testTransactionAbortedExceptionOnAbortWithoutError", 60000, 100, apiVersions); setupWithTransactionState(txnManager, false, null); doInitTransactions(txnManager, producerIdAndEpoch); @@ -2769,7 +2769,7 @@ public void testTransactionAbortedExceptionOnAbortWithoutError() throws Interrup public void testDoNotPollWhenNoRequestSent() { client = spy(new MockClient(time, metadata)); - TransactionManager txnManager = new TransactionManager(logContext, "testDoNotPollWhenNoRequestSent", 6000, 100, apiVersions, false); + TransactionManager txnManager = new TransactionManager(logContext, "testDoNotPollWhenNoRequestSent", 6000, 100, apiVersions); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -2781,7 +2781,7 @@ public void testDoNotPollWhenNoRequestSent() { @Test public void testTooLargeBatchesAreSafelyRemoved() throws InterruptedException { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); - TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions, false); + TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions); setupWithTransactionState(txnManager, false, null); doInitTransactions(txnManager, producerIdAndEpoch); @@ -2980,7 +2980,7 @@ private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors e } private TransactionManager createTransactionManager() { - return new TransactionManager(new LogContext(), null, 0, 100L, new ApiVersions(), false); + return new TransactionManager(new LogContext(), null, 0, 100L, new ApiVersions()); } private void setupWithTransactionState(TransactionManager transactionManager) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index a14d29522cb18..c5e3466d138aa 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -146,10 +146,10 @@ public void setup() { this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap("test", 2))); this.brokerNode = new Node(0, "localhost", 2211); - initializeTransactionManager(Optional.of(transactionalId), false); + initializeTransactionManager(Optional.of(transactionalId)); } - private void initializeTransactionManager(Optional transactionalId, boolean autoDowngrade) { + private void initializeTransactionManager(Optional transactionalId) { Metrics metrics = new Metrics(time); apiVersions.update("0", new NodeApiVersions(Arrays.asList( @@ -162,7 +162,7 @@ private void initializeTransactionManager(Optional transactionalId, bool .setMinVersion((short) 0) .setMaxVersion((short) 7)))); this.transactionManager = new TransactionManager(logContext, transactionalId.orElse(null), - transactionTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, apiVersions, autoDowngrade); + transactionTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, apiVersions); int batchSize = 16 * 1024; int deliveryTimeoutMs = 3000; @@ -223,13 +223,13 @@ public void testFailIfNotReadyForSendNoProducerId() { @Test public void testFailIfNotReadyForSendIdempotentProducer() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); transactionManager.failIfNotReadyForSend(); } @Test public void testFailIfNotReadyForSendIdempotentProducerFatalError() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); transactionManager.transitionToFatalError(new KafkaException()); assertThrows(KafkaException.class, () -> transactionManager.failIfNotReadyForSend()); } @@ -573,7 +573,7 @@ public void testIsSendToPartitionAllowedWithPartitionNotAdded() { @Test public void testDefaultSequenceNumber() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); assertEquals((int) transactionManager.sequenceNumber(tp0), 0); transactionManager.incrementSequenceNumber(tp0, 3); assertEquals((int) transactionManager.sequenceNumber(tp0), 3); @@ -581,7 +581,7 @@ public void testDefaultSequenceNumber() { @Test public void testBumpEpochAndResetSequenceNumbersAfterUnknownProducerId() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); initializeIdempotentProducerId(producerId, epoch); ProducerBatch b1 = writeIdempotentBatchWithValue(transactionManager, tp0, "1"); @@ -618,7 +618,7 @@ public void testBatchFailureAfterProducerReset() { // The partition(s) that triggered the reset will have their sequence number reset, while any others will not final short epoch = Short.MAX_VALUE; - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); initializeIdempotentProducerId(producerId, epoch); ProducerBatch tp0b1 = writeIdempotentBatchWithValue(transactionManager, tp0, "1"); @@ -657,7 +657,7 @@ public void testBatchFailureAfterProducerReset() { public void testBatchCompletedAfterProducerReset() { final short epoch = Short.MAX_VALUE; - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); initializeIdempotentProducerId(producerId, epoch); ProducerBatch b1 = writeIdempotentBatchWithValue(transactionManager, tp0, "1"); @@ -715,7 +715,7 @@ private ProducerBatch batchWithValue(TopicPartition tp, String value) { @Test public void testSequenceNumberOverflow() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); assertEquals((int) transactionManager.sequenceNumber(tp0), 0); transactionManager.incrementSequenceNumber(tp0, Integer.MAX_VALUE); assertEquals((int) transactionManager.sequenceNumber(tp0), Integer.MAX_VALUE); @@ -727,7 +727,7 @@ public void testSequenceNumberOverflow() { @Test public void testProducerIdReset() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); initializeIdempotentProducerId(15L, Short.MAX_VALUE); assertEquals((int) transactionManager.sequenceNumber(tp0), 0); assertEquals((int) transactionManager.sequenceNumber(tp1), 0); @@ -2219,32 +2219,6 @@ public void testSendOffsetWithGroupMetadataFailAsAutoDowngradeTxnCommitNotEnable assertThrows(UnsupportedVersionException.class, () -> sender.runOnce()); } - @Test - public void testSendOffsetWithGroupMetadataSuccessAsAutoDowngradeTxnCommitEnabled() { - initializeTransactionManager(Optional.of(transactionalId), true); - - client.setNodeApiVersions(NodeApiVersions.create(ApiKeys.TXN_OFFSET_COMMIT.id, (short) 0, (short) 2)); - - Map txnOffsetCommitResponse = new HashMap<>(); - txnOffsetCommitResponse.put(tp0, Errors.NONE); - txnOffsetCommitResponse.put(tp1, Errors.COORDINATOR_LOAD_IN_PROGRESS); - - TransactionalRequestResult addOffsetsResult = prepareGroupMetadataCommit( - () -> prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, txnOffsetCommitResponse)); - - sender.runOnce(); // Send TxnOffsetCommitRequest request. - - assertTrue(transactionManager.hasPendingOffsetCommits()); // The TxnOffsetCommit failed. - assertFalse(addOffsetsResult.isCompleted()); // We should only be done after both RPCs complete successfully. - - txnOffsetCommitResponse.put(tp1, Errors.NONE); - prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, txnOffsetCommitResponse); - sender.runOnce(); // Send TxnOffsetCommitRequest again. - - assertTrue(addOffsetsResult.isCompleted()); - assertTrue(addOffsetsResult.isSuccessful()); - } - private TransactionalRequestResult prepareGroupMetadataCommit(Runnable prepareTxnCommitResponse) { doInitTransactions(); @@ -2603,7 +2577,7 @@ public void testTransitionToFatalErrorWhenRetriedBatchIsExpired() throws Interru @Test public void testBumpEpochAfterTimeoutWithoutPendingInflightRequests() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); long producerId = 15L; short epoch = 5; ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(producerId, epoch); @@ -2646,7 +2620,7 @@ public void testBumpEpochAfterTimeoutWithoutPendingInflightRequests() { @Test public void testNoProducerIdResetAfterLastInFlightBatchSucceeds() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); long producerId = 15L; short epoch = 5; ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(producerId, epoch); @@ -2686,7 +2660,7 @@ public void testNoProducerIdResetAfterLastInFlightBatchSucceeds() { @Test public void testEpochBumpAfterLastInflightBatchFails() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(producerId, epoch); initializeIdempotentProducerId(producerId, epoch); @@ -2719,7 +2693,7 @@ public void testEpochBumpAfterLastInflightBatchFails() { @Test public void testNoFailedBatchHandlingWhenTxnManagerIsInFatalError() { - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); long producerId = 15L; short epoch = 5; initializeIdempotentProducerId(producerId, epoch); @@ -3066,7 +3040,7 @@ public void testBumpTransactionalEpochOnRecoverableAddOffsetsRequestError() thro @Test public void testHealthyPartitionRetriesDuringEpochBump() throws InterruptedException { // Use a custom Sender to allow multiple inflight requests - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); Sender sender = new Sender(logContext, this.client, this.metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(new Metrics(time)), this.time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); @@ -3173,7 +3147,7 @@ public void testRetryAbortTransactionAfterCommitTimeout() { } @Test - public void testRetryCommitTransactionAfterAbortTimeout() throws InterruptedException { + public void testRetryCommitTransactionAfterAbortTimeout() { assertThrows(KafkaException.class, () -> verifyCommitOrAbortTransactionRetriable(TransactionResult.ABORT, TransactionResult.COMMIT)); } @@ -3190,7 +3164,7 @@ public void testCanBumpEpochDuringCoordinatorDisconnect() { @Test public void testFailedInflightBatchAfterEpochBump() throws InterruptedException { // Use a custom Sender to allow multiple inflight requests - initializeTransactionManager(Optional.empty(), false); + initializeTransactionManager(Optional.empty()); Sender sender = new Sender(logContext, this.client, this.metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(new Metrics(time)), this.time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java index 17d2e1ce26e43..3c66b211bec4f 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java @@ -62,7 +62,7 @@ public void testAlterIsrIsClusterAction() { public void testResponseThrottleTime() { Set authenticationKeys = EnumSet.of(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE); // Newer protocol apis include throttle time ms even for cluster actions - Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_ISR); + Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_ISR, ApiKeys.ALLOCATE_PRODUCER_IDS); for (ApiKeys apiKey: ApiKeys.zkBrokerApis()) { Schema responseSchema = apiKey.messageType.responseSchemas()[apiKey.latestVersion()]; BoundField throttleTimeField = responseSchema.get("throttle_time_ms"); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 8f567aa915433..fc709dc584327 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -2118,8 +2118,7 @@ private TxnOffsetCommitRequest createTxnOffsetCommitRequest(int version) { "groupId", 21L, (short) 42, - offsets, - false).build(); + offsets).build(); } else { return new TxnOffsetCommitRequest.Builder("transactionalId", "groupId", @@ -2128,8 +2127,7 @@ private TxnOffsetCommitRequest createTxnOffsetCommitRequest(int version) { offsets, "member", 2, - Optional.of("instance"), - false).build(); + Optional.of("instance")).build(); } } @@ -2147,8 +2145,7 @@ private TxnOffsetCommitRequest createTxnOffsetCommitRequestWithAutoDowngrade(int offsets, "member", 2, - Optional.of("instance"), - true).build(); + Optional.of("instance")).build(); } private TxnOffsetCommitResponse createTxnOffsetCommitResponse() { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java index 037066e9a036d..d49bdce4cef4a 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition; import org.apache.kafka.common.message.TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic; import org.apache.kafka.common.protocol.ApiKeys; @@ -34,8 +33,6 @@ import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertThrows; public class TxnOffsetCommitRequestTest extends OffsetCommitRequestTest { @@ -68,13 +65,9 @@ public void setUp() { groupId, producerId, producerEpoch, - offsets, - false); - - initializeBuilderWithGroupMetadata(false); - } + offsets + ); - private void initializeBuilderWithGroupMetadata(final boolean autoDowngrade) { builderWithGroupMetadata = new TxnOffsetCommitRequest.Builder( transactionalId, groupId, @@ -83,8 +76,8 @@ private void initializeBuilderWithGroupMetadata(final boolean autoDowngrade) { offsets, memberId, generationId, - Optional.of(groupInstanceId), - autoDowngrade); + Optional.of(groupInstanceId) + ); } @Test @@ -134,25 +127,4 @@ public void testConstructor() { assertEquals(throttleTimeMs, response.throttleTimeMs()); } } - - @Test - public void testEnableGroupMetadataAutoDowngrade() { - for (short version = 0; version <= 2; version++) { - initializeBuilderWithGroupMetadata(true); - final TxnOffsetCommitRequest request = builderWithGroupMetadata.build(version); - - assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, request.data().memberId()); - assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, request.data().generationId()); - assertNull(request.data().groupInstanceId()); - } - } - - @Test - public void testDisableGroupMetadataAutoDowngrade() { - for (short version = 0; version <= 2; version++) { - initializeBuilderWithGroupMetadata(false); - final short finalVersion = version; - assertThrows(UnsupportedVersionException.class, () -> builderWithGroupMetadata.build(finalVersion)); - } - } } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ConfigUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ConfigUtilsTest.java index b7279bb2e6b66..d760330eb2bc7 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ConfigUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/ConfigUtilsTest.java @@ -17,8 +17,12 @@ package org.apache.kafka.common.utils; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; import org.junit.jupiter.api.Test; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -140,4 +144,28 @@ public void testMultipleDeprecations() { assertNull(newConfig.get("foo.bar.deprecated")); assertNull(newConfig.get("foo.bar.even.more.deprecated")); } + + private static final ConfigDef CONFIG = new ConfigDef(). + define("myPassword", Type.PASSWORD, Importance.HIGH, ""). + define("myString", Type.STRING, Importance.HIGH, ""). + define("myInt", Type.INT, Importance.HIGH, ""). + define("myString2", Type.STRING, Importance.HIGH, ""); + + @Test + public void testConfigMapToRedactedStringForEmptyMap() { + assertEquals("{}", ConfigUtils. + configMapToRedactedString(Collections.emptyMap(), CONFIG)); + } + + @Test + public void testConfigMapToRedactedStringWithSecrets() { + Map testMap1 = new HashMap<>(); + testMap1.put("myString", "whatever"); + testMap1.put("myInt", Integer.valueOf(123)); + testMap1.put("myPassword", "foosecret"); + testMap1.put("myString2", null); + testMap1.put("myUnknown", Integer.valueOf(456)); + assertEquals("{myInt=123, myPassword=(redacted), myString=\"whatever\", myString2=null, myUnknown=(redacted)}", + ConfigUtils.configMapToRedactedString(testMap1, CONFIG)); + } } diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java index 0a763cc24f85b..cf31a0082ed1f 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java @@ -120,6 +120,10 @@ public void configure(Map props) { @Override public R apply(R record) { + if (operatingValue(record) == null) { + return record; + } + if (operatingSchema(record) == null) { return applySchemaless(record); } else { diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java index 764b904ea3e24..60744b275e42f 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java @@ -89,6 +89,42 @@ public void testConfigMixWholeAndFieldTransformation() { assertThrows(ConfigException.class, () -> xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int8,int32"))); } + @Test + public void castNullValueRecordWithSchema() { + xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int64")); + SourceRecord original = new SourceRecord(null, null, "topic", 0, + Schema.STRING_SCHEMA, "key", Schema.STRING_SCHEMA, null); + SourceRecord transformed = xformValue.apply(original); + assertEquals(original, transformed); + } + + @Test + public void castNullValueRecordSchemaless() { + xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int64")); + SourceRecord original = new SourceRecord(null, null, "topic", 0, + Schema.STRING_SCHEMA, "key", null, null); + SourceRecord transformed = xformValue.apply(original); + assertEquals(original, transformed); + } + + @Test + public void castNullKeyRecordWithSchema() { + xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int64")); + SourceRecord original = new SourceRecord(null, null, "topic", 0, + Schema.STRING_SCHEMA, null, Schema.STRING_SCHEMA, "value"); + SourceRecord transformed = xformKey.apply(original); + assertEquals(original, transformed); + } + + @Test + public void castNullKeyRecordSchemaless() { + xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int64")); + SourceRecord original = new SourceRecord(null, null, "topic", 0, + null, null, Schema.STRING_SCHEMA, "value"); + SourceRecord transformed = xformKey.apply(original); + assertEquals(original, transformed); + } + @Test public void castWholeRecordKeyWithSchema() { xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8")); diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 879373787365f..724bc9894045a 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -112,7 +112,9 @@ object ApiVersion { // Flexible versioning on ListOffsets, WriteTxnMarkers and OffsetsForLeaderEpoch. Also adds topic IDs (KIP-516) KAFKA_2_8_IV0, // Introduced topic IDs to LeaderAndIsr and UpdateMetadata requests/responses (KIP-516) - KAFKA_2_8_IV1 + KAFKA_2_8_IV1, + // Introduce AllocateProducerIds (KIP-730) + KAFKA_3_0_IV0 ) // Map keys are the union of the short and full versions @@ -197,6 +199,8 @@ sealed trait ApiVersion extends Ordered[ApiVersion] { def isAlterIsrSupported: Boolean = this >= KAFKA_2_7_IV2 + def isAllocateProducerIdsSupported: Boolean = this >= KAFKA_3_0_IV0 + override def compare(that: ApiVersion): Int = ApiVersion.orderingByVersion.compare(this, that) @@ -447,6 +451,13 @@ case object KAFKA_2_8_IV1 extends DefaultApiVersion { val id: Int = 32 } +case object KAFKA_3_0_IV0 extends DefaultApiVersion { + val shortVersion: String = "3.0" + val subVersion = "IV0" + val recordVersion = RecordVersion.V2 + val id: Int = 33 +} + object ApiVersionValidator extends Validator { override def ensureValid(name: String, value: Any): Unit = { diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 769e9c4c05329..89cadf4485851 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -548,9 +548,6 @@ class Partition(val topicPartition: TopicPartition, leaderEpochStartOffsetOpt = Some(leaderEpochStartOffset) zkVersion = partitionState.zkVersion - // Clear any pending AlterIsr requests and check replica state - alterIsrManager.clearPending(topicPartition) - // In the case of successive leader elections in a short time period, a follower may have // entries in its log from a later epoch than any entry in the new leader's log. In order // to ensure that these followers can truncate to the right offset, we must cache the new @@ -629,9 +626,6 @@ class Partition(val topicPartition: TopicPartition, leaderEpochStartOffsetOpt = None zkVersion = partitionState.zkVersion - // Since we might have been a leader previously, still clear any pending AlterIsr requests - alterIsrManager.clearPending(topicPartition) - if (leaderReplicaIdOpt.contains(newLeaderBrokerId) && leaderEpoch == oldLeaderEpoch) { false } else { @@ -1341,13 +1335,15 @@ class Partition(val topicPartition: TopicPartition, isrState = proposedIsrState if (!alterIsrManager.submit(alterIsrItem)) { - // If the ISR manager did not accept our update, we need to revert back to previous state + // If the ISR manager did not accept our update, we need to revert the proposed state. + // This can happen if the ISR state was updated by the controller (via LeaderAndIsr in ZK-mode or + // ChangePartitionRecord in KRaft mode) but we have an AlterIsr request still in-flight. isrState = oldState isrChangeListener.markFailed() - throw new IllegalStateException(s"Failed to enqueue ISR change state $newLeaderAndIsr for partition $topicPartition") + warn(s"Failed to enqueue ISR change state $newLeaderAndIsr for partition $topicPartition") + } else { + debug(s"Enqueued ISR change to state $newLeaderAndIsr after transition to $proposedIsrState") } - - debug(s"Enqueued ISR change to state $newLeaderAndIsr after transition to $proposedIsrState") } /** @@ -1385,10 +1381,15 @@ class Partition(val topicPartition: TopicPartition, if (leaderAndIsr.leaderEpoch != leaderEpoch) { debug(s"Ignoring new ISR ${leaderAndIsr} since we have a stale leader epoch $leaderEpoch.") isrChangeListener.markFailed() - } else if (leaderAndIsr.zkVersion <= zkVersion) { + } else if (leaderAndIsr.zkVersion < zkVersion) { debug(s"Ignoring new ISR ${leaderAndIsr} since we have a newer version $zkVersion.") isrChangeListener.markFailed() } else { + // This is one of two states: + // 1) leaderAndIsr.zkVersion > zkVersion: Controller updated to new version with proposedIsrState. + // 2) leaderAndIsr.zkVersion == zkVersion: No update was performed since proposed and actual state are the same. + // In both cases, we want to move from Pending to Committed state to ensure new updates are processed. + isrState = CommittedIsr(leaderAndIsr.isr.toSet) zkVersion = leaderAndIsr.zkVersion info(s"ISR updated to ${isrState.isr.mkString(",")} and version updated to [$zkVersion]") diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index b17a6e8e6f7da..654649b4f48c2 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -18,13 +18,13 @@ package kafka.controller import java.util import java.util.concurrent.TimeUnit - import kafka.admin.AdminOperationException import kafka.api._ import kafka.common._ import kafka.controller.KafkaController.AlterIsrCallback import kafka.cluster.Broker import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFeaturesCallback} +import kafka.coordinator.transaction.ZkProducerIdManager import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.server._ import kafka.utils._ @@ -37,20 +37,20 @@ import org.apache.kafka.common.ElectionType import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException, StaleBrokerEpochException} -import org.apache.kafka.common.message.{AlterIsrRequestData, AlterIsrResponseData} +import org.apache.kafka.common.message.{AllocateProducerIdsRequestData, AllocateProducerIdsResponseData, AlterIsrRequestData, AlterIsrResponseData, UpdateFeaturesRequestData} import org.apache.kafka.common.feature.{Features, FinalizedVersionRange} -import org.apache.kafka.common.message.UpdateFeaturesRequestData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse} import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.server.common.ProducerIdsBlock import org.apache.zookeeper.KeeperException import org.apache.zookeeper.KeeperException.Code import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ -import scala.util.{Failure, Try} +import scala.util.{Failure, Success, Try} sealed trait ElectionTrigger final case object AutoTriggered extends ElectionTrigger @@ -2376,6 +2376,54 @@ class KafkaController(val config: KafkaConfig, } } + def allocateProducerIds(allocateProducerIdsRequest: AllocateProducerIdsRequestData, + callback: AllocateProducerIdsResponseData => Unit): Unit = { + + def eventManagerCallback(results: Either[Errors, ProducerIdsBlock]): Unit = { + results match { + case Left(error) => callback.apply(new AllocateProducerIdsResponseData().setErrorCode(error.code)) + case Right(pidBlock) => callback.apply( + new AllocateProducerIdsResponseData() + .setProducerIdStart(pidBlock.producerIdStart()) + .setProducerIdLen(pidBlock.producerIdLen())) + } + } + eventManager.put(AllocateProducerIds(allocateProducerIdsRequest.brokerId, + allocateProducerIdsRequest.brokerEpoch, eventManagerCallback)) + } + + def processAllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, ProducerIdsBlock] => Unit): Unit = { + // Handle a few short-circuits + if (!isActive) { + callback.apply(Left(Errors.NOT_CONTROLLER)) + return + } + + val brokerEpochOpt = controllerContext.liveBrokerIdAndEpochs.get(brokerId) + if (brokerEpochOpt.isEmpty) { + warn(s"Ignoring AllocateProducerIds due to unknown broker $brokerId") + callback.apply(Left(Errors.BROKER_ID_NOT_REGISTERED)) + return + } + + if (!brokerEpochOpt.contains(brokerEpoch)) { + warn(s"Ignoring AllocateProducerIds due to stale broker epoch $brokerEpoch for broker $brokerId") + callback.apply(Left(Errors.STALE_BROKER_EPOCH)) + return + } + + val maybeNewProducerIdsBlock = try { + Try(ZkProducerIdManager.getNewProducerIdBlock(brokerId, zkClient, this)) + } catch { + case ke: KafkaException => Failure(ke) + } + + maybeNewProducerIdsBlock match { + case Failure(exception) => callback.apply(Left(Errors.forException(exception))) + case Success(newProducerIdBlock) => callback.apply(Right(newProducerIdBlock)) + } + } + private def processControllerChange(): Unit = { maybeResign() } @@ -2454,6 +2502,8 @@ class KafkaController(val config: KafkaConfig, processIsrChangeNotification() case AlterIsrReceived(brokerId, brokerEpoch, isrsToAlter, callback) => processAlterIsr(brokerId, brokerEpoch, isrsToAlter, callback) + case AllocateProducerIds(brokerId, brokerEpoch, callback) => + processAllocateProducerIds(brokerId, brokerEpoch, callback) case Startup => processStartup() } @@ -2747,6 +2797,12 @@ case class UpdateFeatures(request: UpdateFeaturesRequest, override def preempt(): Unit = {} } +case class AllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, ProducerIdsBlock] => Unit) + extends ControllerEvent { + override def state: ControllerState = ControllerState.Idle + override def preempt(): Unit = {} +} + // Used only in test cases abstract class MockEvent(val state: ControllerState) extends ControllerEvent { diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index 8d3f542c2b725..b5d419ddf4acb 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -16,146 +16,227 @@ */ package kafka.coordinator.transaction -import java.nio.charset.StandardCharsets - -import kafka.utils.{Json, Logging} +import kafka.server.{BrokerToControllerChannelManager, ControllerRequestCompletionHandler} +import kafka.utils.Logging import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode} +import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.message.AllocateProducerIdsRequestData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{AllocateProducerIdsRequest, AllocateProducerIdsResponse} +import org.apache.kafka.server.common.ProducerIdsBlock -import scala.jdk.CollectionConverters._ +import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicBoolean +import scala.util.{Failure, Success, Try} /** * ProducerIdManager is the part of the transaction coordinator that provides ProducerIds in a unique way * such that the same producerId will not be assigned twice across multiple transaction coordinators. * - * ProducerIds are managed via ZooKeeper, where the latest producerId block is written on the corresponding ZK - * path by the manager who claims the block, where the written block_start and block_end are both inclusive. + * ProducerIds are managed by the controller. When requesting a new range of IDs, we are guaranteed to receive + * a unique block. */ -object ProducerIdManager extends Logging { - val CurrentVersion: Long = 1L - val PidBlockSize: Long = 1000L - - def generateProducerIdBlockJson(producerIdBlock: ProducerIdBlock): Array[Byte] = { - Json.encodeAsBytes(Map("version" -> CurrentVersion, - "broker" -> producerIdBlock.brokerId, - "block_start" -> producerIdBlock.blockStartId.toString, - "block_end" -> producerIdBlock.blockEndId.toString).asJava - ) - } - def parseProducerIdBlockData(jsonData: Array[Byte]): ProducerIdBlock = { - try { - Json.parseBytes(jsonData).map(_.asJsonObject).flatMap { js => - val brokerId = js("broker").to[Int] - val blockStart = js("block_start").to[String].toLong - val blockEnd = js("block_end").to[String].toLong - Some(ProducerIdBlock(brokerId, blockStart, blockEnd)) - }.getOrElse(throw new KafkaException(s"Failed to parse the producerId block json $jsonData")) - } catch { - case e: java.lang.NumberFormatException => - // this should never happen: the written data has exceeded long type limit - fatal(s"Read jason data $jsonData contains producerIds that have exceeded long type limit") - throw e - } +object ProducerIdManager { + // Once we reach this percentage of PIDs consumed from the current block, trigger a fetch of the next block + val PidPrefetchThreshold = 0.90 + + // Creates a ProducerIdGenerate that directly interfaces with ZooKeeper, IBP < 3.0-IV0 + def zk(brokerId: Int, zkClient: KafkaZkClient): ZkProducerIdManager = { + new ZkProducerIdManager(brokerId, zkClient) } -} -case class ProducerIdBlock(brokerId: Int, blockStartId: Long, blockEndId: Long) { - override def toString: String = { - val producerIdBlockInfo = new StringBuilder - producerIdBlockInfo.append("(brokerId:" + brokerId) - producerIdBlockInfo.append(",blockStartProducerId:" + blockStartId) - producerIdBlockInfo.append(",blockEndProducerId:" + blockEndId + ")") - producerIdBlockInfo.toString() + // Creates a ProducerIdGenerate that uses AllocateProducerIds RPC, IBP >= 3.0-IV0 + def rpc(brokerId: Int, + brokerEpochSupplier: () => Long, + controllerChannel: BrokerToControllerChannelManager, + maxWaitMs: Int): RPCProducerIdManager = { + new RPCProducerIdManager(brokerId, brokerEpochSupplier, controllerChannel, maxWaitMs) } } -trait ProducerIdGenerator { +trait ProducerIdManager { def generateProducerId(): Long def shutdown() : Unit = {} } -class ProducerIdManager(val brokerId: Int, val zkClient: KafkaZkClient) extends ProducerIdGenerator with Logging { - - this.logIdent = "[ProducerId Manager " + brokerId + "]: " - - private var currentProducerIdBlock: ProducerIdBlock = null - private var nextProducerId: Long = -1L - - // grab the first block of producerIds - this synchronized { - getNewProducerIdBlock() - nextProducerId = currentProducerIdBlock.blockStartId - } - - private def getNewProducerIdBlock(): Unit = { +object ZkProducerIdManager { + def getNewProducerIdBlock(brokerId: Int, zkClient: KafkaZkClient, logger: Logging): ProducerIdsBlock = { + // Get or create the existing PID block from ZK and attempt to update it. We retry in a loop here since other + // brokers may be generating PID blocks during a rolling upgrade var zkWriteComplete = false while (!zkWriteComplete) { // refresh current producerId block from zookeeper again val (dataOpt, zkVersion) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path) // generate the new producerId block - currentProducerIdBlock = dataOpt match { + val newProducerIdBlock = dataOpt match { case Some(data) => - val currProducerIdBlock = ProducerIdManager.parseProducerIdBlockData(data) - debug(s"Read current producerId block $currProducerIdBlock, Zk path version $zkVersion") + val currProducerIdBlock = ProducerIdBlockZNode.parseProducerIdBlockData(data) + logger.debug(s"Read current producerId block $currProducerIdBlock, Zk path version $zkVersion") - if (currProducerIdBlock.blockEndId > Long.MaxValue - ProducerIdManager.PidBlockSize) { + if (currProducerIdBlock.producerIdEnd > Long.MaxValue - ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) { // we have exhausted all producerIds (wow!), treat it as a fatal error - fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.blockEndId})") + logger.fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.producerIdEnd})") throw new KafkaException("Have exhausted all producerIds.") } - ProducerIdBlock(brokerId, currProducerIdBlock.blockEndId + 1L, currProducerIdBlock.blockEndId + ProducerIdManager.PidBlockSize) + new ProducerIdsBlock(brokerId, currProducerIdBlock.producerIdEnd + 1L, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) case None => - debug(s"There is no producerId block yet (Zk path version $zkVersion), creating the first block") - ProducerIdBlock(brokerId, 0L, ProducerIdManager.PidBlockSize - 1) + logger.debug(s"There is no producerId block yet (Zk path version $zkVersion), creating the first block") + new ProducerIdsBlock(brokerId, 0L, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) } - val newProducerIdBlockData = ProducerIdManager.generateProducerIdBlockJson(currentProducerIdBlock) + val newProducerIdBlockData = ProducerIdBlockZNode.generateProducerIdBlockJson(newProducerIdBlock) // try to write the new producerId block into zookeeper - val (succeeded, version) = zkClient.conditionalUpdatePath(ProducerIdBlockZNode.path, - newProducerIdBlockData, zkVersion, Some(checkProducerIdBlockZkData)) + val (succeeded, version) = zkClient.conditionalUpdatePath(ProducerIdBlockZNode.path, newProducerIdBlockData, zkVersion, None) zkWriteComplete = succeeded - if (zkWriteComplete) - info(s"Acquired new producerId block $currentProducerIdBlock by writing to Zk with path version $version") + if (zkWriteComplete) { + logger.info(s"Acquired new producerId block $newProducerIdBlock by writing to Zk with path version $version") + return newProducerIdBlock + } } + throw new IllegalStateException() } +} - private def checkProducerIdBlockZkData(zkClient: KafkaZkClient, path: String, expectedData: Array[Byte]): (Boolean, Int) = { - try { - val expectedPidBlock = ProducerIdManager.parseProducerIdBlockData(expectedData) - zkClient.getDataAndVersion(ProducerIdBlockZNode.path) match { - case (Some(data), zkVersion) => - val currProducerIdBLock = ProducerIdManager.parseProducerIdBlockData(data) - (currProducerIdBLock == expectedPidBlock, zkVersion) - case (None, _) => (false, -1) - } - } catch { - case e: Exception => - warn(s"Error while checking for producerId block Zk data on path $path: expected data " + - s"${new String(expectedData, StandardCharsets.UTF_8)}", e) - (false, -1) +class ZkProducerIdManager(brokerId: Int, + zkClient: KafkaZkClient) extends ProducerIdManager with Logging { + + this.logIdent = "[ZK ProducerId Manager " + brokerId + "]: " + + private var currentProducerIdBlock: ProducerIdsBlock = ProducerIdsBlock.EMPTY + private var nextProducerId: Long = _ + + // grab the first block of producerIds + this synchronized { + allocateNewProducerIdBlock() + nextProducerId = currentProducerIdBlock.producerIdStart + } + + private def allocateNewProducerIdBlock(): Unit = { + this synchronized { + currentProducerIdBlock = ZkProducerIdManager.getNewProducerIdBlock(brokerId, zkClient, this) } } def generateProducerId(): Long = { this synchronized { // grab a new block of producerIds if this block has been exhausted - if (nextProducerId > currentProducerIdBlock.blockEndId) { - getNewProducerIdBlock() - nextProducerId = currentProducerIdBlock.blockStartId + 1 + if (nextProducerId > currentProducerIdBlock.producerIdEnd) { + allocateNewProducerIdBlock() + nextProducerId = currentProducerIdBlock.producerIdStart + } + nextProducerId += 1 + nextProducerId - 1 + } + } +} + +class RPCProducerIdManager(brokerId: Int, + brokerEpochSupplier: () => Long, + controllerChannel: BrokerToControllerChannelManager, + maxWaitMs: Int) extends ProducerIdManager with Logging { + + this.logIdent = "[RPC ProducerId Manager " + brokerId + "]: " + + private val nextProducerIdBlock = new ArrayBlockingQueue[Try[ProducerIdsBlock]](1) + private val requestInFlight = new AtomicBoolean(false) + private var currentProducerIdBlock: ProducerIdsBlock = ProducerIdsBlock.EMPTY + private var nextProducerId: Long = -1L + + override def generateProducerId(): Long = { + this synchronized { + if (nextProducerId == -1L) { + // Send an initial request to get the first block + maybeRequestNextBlock() + nextProducerId = 0L } else { nextProducerId += 1 + + // Check if we need to fetch the next block + if (nextProducerId >= (currentProducerIdBlock.producerIdStart + currentProducerIdBlock.producerIdLen * ProducerIdManager.PidPrefetchThreshold)) { + maybeRequestNextBlock() + } } - nextProducerId - 1 + // If we've exhausted the current block, grab the next block (waiting if necessary) + if (nextProducerId > currentProducerIdBlock.producerIdEnd) { + val block = nextProducerIdBlock.poll(maxWaitMs, TimeUnit.MILLISECONDS) + if (block == null) { + throw Errors.REQUEST_TIMED_OUT.exception("Timed out waiting for next producer ID block") + } else { + block match { + case Success(nextBlock) => + currentProducerIdBlock = nextBlock + nextProducerId = currentProducerIdBlock.producerIdStart + case Failure(t) => throw t + } + } + } + nextProducerId + } + } + + + private def maybeRequestNextBlock(): Unit = { + if (nextProducerIdBlock.isEmpty && requestInFlight.compareAndSet(false, true)) { + sendRequest() + } + } + + private[transaction] def sendRequest(): Unit = { + val message = new AllocateProducerIdsRequestData() + .setBrokerEpoch(brokerEpochSupplier.apply()) + .setBrokerId(brokerId) + + val request = new AllocateProducerIdsRequest.Builder(message) + debug("Requesting next Producer ID block") + controllerChannel.sendRequest(request, new ControllerRequestCompletionHandler() { + override def onComplete(response: ClientResponse): Unit = { + val message = response.responseBody().asInstanceOf[AllocateProducerIdsResponse] + handleAllocateProducerIdsResponse(message) + } + + override def onTimeout(): Unit = handleTimeout() + }) + } + + private[transaction] def handleAllocateProducerIdsResponse(response: AllocateProducerIdsResponse): Unit = { + requestInFlight.set(false) + val data = response.data + Errors.forCode(data.errorCode()) match { + case Errors.NONE => + debug(s"Got next producer ID block from controller $data") + // Do some sanity checks on the response + if (data.producerIdStart() < currentProducerIdBlock.producerIdEnd) { + nextProducerIdBlock.put(Failure(new KafkaException( + s"Producer ID block is not monotonic with current block: current=$currentProducerIdBlock response=$data"))) + } else if (data.producerIdStart() < 0 || data.producerIdLen() < 0 || data.producerIdStart() > Long.MaxValue - data.producerIdLen()) { + nextProducerIdBlock.put(Failure(new KafkaException(s"Producer ID block includes invalid ID range: $data"))) + } else { + nextProducerIdBlock.put( + Success(new ProducerIdsBlock(brokerId, data.producerIdStart(), data.producerIdLen()))) + } + case Errors.STALE_BROKER_EPOCH => + warn("Our broker epoch was stale, trying again.") + maybeRequestNextBlock() + case Errors.BROKER_ID_NOT_REGISTERED => + warn("Our broker ID is not yet known by the controller, trying again.") + maybeRequestNextBlock() + case e: Errors => + warn("Had an unknown error from the controller, giving up.") + nextProducerIdBlock.put(Failure(e.exception())) } } - override def shutdown(): Unit = { - info(s"Shutdown complete: last producerId assigned $nextProducerId") + private[transaction] def handleTimeout(): Unit = { + warn("Timed out when requesting AllocateProducerIds from the controller.") + requestInFlight.set(false) + nextProducerIdBlock.put(Failure(Errors.REQUEST_TIMED_OUT.exception)) + maybeRequestNextBlock() } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 87029faa7430f..78983c16cbf67 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -34,7 +34,7 @@ object TransactionCoordinator { def apply(config: KafkaConfig, replicaManager: ReplicaManager, scheduler: Scheduler, - createProducerIdGenerator: () => ProducerIdGenerator, + createProducerIdGenerator: () => ProducerIdManager, metrics: Metrics, metadataCache: MetadataCache, time: Time): TransactionCoordinator = { @@ -81,7 +81,7 @@ object TransactionCoordinator { class TransactionCoordinator(brokerId: Int, txnConfig: TransactionConfig, scheduler: Scheduler, - createProducerIdGenerator: () => ProducerIdGenerator, + createProducerIdManager: () => ProducerIdManager, txnManager: TransactionStateManager, txnMarkerChannelManager: TransactionMarkerChannelManager, time: Time, @@ -98,7 +98,7 @@ class TransactionCoordinator(brokerId: Int, /* Active flag of the coordinator */ private val isActive = new AtomicBoolean(false) - val producerIdGenerator = createProducerIdGenerator() + val producerIdManager = createProducerIdManager() def handleInitProducerId(transactionalId: String, transactionTimeoutMs: Int, @@ -109,7 +109,7 @@ class TransactionCoordinator(brokerId: Int, if (transactionalId == null) { // if the transactional id is null, then always blindly accept the request // and return a new producerId from the producerId manager - val producerId = producerIdGenerator.generateProducerId() + val producerId = producerIdManager.generateProducerId() responseCallback(InitProducerIdResult(producerId, producerEpoch = 0, Errors.NONE)) } else if (transactionalId.isEmpty) { // if transactional id is empty then return error as invalid request. This is @@ -121,7 +121,7 @@ class TransactionCoordinator(brokerId: Int, } else { val coordinatorEpochAndMetadata = txnManager.getTransactionState(transactionalId).flatMap { case None => - val producerId = producerIdGenerator.generateProducerId() + val producerId = producerIdManager.generateProducerId() val createdMetadata = new TransactionMetadata(transactionalId = transactionalId, producerId = producerId, lastProducerId = RecordBatch.NO_PRODUCER_ID, @@ -227,7 +227,7 @@ class TransactionCoordinator(brokerId: Int, // If the epoch is exhausted and the expected epoch (if provided) matches it, generate a new producer ID if (txnMetadata.isProducerEpochExhausted && expectedProducerIdAndEpoch.forall(_.epoch == txnMetadata.producerEpoch)) { - val newProducerId = producerIdGenerator.generateProducerId() + val newProducerId = producerIdManager.generateProducerId() Right(txnMetadata.prepareProducerIdRotation(newProducerId, transactionTimeoutMs, time.milliseconds(), expectedProducerIdAndEpoch.isDefined)) } else { @@ -684,7 +684,7 @@ class TransactionCoordinator(brokerId: Int, info("Shutting down.") isActive.set(false) scheduler.shutdown() - producerIdGenerator.shutdown() + producerIdManager.shutdown() txnManager.shutdown() txnMarkerChannelManager.shutdown() info("Shutdown complete.") diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index bb0297e414bd1..c75a2e9ebb152 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -239,7 +239,6 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason { * @param scheduler The thread pool scheduler used for background actions * @param brokerTopicStats Container for Broker Topic Yammer Metrics * @param time The time instance used for checking the clock - * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired * @param producerIdExpirationCheckIntervalMs How often to check for producer ids which need to be expired * @param topicPartition The topic partition associated with this Log instance * @param leaderEpochCache The LeaderEpochFileCache instance (if any) containing state associated @@ -267,7 +266,6 @@ class Log(@volatile private var _dir: File, scheduler: Scheduler, brokerTopicStats: BrokerTopicStats, val time: Time, - val maxProducerIdExpirationMs: Int, val producerIdExpirationCheckIntervalMs: Int, val topicPartition: TopicPartition, @volatile var leaderEpochCache: Option[LeaderEpochFileCache], @@ -558,7 +556,7 @@ class Log(@volatile private var _dir: File, } private def initializeLeaderEpochCache(): Unit = lock synchronized { - leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion) + leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion, logIdent) } private def updateLogEndOffset(offset: Long): Unit = { @@ -593,7 +591,7 @@ class Log(@volatile private var _dir: File, producerStateManager: ProducerStateManager): Unit = lock synchronized { checkIfMemoryMappedBufferClosed() Log.rebuildProducerState(producerStateManager, segments, logStartOffset, lastOffset, recordVersion, time, - reloadFromCleanShutdown = false) + reloadFromCleanShutdown = false, logIdent) } def activeProducers: Seq[DescribeProducersResponseData.ProducerState] = { @@ -1896,14 +1894,14 @@ class Log(@volatile private var _dir: File, private def deleteSegmentFiles(segments: Iterable[LogSegment], asyncDelete: Boolean, deleteProducerStateSnapshots: Boolean = true): Unit = { Log.deleteSegmentFiles(segments, asyncDelete, deleteProducerStateSnapshots, dir, topicPartition, - config, scheduler, logDirFailureChannel, producerStateManager) + config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent) } private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = { lock synchronized { checkIfMemoryMappedBufferClosed() Log.replaceSegments(segments, newSegments, oldSegments, isRecoveredSwapFile, dir, topicPartition, - config, scheduler, logDirFailureChannel, producerStateManager) + config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent) } } @@ -1945,7 +1943,7 @@ class Log(@volatile private var _dir: File, } private[log] def splitOverflowedSegment(segment: LogSegment): List[LogSegment] = lock synchronized { - Log.splitOverflowedSegment(segment, segments, dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager) + Log.splitOverflowedSegment(segment, segments, dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent) } } @@ -2013,7 +2011,12 @@ object Log extends Logging { Files.createDirectories(dir.toPath) val topicPartition = Log.parseTopicPartitionName(dir) val segments = new LogSegments(topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache( + dir, + topicPartition, + logDirFailureChannel, + config.messageFormatVersion.recordVersion, + s"[Log partition=$topicPartition, dir=${dir.getParent}] ") val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) val offsets = LogLoader.load(LoadLogParams( dir, @@ -2030,8 +2033,8 @@ object Log extends Logging { leaderEpochCache, producerStateManager)) new Log(dir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, offsets.nextOffsetMetadata, scheduler, - brokerTopicStats, time, maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs, topicPartition, - leaderEpochCache, producerStateManager, logDirFailureChannel, topicId, keepPartitionMetadataFile) + brokerTopicStats, time, producerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, + producerStateManager, logDirFailureChannel, topicId, keepPartitionMetadataFile) } /** @@ -2234,12 +2237,14 @@ object Log extends Logging { * @param topicPartition The topic partition * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure * @param recordVersion The record version + * @param logPrefix The logging prefix * @return The new LeaderEpochFileCache instance (if created), none otherwise */ def maybeCreateLeaderEpochCache(dir: File, topicPartition: TopicPartition, logDirFailureChannel: LogDirFailureChannel, - recordVersion: RecordVersion): Option[LeaderEpochFileCache] = { + recordVersion: RecordVersion, + logPrefix: String): Option[LeaderEpochFileCache] = { val leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir) def newLeaderEpochFileCache(): LeaderEpochFileCache = { @@ -2254,7 +2259,7 @@ object Log extends Logging { None if (currentCache.exists(_.nonEmpty)) - warn(s"Deleting non-empty leader epoch cache due to incompatible message format $recordVersion") + warn(s"${logPrefix}Deleting non-empty leader epoch cache due to incompatible message format $recordVersion") Files.deleteIfExists(leaderEpochFile.toPath) None @@ -2301,6 +2306,7 @@ object Log extends Logging { * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure * @param producerStateManager The ProducerStateManager instance (if any) containing state associated * with the existingSegments + * @param logPrefix The logging prefix */ private[log] def replaceSegments(existingSegments: LogSegments, newSegments: Seq[LogSegment], @@ -2311,7 +2317,8 @@ object Log extends Logging { config: LogConfig, scheduler: Scheduler, logDirFailureChannel: LogDirFailureChannel, - producerStateManager: ProducerStateManager): Unit = { + producerStateManager: ProducerStateManager, + logPrefix: String): Unit = { val sortedNewSegments = newSegments.sortBy(_.baseOffset) // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment() @@ -2340,7 +2347,8 @@ object Log extends Logging { config, scheduler, logDirFailureChannel, - producerStateManager) + producerStateManager, + logPrefix) } // okay we are safe now, remove the swap suffix sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, "")) @@ -2367,7 +2375,7 @@ object Log extends Logging { * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure * @param producerStateManager The ProducerStateManager instance (if any) containing state associated * with the existingSegments - * + * @param logPrefix The logging prefix * @throws IOException if the file can't be renamed and still exists */ private[log] def deleteSegmentFiles(segmentsToDelete: Iterable[LogSegment], @@ -2378,11 +2386,12 @@ object Log extends Logging { config: LogConfig, scheduler: Scheduler, logDirFailureChannel: LogDirFailureChannel, - producerStateManager: ProducerStateManager): Unit = { + producerStateManager: ProducerStateManager, + logPrefix: String): Unit = { segmentsToDelete.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix)) def deleteSegments(): Unit = { - info(s"Deleting segment files ${segmentsToDelete.mkString(",")}") + info(s"${logPrefix}Deleting segment files ${segmentsToDelete.mkString(",")}") val parentDir = dir.getParent maybeHandleIOException(logDirFailureChannel, parentDir, s"Error while deleting segments for $topicPartition in dir $parentDir") { segmentsToDelete.foreach { segment => @@ -2437,6 +2446,7 @@ object Log extends Logging { * @param time The time instance used for checking the clock * @param reloadFromCleanShutdown True if the producer state is being built after a clean shutdown, * false otherwise. + * @param logPrefix The logging prefix */ private[log] def rebuildProducerState(producerStateManager: ProducerStateManager, segments: LogSegments, @@ -2444,7 +2454,8 @@ object Log extends Logging { lastOffset: Long, recordVersion: RecordVersion, time: Time, - reloadFromCleanShutdown: Boolean): Unit = { + reloadFromCleanShutdown: Boolean, + logPrefix: String): Unit = { val allSegments = segments.values val offsetsToSnapshot = if (allSegments.nonEmpty) { @@ -2453,7 +2464,7 @@ object Log extends Logging { } else { Seq(Some(lastOffset)) } - info(s"Loading producer state till offset $lastOffset with message format version ${recordVersion.value}") + info(s"${logPrefix}Loading producer state till offset $lastOffset with message format version ${recordVersion.value}") // We want to avoid unnecessary scanning of the log to build the producer state when the broker is being // upgraded. The basic idea is to use the absence of producer snapshot files to detect the upgrade case, @@ -2477,7 +2488,7 @@ object Log extends Logging { producerStateManager.takeSnapshot() } } else { - info(s"Reloading from producer snapshot and rebuilding producer state from offset $lastOffset") + info(s"${logPrefix}Reloading from producer snapshot and rebuilding producer state from offset $lastOffset") val isEmptyBeforeTruncation = producerStateManager.isEmpty && producerStateManager.mapEndOffset >= lastOffset val producerStateLoadStart = time.milliseconds() producerStateManager.truncateAndReload(logStartOffset, lastOffset, time.milliseconds()) @@ -2516,7 +2527,7 @@ object Log extends Logging { } producerStateManager.updateMapEndOffset(lastOffset) producerStateManager.takeSnapshot() - info(s"Producer state recovery took ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " + + info(s"${logPrefix}Producer state recovery took ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " + s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset") } } @@ -2543,6 +2554,7 @@ object Log extends Logging { * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure * @param producerStateManager The ProducerStateManager instance (if any) containing state associated * with the existingSegments + * @param logPrefix The logging prefix * @return List of new segments that replace the input segment */ private[log] def splitOverflowedSegment(segment: LogSegment, @@ -2552,11 +2564,12 @@ object Log extends Logging { config: LogConfig, scheduler: Scheduler, logDirFailureChannel: LogDirFailureChannel, - producerStateManager: ProducerStateManager): List[LogSegment] = { + producerStateManager: ProducerStateManager, + logPrefix: String): List[LogSegment] = { require(Log.isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}") require(segment.hasOverflow, "Split operation is only permitted for segments with overflow") - info(s"Splitting overflowed segment $segment") + info(s"${logPrefix}Splitting overflowed segment $segment") val newSegments = ListBuffer[LogSegment]() try { @@ -2589,9 +2602,9 @@ object Log extends Logging { s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments") // replace old segment with new ones - info(s"Replacing overflowed segment $segment with split segments $newSegments") + info(s"${logPrefix}Replacing overflowed segment $segment with split segments $newSegments") replaceSegments(existingSegments, newSegments.toList, List(segment), isRecoveredSwapFile = false, - dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager) + dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager, logPrefix) newSegments.toList } catch { case e: Exception => diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala index 6b28ec51bb91d..bfadb78f7ab5a 100644 --- a/core/src/main/scala/kafka/log/LogLoader.scala +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -67,7 +67,7 @@ case class LoadLogParams(dir: File, maxProducerIdExpirationMs: Int, leaderEpochCache: Option[LeaderEpochFileCache], producerStateManager: ProducerStateManager) { - val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}]" + val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}] " } /** @@ -152,7 +152,8 @@ object LogLoader extends Logging { nextOffset, params.config.messageFormatVersion.recordVersion, params.time, - reloadFromCleanShutdown = params.hadCleanShutdown) + reloadFromCleanShutdown = params.hadCleanShutdown, + params.logIdentifier) val activeSegment = params.segments.lastSegment.get LoadedLogOffsets( @@ -172,7 +173,7 @@ object LogLoader extends Logging { private def removeTempFilesAndCollectSwapFiles(params: LoadLogParams): Set[File] = { def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = { - info(s"${params.logIdentifier} Deleting index files with suffix $suffix for baseFile $baseFile") + info(s"${params.logIdentifier}Deleting index files with suffix $suffix for baseFile $baseFile") val offset = offsetFromFile(baseFile) Files.deleteIfExists(Log.offsetIndexFile(params.dir, offset, suffix).toPath) Files.deleteIfExists(Log.timeIndexFile(params.dir, offset, suffix).toPath) @@ -188,7 +189,7 @@ object LogLoader extends Logging { throw new IOException(s"Could not read file $file") val filename = file.getName if (filename.endsWith(DeletedFileSuffix)) { - debug(s"${params.logIdentifier} Deleting stray temporary file ${file.getAbsolutePath}") + debug(s"${params.logIdentifier}Deleting stray temporary file ${file.getAbsolutePath}") Files.deleteIfExists(file.toPath) } else if (filename.endsWith(CleanedFileSuffix)) { minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset) @@ -198,7 +199,7 @@ object LogLoader extends Logging { // if a log, delete the index files, complete the swap operation later // if an index just delete the index files, they will be rebuilt val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, "")) - info(s"${params.logIdentifier} Found file ${file.getAbsolutePath} from interrupted swap operation.") + info(s"${params.logIdentifier}Found file ${file.getAbsolutePath} from interrupted swap operation.") if (Log.isIndexFile(baseFile)) { deleteIndicesIfExist(baseFile) } else if (Log.isLogFile(baseFile)) { @@ -213,7 +214,7 @@ object LogLoader extends Logging { // for more details about the split operation. val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset) invalidSwapFiles.foreach { file => - debug(s"${params.logIdentifier} Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset") + debug(s"${params.logIdentifier}Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset") val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, "")) deleteIndicesIfExist(baseFile, SwapFileSuffix) Files.deleteIfExists(file.toPath) @@ -221,7 +222,7 @@ object LogLoader extends Logging { // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files cleanFiles.foreach { file => - debug(s"${params.logIdentifier} Deleting stray .clean file ${file.getAbsolutePath}") + debug(s"${params.logIdentifier}Deleting stray .clean file ${file.getAbsolutePath}") Files.deleteIfExists(file.toPath) } @@ -245,7 +246,7 @@ object LogLoader extends Logging { return fn } catch { case e: LogSegmentOffsetOverflowException => - info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.") + info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.") Log.splitOverflowedSegment( e.segment, params.segments, @@ -254,7 +255,8 @@ object LogLoader extends Logging { params.config, params.scheduler, params.logDirFailureChannel, - params.producerStateManager) + params.producerStateManager, + params.logIdentifier) } } throw new IllegalStateException() @@ -280,7 +282,7 @@ object LogLoader extends Logging { val offset = offsetFromFile(file) val logFile = Log.logFile(params.dir, offset) if (!logFile.exists) { - warn(s"${params.logIdentifier} Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.") + warn(s"${params.logIdentifier}Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.") Files.deleteIfExists(file.toPath) } } else if (isLogFile(file)) { @@ -297,11 +299,11 @@ object LogLoader extends Logging { try segment.sanityCheck(timeIndexFileNewlyCreated) catch { case _: NoSuchFileException => - error(s"${params.logIdentifier} Could not find offset index file corresponding to log file" + + error(s"${params.logIdentifier}Could not find offset index file corresponding to log file" + s" ${segment.log.file.getAbsolutePath}, recovering segment and rebuilding index files...") recoverSegment(segment, params) case e: CorruptIndexException => - warn(s"${params.logIdentifier} Found a corrupted index file corresponding to log file" + + warn(s"${params.logIdentifier}Found a corrupted index file corresponding to log file" + s" ${segment.log.file.getAbsolutePath} due to ${e.getMessage}}, recovering segment and" + " rebuilding index files...") recoverSegment(segment, params) @@ -330,7 +332,8 @@ object LogLoader extends Logging { segment.baseOffset, params.config.messageFormatVersion.recordVersion, params.time, - reloadFromCleanShutdown = false) + reloadFromCleanShutdown = false, + params.logIdentifier) val bytesTruncated = segment.recover(producerStateManager, params.leaderEpochCache) // once we have recovered the segment's data, take a snapshot to ensure that we won't // need to reload the same segment again while recovering another segment. @@ -390,7 +393,8 @@ object LogLoader extends Logging { params.config, params.scheduler, params.logDirFailureChannel, - params.producerStateManager) + params.producerStateManager, + params.logIdentifier) } } @@ -434,20 +438,20 @@ object LogLoader extends Logging { while (unflushed.hasNext && !truncated) { val segment = unflushed.next() - info(s"${params.logIdentifier} Recovering unflushed segment ${segment.baseOffset}") + info(s"${params.logIdentifier}Recovering unflushed segment ${segment.baseOffset}") val truncatedBytes = try { recoverSegment(segment, params) } catch { case _: InvalidOffsetException => val startOffset = segment.baseOffset - warn(s"${params.logIdentifier} Found invalid offset during recovery. Deleting the" + + warn(s"${params.logIdentifier}Found invalid offset during recovery. Deleting the" + s" corrupt segment and creating an empty one with starting offset $startOffset") segment.truncateTo(startOffset) } if (truncatedBytes > 0) { // we had an invalid message, delete all remaining log - warn(s"${params.logIdentifier} Corruption found in segment ${segment.baseOffset}," + + warn(s"${params.logIdentifier}Corruption found in segment ${segment.baseOffset}," + s" truncating to offset ${segment.readNextOffset}") removeAndDeleteSegmentsAsync(unflushed.toList, params) truncated = true @@ -519,7 +523,8 @@ object LogLoader extends Logging { params.config, params.scheduler, params.logDirFailureChannel, - params.producerStateManager) + params.producerStateManager, + params.logIdentifier) } } } diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index c0fbd68e75bf2..bb8e327b1890a 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -29,6 +29,7 @@ object RequestConvertToJson { request match { case req: AddOffsetsToTxnRequest => AddOffsetsToTxnRequestDataJsonConverter.write(req.data, request.version) case req: AddPartitionsToTxnRequest => AddPartitionsToTxnRequestDataJsonConverter.write(req.data, request.version) + case req: AllocateProducerIdsRequest => AllocateProducerIdsRequestDataJsonConverter.write(req.data, request.version) case req: AlterClientQuotasRequest => AlterClientQuotasRequestDataJsonConverter.write(req.data, request.version) case req: AlterConfigsRequest => AlterConfigsRequestDataJsonConverter.write(req.data, request.version) case req: AlterIsrRequest => AlterIsrRequestDataJsonConverter.write(req.data, request.version) @@ -103,6 +104,7 @@ object RequestConvertToJson { response match { case res: AddOffsetsToTxnResponse => AddOffsetsToTxnResponseDataJsonConverter.write(res.data, version) case res: AddPartitionsToTxnResponse => AddPartitionsToTxnResponseDataJsonConverter.write(res.data, version) + case res: AllocateProducerIdsResponse => AllocateProducerIdsResponseDataJsonConverter.write(res.data, version) case res: AlterClientQuotasResponse => AlterClientQuotasResponseDataJsonConverter.write(res.data, version) case res: AlterConfigsResponse => AlterConfigsResponseDataJsonConverter.write(res.data, version) case res: AlterIsrResponse => AlterIsrResponseDataJsonConverter.write(res.data, version) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index e58593a0848f9..7b5f83dd53ac7 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -16,28 +16,29 @@ */ package kafka.raft -import java.io.{File, IOException} -import java.nio.file.{Files, NoSuchFileException} -import java.util.concurrent.ConcurrentSkipListSet +import java.io.File +import java.nio.file.{Files, NoSuchFileException, Path} import java.util.{Optional, Properties} import kafka.api.ApiVersion import kafka.log.{AppendOrigin, Log, LogConfig, LogOffsetSnapshot, SnapshotGenerated} import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, LogDirFailureChannel, RequestLocal} -import kafka.utils.{Logging, Scheduler} +import kafka.utils.{CoreUtils, Logging, Scheduler} import org.apache.kafka.common.record.{MemoryRecords, Records} -import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.raft.{Isolation, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog} import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} +import scala.annotation.nowarn +import scala.collection.mutable import scala.compat.java8.OptionConverters._ final class KafkaMetadataLog private ( log: Log, scheduler: Scheduler, - // This object needs to be thread-safe because it is used by the snapshotting thread to notify the - // polling thread when snapshots are created. - snapshotIds: ConcurrentSkipListSet[OffsetAndEpoch], + // Access to this object needs to be synchronized because it is used by the snapshotting thread to notify the + // polling thread when snapshots are created. This object is also used to store any opened snapshot reader. + snapshots: mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]], topicPartition: TopicPartition, maxFetchSizeInBytes: Int, val fileDeleteDelayMs: Long // Visible for testing, @@ -161,19 +162,24 @@ final class KafkaMetadataLog private ( override def truncateToLatestSnapshot(): Boolean = { val latestEpoch = log.latestEpoch.getOrElse(0) - latestSnapshotId().asScala match { - case Some(snapshotId) if (snapshotId.epoch > latestEpoch || - (snapshotId.epoch == latestEpoch && snapshotId.offset > endOffset().offset)) => + val (truncated, forgottenSnapshots) = latestSnapshotId().asScala match { + case Some(snapshotId) if ( + snapshotId.epoch > latestEpoch || + (snapshotId.epoch == latestEpoch && snapshotId.offset > endOffset().offset) + ) => // Truncate the log fully if the latest snapshot is greater than the log end offset - log.truncateFullyAndStartAt(snapshotId.offset) - // Delete snapshot after truncating - removeSnapshotFilesBefore(snapshotId) - - true - case _ => false + // Forget snapshots less than the log start offset + snapshots synchronized { + (true, forgetSnapshotsBefore(snapshotId)) + } + case _ => + (false, mutable.TreeMap.empty[OffsetAndEpoch, Option[FileRawSnapshotReader]]) } + + removeSnapshots(forgottenSnapshots) + truncated } override def initializeLeaderEpoch(epoch: Int): Unit = { @@ -242,85 +248,116 @@ final class KafkaMetadataLog private ( } override def readSnapshot(snapshotId: OffsetAndEpoch): Optional[RawSnapshotReader] = { - try { - if (snapshotIds.contains(snapshotId)) { - Optional.of(FileRawSnapshotReader.open(log.dir.toPath, snapshotId)) - } else { - Optional.empty() + snapshots synchronized { + val reader = snapshots.get(snapshotId) match { + case None => + // Snapshot doesn't exists + None + case Some(None) => + // Snapshot exists but has never been read before + try { + val snapshotReader = Some(FileRawSnapshotReader.open(log.dir.toPath, snapshotId)) + snapshots.put(snapshotId, snapshotReader) + snapshotReader + } catch { + case _: NoSuchFileException => + // Snapshot doesn't exists in the data dir; remove + val path = Snapshots.snapshotPath(log.dir.toPath, snapshotId) + warn(s"Couldn't read $snapshotId; expected to find snapshot file $path") + snapshots.remove(snapshotId) + None + } + case Some(value) => + // Snapshot exists and it is already open; do nothing + value } - } catch { - case _: NoSuchFileException => - Optional.empty() + + reader.asJava.asInstanceOf[Optional[RawSnapshotReader]] } } override def latestSnapshotId(): Optional[OffsetAndEpoch] = { - val descending = snapshotIds.descendingIterator - if (descending.hasNext) { - Optional.of(descending.next) - } else { - Optional.empty() + snapshots synchronized { + snapshots.lastOption.map { case (snapshotId, _) => snapshotId }.asJava } } override def earliestSnapshotId(): Optional[OffsetAndEpoch] = { - val ascendingIterator = snapshotIds.iterator - if (ascendingIterator.hasNext) { - Optional.of(ascendingIterator.next) - } else { - Optional.empty() + snapshots synchronized { + snapshots.headOption.map { case (snapshotId, _) => snapshotId }.asJava } } override def onSnapshotFrozen(snapshotId: OffsetAndEpoch): Unit = { - snapshotIds.add(snapshotId) + snapshots synchronized { + snapshots.put(snapshotId, None) + } } override def deleteBeforeSnapshot(logStartSnapshotId: OffsetAndEpoch): Boolean = { - latestSnapshotId().asScala match { - case Some(snapshotId) if (snapshotIds.contains(logStartSnapshotId) && - startOffset < logStartSnapshotId.offset && - logStartSnapshotId.offset <= snapshotId.offset && - log.maybeIncrementLogStartOffset(logStartSnapshotId.offset, SnapshotGenerated)) => - log.deleteOldSegments() + val (deleted, forgottenSnapshots) = snapshots synchronized { + latestSnapshotId().asScala match { + case Some(snapshotId) if (snapshots.contains(logStartSnapshotId) && + startOffset < logStartSnapshotId.offset && + logStartSnapshotId.offset <= snapshotId.offset && + log.maybeIncrementLogStartOffset(logStartSnapshotId.offset, SnapshotGenerated)) => + + // Delete all segments that have a "last offset" less than the log start offset + log.deleteOldSegments() - // Delete snapshot after increasing LogStartOffset - removeSnapshotFilesBefore(logStartSnapshotId) + // Forget snapshots less than the log start offset + (true, forgetSnapshotsBefore(logStartSnapshotId)) + case _ => + (false, mutable.TreeMap.empty[OffsetAndEpoch, Option[FileRawSnapshotReader]]) + } + } - true + removeSnapshots(forgottenSnapshots) + deleted + } - case _ => false - } + /** + * Forget the snapshots earlier than a given snapshot id and return the associated + * snapshot readers. + * + * This method assumes that the lock for `snapshots` is already held. + */ + @nowarn("cat=deprecation") // Needed for TreeMap.until + private def forgetSnapshotsBefore( + logStartSnapshotId: OffsetAndEpoch + ): mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] = { + val expiredSnapshots = snapshots.until(logStartSnapshotId).clone() + snapshots --= expiredSnapshots.keys + + expiredSnapshots } /** - * Removes all snapshots on the log directory whose epoch and end offset is less than the giving epoch and end offset. + * Rename the given snapshots on the log directory. Asynchronously, close and delete the + * given snapshots after some delay. */ - private def removeSnapshotFilesBefore(logStartSnapshotId: OffsetAndEpoch): Unit = { - val expiredSnapshotIdsIter = snapshotIds.headSet(logStartSnapshotId, false).iterator - while (expiredSnapshotIdsIter.hasNext) { - val snapshotId = expiredSnapshotIdsIter.next() - // If snapshotIds contains a snapshot id, the KafkaRaftClient and Listener can expect that the snapshot exists - // on the file system, so we should first remove snapshotId and then delete snapshot file. - expiredSnapshotIdsIter.remove() - - val path = Snapshots.snapshotPath(log.dir.toPath, snapshotId) - val destination = Snapshots.deleteRename(path, snapshotId) - try { - Utils.atomicMoveWithFallback(path, destination, false) - } catch { - case e: IOException => - error(s"Error renaming snapshot file: $path to $destination", e) - } + private def removeSnapshots( + expiredSnapshots: mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] + ): Unit = { + expiredSnapshots.foreach { case (snapshotId, _) => + Snapshots.markForDelete(log.dir.toPath, snapshotId) + } + + if (expiredSnapshots.nonEmpty) { scheduler.schedule( - "delete-snapshot-file", - () => Snapshots.deleteSnapshotIfExists(log.dir.toPath, snapshotId), - fileDeleteDelayMs) + "delete-snapshot-files", + KafkaMetadataLog.deleteSnapshotFiles(log.dir.toPath, expiredSnapshots, this), + fileDeleteDelayMs + ) } } override def close(): Unit = { log.close() + snapshots synchronized { + snapshots.values.flatten.foreach(_.close()) + snapshots.clear() + } } } @@ -376,8 +413,8 @@ object KafkaMetadataLog { private def recoverSnapshots( log: Log - ): ConcurrentSkipListSet[OffsetAndEpoch] = { - val snapshotIds = new ConcurrentSkipListSet[OffsetAndEpoch]() + ): mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] = { + val snapshots = mutable.TreeMap.empty[OffsetAndEpoch, Option[FileRawSnapshotReader]] // Scan the log directory; deleting partial snapshots and older snapshot, only remembering immutable snapshots start // from logStartOffset Files @@ -397,11 +434,23 @@ object KafkaMetadataLog { // Delete partial snapshot, deleted snapshot and older snapshot Files.deleteIfExists(snapshotPath.path) } else { - snapshotIds.add(snapshotPath.snapshotId) + snapshots.put(snapshotPath.snapshotId, None) } } } - snapshotIds + snapshots } + private def deleteSnapshotFiles( + logDir: Path, + expiredSnapshots: mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]], + logging: Logging + ): () => Unit = () => { + expiredSnapshots.foreach { case (snapshotId, snapshotReader) => + snapshotReader.foreach { reader => + CoreUtils.swallow(reader.close(), logging) + } + Snapshots.deleteIfExists(logDir, snapshotId) + } + } } diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index f0d37be8a6837..01d30c6eae029 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -27,7 +27,6 @@ import kafka.server.{KafkaConfig, MetaProperties} import kafka.utils.timer.SystemTimer import kafka.utils.{KafkaScheduler, Logging, ShutdownableThread} import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient} -import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} import org.apache.kafka.common.protocol.ApiMessage @@ -35,8 +34,9 @@ import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} +import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, NON_ROUTABLE_ADDRESS, UnknownAddressSpec} -import org.apache.kafka.raft.{FileBasedStateStore, KafkaRaftClient, RaftClient, RaftConfig, RaftRequest} +import org.apache.kafka.raft.{FileBasedStateStore, KafkaRaftClient, LeaderAndEpoch, RaftClient, RaftConfig, RaftRequest} import org.apache.kafka.server.common.serialization.RecordSerde import scala.jdk.CollectionConverters._ @@ -100,6 +100,10 @@ trait RaftManager[T] { epoch: Int, records: Seq[T] ): Option[Long] + + def leaderAndEpoch: LeaderAndEpoch + + def client: RaftClient[T] } class KafkaRaftManager[T]( @@ -125,10 +129,10 @@ class KafkaRaftManager[T]( private val dataDir = createDataDir() private val metadataLog = buildMetadataLog() private val netChannel = buildNetworkChannel() - private val raftClient = buildRaftClient() - private val raftIoThread = new RaftIoThread(raftClient, threadNamePrefix) + val client: KafkaRaftClient[T] = buildRaftClient() + private val raftIoThread = new RaftIoThread(client, threadNamePrefix) - def kafkaRaftClient: KafkaRaftClient[T] = raftClient + def kafkaRaftClient: KafkaRaftClient[T] = client def startup(): Unit = { // Update the voter endpoints (if valid) with what's in RaftConfig @@ -151,7 +155,7 @@ class KafkaRaftManager[T]( def shutdown(): Unit = { raftIoThread.shutdown() - raftClient.close() + client.close() scheduler.shutdown() netChannel.close() metadataLog.close() @@ -160,7 +164,7 @@ class KafkaRaftManager[T]( override def register( listener: RaftClient.Listener[T] ): Unit = { - raftClient.register(listener) + client.register(listener) } override def scheduleAtomicAppend( @@ -183,9 +187,9 @@ class KafkaRaftManager[T]( isAtomic: Boolean ): Option[Long] = { val offset = if (isAtomic) { - raftClient.scheduleAtomicAppend(epoch, records.asJava) + client.scheduleAtomicAppend(epoch, records.asJava) } else { - raftClient.scheduleAppend(epoch, records.asJava) + client.scheduleAppend(epoch, records.asJava) } Option(offset).map(Long.unbox) @@ -202,7 +206,7 @@ class KafkaRaftManager[T]( createdTimeMs ) - raftClient.handle(inboundRequest) + client.handle(inboundRequest) inboundRequest.completion.thenApply { response => response.data @@ -307,4 +311,7 @@ class KafkaRaftManager[T]( ) } + override def leaderAndEpoch: LeaderAndEpoch = { + client.leaderAndEpoch + } } diff --git a/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala b/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala index cac6480c73c38..5f701d8d40984 100644 --- a/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala @@ -175,9 +175,11 @@ class AclAuthorizer extends Authorizer with Logging { val zkClientConfig = AclAuthorizer.zkClientConfigFromKafkaConfigAndMap(kafkaConfig, configs) val time = Time.SYSTEM + // createChrootIfNecessary=true is necessary in case we are running in a KRaft cluster + // because such a cluster will not create any chroot path in ZooKeeper (it doesn't connect to ZooKeeper) zkClient = KafkaZkClient(zkUrl, kafkaConfig.zkEnableSecureAcls, zkSessionTimeOutMs, zkConnectionTimeoutMs, zkMaxInFlightRequests, time, "kafka.security", "AclAuthorizer", name=Some("ACL authorizer"), - zkClientConfig = zkClientConfig) + zkClientConfig = zkClientConfig, createChrootIfNecessary = true) zkClient.createAclPaths() extendedAclSupport = kafkaConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1 diff --git a/core/src/main/scala/kafka/server/AclApis.scala b/core/src/main/scala/kafka/server/AclApis.scala new file mode 100644 index 0000000000000..a79ce849869b5 --- /dev/null +++ b/core/src/main/scala/kafka/server/AclApis.scala @@ -0,0 +1,155 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.network.RequestChannel +import kafka.security.authorizer.AuthorizerUtils +import kafka.utils.Logging +import org.apache.kafka.common.acl.AclOperation._ +import org.apache.kafka.common.acl.AclBinding +import org.apache.kafka.common.errors._ +import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult +import org.apache.kafka.common.message._ +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests._ +import org.apache.kafka.common.resource.Resource.CLUSTER_NAME +import org.apache.kafka.common.resource.ResourceType +import org.apache.kafka.server.authorizer._ +import java.util + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable +import scala.compat.java8.OptionConverters._ +import scala.jdk.CollectionConverters._ + +/** + * Logic to handle ACL requests. + */ +class AclApis(authHelper: AuthHelper, + authorizer: Option[Authorizer], + requestHelper: RequestHandlerHelper, + name: String, + config: KafkaConfig) extends Logging { + this.logIdent = "[AclApis-%s-%s] ".format(name, config.nodeId) + private val alterAclsPurgatory = + new DelayedFuturePurgatory(purgatoryName = "AlterAcls", brokerId = config.nodeId) + + def close(): Unit = { + alterAclsPurgatory.shutdown() + } + + def handleDescribeAcls(request: RequestChannel.Request): Unit = { + authHelper.authorizeClusterOperation(request, DESCRIBE) + val describeAclsRequest = request.body[DescribeAclsRequest] + authorizer match { + case None => + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new DescribeAclsResponse(new DescribeAclsResponseData() + .setErrorCode(Errors.SECURITY_DISABLED.code) + .setErrorMessage("No Authorizer is configured on the broker") + .setThrottleTimeMs(requestThrottleMs), + describeAclsRequest.version)) + case Some(auth) => + val filter = describeAclsRequest.filter + val returnedAcls = new util.HashSet[AclBinding]() + auth.acls(filter).forEach(returnedAcls.add) + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new DescribeAclsResponse(new DescribeAclsResponseData() + .setThrottleTimeMs(requestThrottleMs) + .setResources(DescribeAclsResponse.aclsResources(returnedAcls)), + describeAclsRequest.version)) + } + } + + def handleCreateAcls(request: RequestChannel.Request): Unit = { + authHelper.authorizeClusterOperation(request, ALTER) + val createAclsRequest = request.body[CreateAclsRequest] + + authorizer match { + case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + createAclsRequest.getErrorResponse(requestThrottleMs, + new SecurityDisabledException("No Authorizer is configured."))) + case Some(auth) => + val allBindings = createAclsRequest.aclCreations.asScala.map(CreateAclsRequest.aclBinding) + val errorResults = mutable.Map[AclBinding, AclCreateResult]() + val validBindings = new ArrayBuffer[AclBinding] + allBindings.foreach { acl => + val resource = acl.pattern + val throwable = if (resource.resourceType == ResourceType.CLUSTER && !AuthorizerUtils.isClusterResource(resource.name)) + new InvalidRequestException("The only valid name for the CLUSTER resource is " + CLUSTER_NAME) + else if (resource.name.isEmpty) + new InvalidRequestException("Invalid empty resource name") + else + null + if (throwable != null) { + debug(s"Failed to add acl $acl to $resource", throwable) + errorResults(acl) = new AclCreateResult(throwable) + } else + validBindings += acl + } + + val createResults = auth.createAcls(request.context, validBindings.asJava).asScala.map(_.toCompletableFuture) + + def sendResponseCallback(): Unit = { + val aclCreationResults = allBindings.map { acl => + val result = errorResults.getOrElse(acl, createResults(validBindings.indexOf(acl)).get) + val creationResult = new AclCreationResult() + result.exception.asScala.foreach { throwable => + val apiError = ApiError.fromThrowable(throwable) + creationResult + .setErrorCode(apiError.error.code) + .setErrorMessage(apiError.message) + } + creationResult + } + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new CreateAclsResponse(new CreateAclsResponseData() + .setThrottleTimeMs(requestThrottleMs) + .setResults(aclCreationResults.asJava))) + } + + alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, createResults, sendResponseCallback) + } + } + + def handleDeleteAcls(request: RequestChannel.Request): Unit = { + authHelper.authorizeClusterOperation(request, ALTER) + val deleteAclsRequest = request.body[DeleteAclsRequest] + authorizer match { + case None => + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + deleteAclsRequest.getErrorResponse(requestThrottleMs, + new SecurityDisabledException("No Authorizer is configured."))) + case Some(auth) => + + val deleteResults = auth.deleteAcls(request.context, deleteAclsRequest.filters) + .asScala.map(_.toCompletableFuture).toList + + def sendResponseCallback(): Unit = { + val filterResults = deleteResults.map(_.get).map(DeleteAclsResponse.filterResult).asJava + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new DeleteAclsResponse( + new DeleteAclsResponseData() + .setThrottleTimeMs(requestThrottleMs) + .setFilterResults(filterResults), + deleteAclsRequest.version)) + } + alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, deleteResults, sendResponseCallback) + } + } +} diff --git a/core/src/main/scala/kafka/server/AlterIsrManager.scala b/core/src/main/scala/kafka/server/AlterIsrManager.scala index e6a934df61423..489152ddeaf31 100644 --- a/core/src/main/scala/kafka/server/AlterIsrManager.scala +++ b/core/src/main/scala/kafka/server/AlterIsrManager.scala @@ -49,8 +49,6 @@ trait AlterIsrManager { def shutdown(): Unit = {} def submit(alterIsrItem: AlterIsrItem): Boolean - - def clearPending(topicPartition: TopicPartition): Unit } case class AlterIsrItem(topicPartition: TopicPartition, @@ -134,10 +132,6 @@ class DefaultAlterIsrManager( enqueued } - override def clearPending(topicPartition: TopicPartition): Unit = { - unsentIsrUpdates.remove(topicPartition) - } - private[server] def maybePropagateIsrChanges(): Unit = { // Send all pending items if there is not already a request in-flight. if (!unsentIsrUpdates.isEmpty && inflightRequest.compareAndSet(false, true)) { diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 49edbfe295068..bea0c53c1d78f 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -18,17 +18,18 @@ package kafka.server import java.util -import java.util.concurrent.{CompletableFuture, TimeUnit, TimeoutException} import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantLock +import java.util.concurrent.{CompletableFuture, TimeUnit, TimeoutException} import java.net.InetAddress import kafka.cluster.Broker.ServerInfo import kafka.coordinator.group.GroupCoordinator -import kafka.coordinator.transaction.{ProducerIdGenerator, TransactionCoordinator} +import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator} import kafka.log.LogManager import kafka.metrics.KafkaYammerMetrics import kafka.network.SocketServer +import kafka.raft.RaftManager import kafka.security.CredentialProvider import kafka.server.metadata.{BrokerMetadataListener, CachedConfigRepository, ClientQuotaCache, ClientQuotaMetadataManager, RaftMetadataCache} import kafka.utils.{CoreUtils, KafkaScheduler} @@ -43,10 +44,10 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils} import org.apache.kafka.common.{ClusterResource, Endpoint, KafkaException} import org.apache.kafka.metadata.{BrokerState, VersionRange} -import org.apache.kafka.metalog.MetaLogManager import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig.AddressSpec import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.common.ApiMessageAndVersion; import scala.collection.{Map, Seq} import scala.jdk.CollectionConverters._ @@ -55,16 +56,16 @@ import scala.jdk.CollectionConverters._ * A Kafka broker that runs in KRaft (Kafka Raft) mode. */ class BrokerServer( - val config: KafkaConfig, - val metaProps: MetaProperties, - val metaLogManager: MetaLogManager, - val time: Time, - val metrics: Metrics, - val threadNamePrefix: Option[String], - val initialOfflineDirs: Seq[String], - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], - val supportedFeatures: util.Map[String, VersionRange] - ) extends KafkaBroker { + val config: KafkaConfig, + val metaProps: MetaProperties, + val raftManager: RaftManager[ApiMessageAndVersion], + val time: Time, + val metrics: Metrics, + val threadNamePrefix: Option[String], + val initialOfflineDirs: Seq[String], + val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], + val supportedFeatures: util.Map[String, VersionRange] +) extends KafkaBroker { import kafka.server.Server._ @@ -181,7 +182,7 @@ class BrokerServer( credentialProvider = new CredentialProvider(ScramMechanism.mechanismNames, tokenCache) val controllerNodes = RaftConfig.voterConnectionsToNodes(controllerQuorumVotersFuture.get()).asScala - val controllerNodeProvider = RaftControllerNodeProvider(metaLogManager, config, controllerNodes) + val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes) clientToControllerChannelManager = BrokerToControllerChannelManager( controllerNodeProvider, @@ -284,7 +285,7 @@ class BrokerServer( metaProps.clusterId, networkListeners, supportedFeatures) // Register a listener with the Raft layer to receive metadata event notifications - metaLogManager.register(brokerMetadataListener) + raftManager.register(brokerMetadataListener) val endpoints = new util.ArrayList[Endpoint](networkListeners.size()) var interBrokerListener: Endpoint = null @@ -375,7 +376,7 @@ class BrokerServer( } } - class TemporaryProducerIdManager() extends ProducerIdGenerator { + class TemporaryProducerIdManager() extends ProducerIdManager { val maxProducerIdsPerBrokerEpoch = 1000000 var currentOffset = -1 override def generateProducerId(): Long = { @@ -389,7 +390,7 @@ class BrokerServer( } } - def createTemporaryProducerIdManager(): ProducerIdGenerator = { + def createTemporaryProducerIdManager(): ProducerIdManager = { new TemporaryProducerIdManager() } diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala index 5834a17942286..974cce1ae93f5 100644 --- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala @@ -21,6 +21,7 @@ import java.util.concurrent.LinkedBlockingDeque import java.util.concurrent.atomic.AtomicReference import kafka.common.{InterBrokerSendThread, RequestAndCompletionHandler} +import kafka.raft.RaftManager import kafka.utils.Logging import org.apache.kafka.clients._ import org.apache.kafka.common.Node @@ -31,9 +32,10 @@ import org.apache.kafka.common.requests.AbstractRequest import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} -import org.apache.kafka.metalog.MetaLogManager +import org.apache.kafka.server.common.ApiMessageAndVersion; import scala.collection.Seq +import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ trait ControllerNodeProvider { @@ -77,15 +79,14 @@ class MetadataCacheControllerNodeProvider( } object RaftControllerNodeProvider { - def apply(metaLogManager: MetaLogManager, + def apply(raftManager: RaftManager[ApiMessageAndVersion], config: KafkaConfig, controllerQuorumVoterNodes: Seq[Node]): RaftControllerNodeProvider = { - val controllerListenerName = new ListenerName(config.controllerListenerNames.head) val controllerSecurityProtocol = config.listenerSecurityProtocolMap.getOrElse(controllerListenerName, SecurityProtocol.forName(controllerListenerName.value())) val controllerSaslMechanism = config.saslMechanismControllerProtocol new RaftControllerNodeProvider( - metaLogManager, + raftManager, controllerQuorumVoterNodes, controllerListenerName, controllerSecurityProtocol, @@ -98,7 +99,7 @@ object RaftControllerNodeProvider { * Finds the controller node by checking the metadata log manager. * This provider is used when we are using a Raft-based metadata quorum. */ -class RaftControllerNodeProvider(val metaLogManager: MetaLogManager, +class RaftControllerNodeProvider(val raftManager: RaftManager[ApiMessageAndVersion], controllerQuorumVoterNodes: Seq[Node], val listenerName: ListenerName, val securityProtocol: SecurityProtocol, @@ -107,14 +108,7 @@ class RaftControllerNodeProvider(val metaLogManager: MetaLogManager, val idToNode = controllerQuorumVoterNodes.map(node => node.id() -> node).toMap override def get(): Option[Node] = { - val leader = metaLogManager.leader() - if (leader == null) { - None - } else if (leader.nodeId() < 0) { - None - } else { - idToNode.get(leader.nodeId()) - } + raftManager.leaderAndEpoch.leaderId.asScala.map(idToNode) } } diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index cc51998054991..fa0dc79465af3 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -74,6 +74,7 @@ class ControllerApis(val requestChannel: RequestChannel, val authHelper = new AuthHelper(authorizer) val requestHelper = new RequestHandlerHelper(requestChannel, quotas, time) + private val aclApis = new AclApis(authHelper, authorizer, requestHelper, "controller", config) override def handle(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = { try { @@ -100,6 +101,9 @@ class ControllerApis(val requestChannel: RequestChannel, case ApiKeys.SASL_HANDSHAKE => handleSaslHandshakeRequest(request) case ApiKeys.SASL_AUTHENTICATE => handleSaslAuthenticateRequest(request) case ApiKeys.CREATE_PARTITIONS => handleCreatePartitions(request) + case ApiKeys.DESCRIBE_ACLS => aclApis.handleDescribeAcls(request) + case ApiKeys.CREATE_ACLS => aclApis.handleCreateAcls(request) + case ApiKeys.DELETE_ACLS => aclApis.handleDeleteAcls(request) case _ => throw new ApiException(s"Unsupported ApiKey ${request.context.header.apiKey}") } } catch { diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 638e23ac84c4c..3f574bc1a4bc3 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -17,9 +17,10 @@ package kafka.server -import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util import java.util.concurrent.locks.ReentrantLock +import java.util.concurrent.{CompletableFuture, TimeUnit} + import kafka.cluster.Broker.ServerInfo import kafka.log.LogConfig import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector} @@ -37,7 +38,6 @@ import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{ClusterResource, Endpoint} import org.apache.kafka.controller.{Controller, QuorumController, QuorumControllerMetrics} import org.apache.kafka.metadata.VersionRange -import org.apache.kafka.metalog.MetaLogManager import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig.AddressSpec import org.apache.kafka.server.authorizer.Authorizer @@ -49,15 +49,14 @@ import scala.jdk.CollectionConverters._ * A Kafka controller that runs in KRaft (Kafka Raft) mode. */ class ControllerServer( - val metaProperties: MetaProperties, - val config: KafkaConfig, - val metaLogManager: MetaLogManager, - val raftManager: RaftManager[ApiMessageAndVersion], - val time: Time, - val metrics: Metrics, - val threadNamePrefix: Option[String], - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]] - ) extends Logging with KafkaMetricsGroup { + val metaProperties: MetaProperties, + val config: KafkaConfig, + val raftManager: RaftManager[ApiMessageAndVersion], + val time: Time, + val metrics: Metrics, + val threadNamePrefix: Option[String], + val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]] +) extends Logging with KafkaMetricsGroup { import kafka.server.Server._ val lock = new ReentrantLock() @@ -148,7 +147,7 @@ class ControllerServer( setTime(time). setThreadNamePrefix(threadNamePrefixAsString). setConfigDefs(configDefs). - setLogManager(metaLogManager). + setRaftClient(raftManager.client). setDefaultReplicationFactor(config.defaultReplicationFactor.toShort). setDefaultNumPartitions(config.numPartitions.intValue()). setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(), diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 91d214fd8ee97..2cf24c87ceba9 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.metrics.MetricsReporter import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} import org.apache.kafka.common.security.authenticator.LoginManager -import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.utils.{ConfigUtils, Utils} import scala.collection._ import scala.jdk.CollectionConverters._ @@ -601,7 +601,8 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging } if (!validateOnly) { - info(s"Reconfiguring $reconfigurable, updated configs: $updatedConfigNames custom configs: $newCustomConfigs") + info(s"Reconfiguring $reconfigurable, updated configs: $updatedConfigNames " + + s"custom configs: ${ConfigUtils.configMapToRedactedString(newCustomConfigs, KafkaConfig.configDef)}") reconfigurable.reconfigure(newConfigs) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index e9e063fb030d7..c5e140494b31e 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -26,7 +26,6 @@ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinat import kafka.log.AppendOrigin import kafka.message.ZStdCompressionCodec import kafka.network.RequestChannel -import kafka.security.authorizer.AuthorizerUtils import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.metadata.ConfigRepository import kafka.utils.Implicits._ @@ -34,14 +33,13 @@ import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} import org.apache.kafka.common.acl.AclOperation._ -import org.apache.kafka.common.acl.{AclBinding, AclOperation} +import org.apache.kafka.common.acl.AclOperation import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal} import org.apache.kafka.common.internals.{FatalExitError, Topic} import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} -import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} @@ -82,9 +80,7 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger import java.util.{Collections, Optional} import scala.annotation.nowarn -import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Seq, Set, immutable, mutable} -import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} @@ -111,18 +107,15 @@ class KafkaApis(val requestChannel: RequestChannel, val tokenManager: DelegationTokenManager, val apiVersionManager: ApiVersionManager) extends ApiRequestHandler with Logging { - metadataSupport.ensureConsistentWith(config) - type FetchResponseStats = Map[TopicPartition, RecordConversionStats] this.logIdent = "[KafkaApi-%d] ".format(brokerId) val configHelper = new ConfigHelper(metadataCache, config, configRepository) - private val alterAclsPurgatory = new DelayedFuturePurgatory(purgatoryName = "AlterAcls", brokerId = config.brokerId) - val authHelper = new AuthHelper(authorizer) val requestHelper = new RequestHandlerHelper(requestChannel, quotas, time) + val aclApis = new AclApis(authHelper, authorizer, requestHelper, "broker", config) def close(): Unit = { - alterAclsPurgatory.shutdown() + aclApis.close() info("Shutdown complete.") } @@ -223,6 +216,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.UNREGISTER_BROKER => maybeForwardToController(request, handleUnregisterBrokerRequest) case ApiKeys.DESCRIBE_TRANSACTIONS => handleDescribeTransactionsRequest(request) case ApiKeys.LIST_TRANSACTIONS => handleListTransactionsRequest(request) + case ApiKeys.ALLOCATE_PRODUCER_IDS => handleAllocateProducerIdsRequest(request) case _ => throw new IllegalStateException(s"No handler for request api key ${request.header.apiKey}") } } catch { @@ -2401,105 +2395,17 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDescribeAcls(request: RequestChannel.Request): Unit = { - authHelper.authorizeClusterOperation(request, DESCRIBE) - val describeAclsRequest = request.body[DescribeAclsRequest] - authorizer match { - case None => - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new DescribeAclsResponse(new DescribeAclsResponseData() - .setErrorCode(Errors.SECURITY_DISABLED.code) - .setErrorMessage("No Authorizer is configured on the broker") - .setThrottleTimeMs(requestThrottleMs), - describeAclsRequest.version)) - case Some(auth) => - val filter = describeAclsRequest.filter - val returnedAcls = new util.HashSet[AclBinding]() - auth.acls(filter).forEach(returnedAcls.add) - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new DescribeAclsResponse(new DescribeAclsResponseData() - .setThrottleTimeMs(requestThrottleMs) - .setResources(DescribeAclsResponse.aclsResources(returnedAcls)), - describeAclsRequest.version)) - } + aclApis.handleDescribeAcls(request) } def handleCreateAcls(request: RequestChannel.Request): Unit = { metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) - authHelper.authorizeClusterOperation(request, ALTER) - val createAclsRequest = request.body[CreateAclsRequest] - - authorizer match { - case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - createAclsRequest.getErrorResponse(requestThrottleMs, - new SecurityDisabledException("No Authorizer is configured on the broker."))) - case Some(auth) => - val allBindings = createAclsRequest.aclCreations.asScala.map(CreateAclsRequest.aclBinding) - val errorResults = mutable.Map[AclBinding, AclCreateResult]() - val validBindings = new ArrayBuffer[AclBinding] - allBindings.foreach { acl => - val resource = acl.pattern - val throwable = if (resource.resourceType == ResourceType.CLUSTER && !AuthorizerUtils.isClusterResource(resource.name)) - new InvalidRequestException("The only valid name for the CLUSTER resource is " + CLUSTER_NAME) - else if (resource.name.isEmpty) - new InvalidRequestException("Invalid empty resource name") - else - null - if (throwable != null) { - debug(s"Failed to add acl $acl to $resource", throwable) - errorResults(acl) = new AclCreateResult(throwable) - } else - validBindings += acl - } - - val createResults = auth.createAcls(request.context, validBindings.asJava).asScala.map(_.toCompletableFuture) - - def sendResponseCallback(): Unit = { - val aclCreationResults = allBindings.map { acl => - val result = errorResults.getOrElse(acl, createResults(validBindings.indexOf(acl)).get) - val creationResult = new AclCreationResult() - result.exception.asScala.foreach { throwable => - val apiError = ApiError.fromThrowable(throwable) - creationResult - .setErrorCode(apiError.error.code) - .setErrorMessage(apiError.message) - } - creationResult - } - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new CreateAclsResponse(new CreateAclsResponseData() - .setThrottleTimeMs(requestThrottleMs) - .setResults(aclCreationResults.asJava))) - } - - alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, createResults, sendResponseCallback) - } + aclApis.handleCreateAcls(request) } def handleDeleteAcls(request: RequestChannel.Request): Unit = { metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) - authHelper.authorizeClusterOperation(request, ALTER) - val deleteAclsRequest = request.body[DeleteAclsRequest] - authorizer match { - case None => - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - deleteAclsRequest.getErrorResponse(requestThrottleMs, - new SecurityDisabledException("No Authorizer is configured on the broker."))) - case Some(auth) => - - val deleteResults = auth.deleteAcls(request.context, deleteAclsRequest.filters) - .asScala.map(_.toCompletableFuture).toList - - def sendResponseCallback(): Unit = { - val filterResults = deleteResults.map(_.get).map(DeleteAclsResponse.filterResult).asJava - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new DeleteAclsResponse( - new DeleteAclsResponseData() - .setThrottleTimeMs(requestThrottleMs) - .setFilterResults(filterResults), - deleteAclsRequest.version)) - } - alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, deleteResults, sendResponseCallback) - } + aclApis.handleDeleteAcls(request) } def handleOffsetForLeaderEpochRequest(request: RequestChannel.Request): Unit = { @@ -3372,6 +3278,22 @@ class KafkaApis(val requestChannel: RequestChannel, new ListTransactionsResponse(response.setThrottleTimeMs(requestThrottleMs))) } + def handleAllocateProducerIdsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) + authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) + + val allocateProducerIdsRequest = request.body[AllocateProducerIdsRequest] + + if (!zkSupport.controller.isActive) + requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => + allocateProducerIdsRequest.getErrorResponse(throttleTimeMs, Errors.NOT_CONTROLLER.exception)) + else + zkSupport.controller.allocateProducerIds(allocateProducerIdsRequest.data, producerIdsResponse => + requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => + new AllocateProducerIdsResponse(producerIdsResponse.setThrottleTimeMs(throttleTimeMs))) + ) + } + private def updateRecordConversionStats(request: RequestChannel.Request, tp: TopicPartition, conversionStats: RecordConversionStats): Unit = { diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 1f96eaa7bacf6..8e77357383f9b 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -24,10 +24,10 @@ import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics} import kafka.raft.KafkaRaftManager import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole} import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties} -import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.utils.{AppInfoParser, Time} +import org.apache.kafka.common.{TopicPartition, Uuid} +import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.RaftConfig -import org.apache.kafka.raft.metadata.{MetaLogRaftShim, MetadataRecordSerde} import org.apache.kafka.server.common.ApiMessageAndVersion import scala.collection.Seq @@ -55,7 +55,7 @@ class KafkaRaftServer( private val metrics = Server.initializeMetrics( config, time, - metaProps.clusterId.toString + metaProps.clusterId ) private val controllerQuorumVotersFuture = CompletableFuture.completedFuture( @@ -73,13 +73,11 @@ class KafkaRaftServer( controllerQuorumVotersFuture ) - private val metaLogShim = new MetaLogRaftShim(raftManager.kafkaRaftClient, config.nodeId) - private val broker: Option[BrokerServer] = if (config.processRoles.contains(BrokerRole)) { Some(new BrokerServer( config, metaProps, - metaLogShim, + raftManager, time, metrics, threadNamePrefix, @@ -95,7 +93,6 @@ class KafkaRaftServer( Some(new ControllerServer( metaProps, config, - metaLogShim, raftManager, time, metrics, diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 02fb28c67dbb1..fa182c0609daa 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -134,7 +134,7 @@ class KafkaServer( var autoTopicCreationManager: AutoTopicCreationManager = null - var clientToControllerChannelManager: Option[BrokerToControllerChannelManager] = None + var clientToControllerChannelManager: BrokerToControllerChannelManager = null var alterIsrManager: AlterIsrManager = null @@ -257,19 +257,21 @@ class KafkaServer( tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames) credentialProvider = new CredentialProvider(ScramMechanism.mechanismNames, tokenCache) + clientToControllerChannelManager = BrokerToControllerChannelManager( + controllerNodeProvider = MetadataCacheControllerNodeProvider(config, metadataCache), + time = time, + metrics = metrics, + config = config, + channelName = "forwarding", + threadNamePrefix = threadNamePrefix, + retryTimeoutMs = config.requestTimeoutMs.longValue) + clientToControllerChannelManager.start() + /* start forwarding manager */ + var autoTopicCreationChannel = Option.empty[BrokerToControllerChannelManager] if (enableForwarding) { - val brokerToControllerManager = BrokerToControllerChannelManager( - controllerNodeProvider = MetadataCacheControllerNodeProvider(config, metadataCache), - time = time, - metrics = metrics, - config = config, - channelName = "forwarding", - threadNamePrefix = threadNamePrefix, - retryTimeoutMs = config.requestTimeoutMs.longValue) - brokerToControllerManager.start() - this.forwardingManager = Some(ForwardingManager(brokerToControllerManager)) - clientToControllerChannelManager = Some(brokerToControllerManager) + this.forwardingManager = Some(ForwardingManager(clientToControllerChannelManager)) + autoTopicCreationChannel = Some(clientToControllerChannelManager) } val apiVersionManager = ApiVersionManager( @@ -330,10 +332,21 @@ class KafkaServer( groupCoordinator = GroupCoordinator(config, replicaManager, Time.SYSTEM, metrics) groupCoordinator.startup(() => zkClient.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicPartitions)) + /* create producer ids manager */ + val producerIdManager = if (config.interBrokerProtocolVersion.isAllocateProducerIdsSupported) { + ProducerIdManager.rpc( + config.brokerId, + brokerEpochSupplier = () => kafkaController.brokerEpoch, + clientToControllerChannelManager, + config.requestTimeoutMs + ) + } else { + ProducerIdManager.zk(config.brokerId, zkClient) + } /* start transaction coordinator, with a separate background thread scheduler for transaction expiration and log loading */ // Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue transactionCoordinator = TransactionCoordinator(config, replicaManager, new KafkaScheduler(threads = 1, threadNamePrefix = "transaction-log-manager-"), - () => new ProducerIdManager(config.brokerId, zkClient), metrics, metadataCache, Time.SYSTEM) + () => producerIdManager, metrics, metadataCache, Time.SYSTEM) transactionCoordinator.startup( () => zkClient.getTopicPartitionCount(Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(config.transactionTopicPartitions)) @@ -342,7 +355,7 @@ class KafkaServer( config, metadataCache, threadNamePrefix, - clientToControllerChannelManager, + autoTopicCreationChannel, Some(adminManager), Some(kafkaController), groupCoordinator, @@ -428,17 +441,6 @@ class KafkaServer( private def initZkClient(time: Time): Unit = { info(s"Connecting to zookeeper on ${config.zkConnect}") - def createZkClient(zkConnect: String, isSecure: Boolean) = { - KafkaZkClient(zkConnect, isSecure, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - config.zkMaxInFlightRequests, time, name = Some("Kafka server"), zkClientConfig = Some(zkClientConfig)) - } - - val chrootIndex = config.zkConnect.indexOf("/") - val chrootOption = { - if (chrootIndex > 0) Some(config.zkConnect.substring(chrootIndex)) - else None - } - val secureAclsEnabled = config.zkEnableSecureAcls val isZkSecurityEnabled = JaasUtils.isZkSaslEnabled() || KafkaConfig.zkTlsClientAuthEnabled(zkClientConfig) @@ -446,16 +448,9 @@ class KafkaServer( throw new java.lang.SecurityException(s"${KafkaConfig.ZkEnableSecureAclsProp} is true, but ZooKeeper client TLS configuration identifying at least $KafkaConfig.ZkSslClientEnableProp, $KafkaConfig.ZkClientCnxnSocketProp, and $KafkaConfig.ZkSslKeyStoreLocationProp was not present and the " + s"verification of the JAAS login file failed ${JaasUtils.zkSecuritySysConfigString}") - // make sure chroot path exists - chrootOption.foreach { chroot => - val zkConnForChrootCreation = config.zkConnect.substring(0, chrootIndex) - val zkClient = createZkClient(zkConnForChrootCreation, secureAclsEnabled) - zkClient.makeSurePersistentPathExists(chroot) - info(s"Created zookeeper path $chroot") - zkClient.close() - } - - _zkClient = createZkClient(config.zkConnect, secureAclsEnabled) + _zkClient = KafkaZkClient(config.zkConnect, secureAclsEnabled, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, + config.zkMaxInFlightRequests, time, name = Some("Kafka server"), zkClientConfig = Some(zkClientConfig), + createChrootIfNecessary = true) _zkClient.createTopLevelPaths() } @@ -698,7 +693,8 @@ class KafkaServer( if (alterIsrManager != null) CoreUtils.swallow(alterIsrManager.shutdown(), this) - CoreUtils.swallow(clientToControllerChannelManager.foreach(_.shutdown()), this) + if (clientToControllerChannelManager != null) + CoreUtils.swallow(clientToControllerChannelManager.shutdown(), this) if (logManager != null) CoreUtils.swallow(logManager.shutdown(), this) diff --git a/core/src/main/scala/kafka/server/ZkIsrManager.scala b/core/src/main/scala/kafka/server/ZkIsrManager.scala index 2d88aac6b4a1f..8dffcdf307cd7 100644 --- a/core/src/main/scala/kafka/server/ZkIsrManager.scala +++ b/core/src/main/scala/kafka/server/ZkIsrManager.scala @@ -55,12 +55,6 @@ class ZkIsrManager(scheduler: Scheduler, time: Time, zkClient: KafkaZkClient) ex period = isrChangeNotificationConfig.checkIntervalMs, unit = TimeUnit.MILLISECONDS) } - override def clearPending(topicPartition: TopicPartition): Unit = { - // Since we always immediately process ZK updates and never actually enqueue anything, there is nothing to - // clear here so this is a no-op. Even if there are changes that have not been propagated, the write to ZK - // has already happened, so we may as well send the notification to the controller. - } - override def submit(alterIsrItem: AlterIsrItem): Boolean = { debug(s"Writing new ISR ${alterIsrItem.leaderAndIsr.isr} to ZooKeeper with version " + s"${alterIsrItem.leaderAndIsr.zkVersion} for partition ${alterIsrItem.topicPartition}") diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala index b4c6265154f1e..70e44c89e10ea 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala @@ -16,8 +16,8 @@ */ package kafka.server.metadata -import java.util import java.util.concurrent.TimeUnit + import kafka.coordinator.group.GroupCoordinator import kafka.coordinator.transaction.TransactionCoordinator import kafka.metrics.KafkaMetricsGroup @@ -27,9 +27,12 @@ import org.apache.kafka.common.metadata.MetadataRecordType._ import org.apache.kafka.common.metadata._ import org.apache.kafka.common.protocol.ApiMessage import org.apache.kafka.common.utils.{LogContext, Time} -import org.apache.kafka.metalog.{MetaLogLeader, MetaLogListener} import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} +import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient} +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.snapshot.SnapshotReader +import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ object BrokerMetadataListener{ @@ -37,16 +40,17 @@ object BrokerMetadataListener{ val MetadataBatchSizes = "MetadataBatchSizes" } -class BrokerMetadataListener(brokerId: Int, - time: Time, - metadataCache: RaftMetadataCache, - configRepository: CachedConfigRepository, - groupCoordinator: GroupCoordinator, - replicaManager: RaftReplicaManager, - txnCoordinator: TransactionCoordinator, - threadNamePrefix: Option[String], - clientQuotaManager: ClientQuotaMetadataManager - ) extends MetaLogListener with KafkaMetricsGroup { +class BrokerMetadataListener( + brokerId: Int, + time: Time, + metadataCache: RaftMetadataCache, + configRepository: CachedConfigRepository, + groupCoordinator: GroupCoordinator, + replicaManager: RaftReplicaManager, + txnCoordinator: TransactionCoordinator, + threadNamePrefix: Option[String], + clientQuotaManager: ClientQuotaMetadataManager +) extends RaftClient.Listener[ApiMessageAndVersion] with KafkaMetricsGroup { private val logContext = new LogContext(s"[BrokerMetadataListener id=${brokerId}] ") private val log = logContext.logger(classOf[BrokerMetadataListener]) logIdent = logContext.logPrefix() @@ -73,21 +77,42 @@ class BrokerMetadataListener(brokerId: Int, /** * Handle new metadata records. */ - override def handleCommits(lastOffset: Long, records: util.List[ApiMessage]): Unit = { - eventQueue.append(new HandleCommitsEvent(lastOffset, records)) + override def handleCommit(reader: BatchReader[ApiMessageAndVersion]): Unit = { + eventQueue.append(new HandleCommitsEvent(reader)) + } + + /** + * Handle metadata snapshots + */ + override def handleSnapshot(reader: SnapshotReader[ApiMessageAndVersion]): Unit = { + // Loading snapshot on the broker is currently not supported. + reader.close(); + throw new UnsupportedOperationException(s"Loading snapshot (${reader.snapshotId()}) is not supported") } - // Visible for testing. It's useful to execute events synchronously - private[metadata] def execCommits(lastOffset: Long, records: util.List[ApiMessage]): Unit = { - new HandleCommitsEvent(lastOffset, records).run() + // Visible for testing. It's useful to execute events synchronously in order + // to make tests deterministic. This object is responsible for closing the reader. + private[metadata] def execCommits(batchReader: BatchReader[ApiMessageAndVersion]): Unit = { + new HandleCommitsEvent(batchReader).run() } - class HandleCommitsEvent(lastOffset: Long, - records: util.List[ApiMessage]) - extends EventQueue.FailureLoggingEvent(log) { + class HandleCommitsEvent( + reader: BatchReader[ApiMessageAndVersion] + ) extends EventQueue.FailureLoggingEvent(log) { override def run(): Unit = { + try { + apply(reader.next()) + } finally { + reader.close() + } + } + + private def apply(batch: Batch[ApiMessageAndVersion]): Unit = { + val records = batch.records + val lastOffset = batch.lastOffset + if (isDebugEnabled) { - debug(s"Metadata batch ${lastOffset}: handling ${records.size()} record(s).") + debug(s"Metadata batch $lastOffset: handling ${records.size()} record(s).") } val imageBuilder = MetadataImageBuilder(brokerId, log, metadataCache.currentImage()) @@ -100,37 +125,37 @@ class BrokerMetadataListener(brokerId: Int, trace("Metadata batch %d: processing [%d/%d]: %s.".format(lastOffset, index + 1, records.size(), record.toString)) } - handleMessage(imageBuilder, record, lastOffset) + handleMessage(imageBuilder, record.message, lastOffset) } catch { - case e: Exception => error(s"Unable to handle record ${index} in batch " + - s"ending at offset ${lastOffset}", e) + case e: Exception => error(s"Unable to handle record $index in batch " + + s"ending at offset $lastOffset", e) } index = index + 1 } if (imageBuilder.hasChanges) { val newImage = imageBuilder.build() if (isTraceEnabled) { - trace(s"Metadata batch ${lastOffset}: creating new metadata image ${newImage}") + trace(s"Metadata batch $lastOffset: creating new metadata image ${newImage}") } else if (isDebugEnabled) { - debug(s"Metadata batch ${lastOffset}: creating new metadata image") + debug(s"Metadata batch $lastOffset: creating new metadata image") } metadataCache.image(newImage) } else if (isDebugEnabled) { - debug(s"Metadata batch ${lastOffset}: no new metadata image required.") + debug(s"Metadata batch $lastOffset: no new metadata image required.") } if (imageBuilder.hasPartitionChanges) { if (isDebugEnabled) { - debug(s"Metadata batch ${lastOffset}: applying partition changes") + debug(s"Metadata batch $lastOffset: applying partition changes") } replicaManager.handleMetadataRecords(imageBuilder, lastOffset, RequestHandlerHelper.onLeadershipChange(groupCoordinator, txnCoordinator, _, _)) } else if (isDebugEnabled) { - debug(s"Metadata batch ${lastOffset}: no partition changes found.") + debug(s"Metadata batch $lastOffset: no partition changes found.") } _highestMetadataOffset = lastOffset val endNs = time.nanoseconds() val deltaUs = TimeUnit.MICROSECONDS.convert(endNs - startNs, TimeUnit.NANOSECONDS) - debug(s"Metadata batch ${lastOffset}: advanced highest metadata offset in ${deltaUs} " + + debug(s"Metadata batch $lastOffset: advanced highest metadata offset in ${deltaUs} " + "microseconds.") batchProcessingTimeHist.update(deltaUs) } @@ -234,21 +259,17 @@ class BrokerMetadataListener(brokerId: Int, clientQuotaManager.handleQuotaRecord(record) } - class HandleNewLeaderEvent(leader: MetaLogLeader) + class HandleNewLeaderEvent(leaderAndEpoch: LeaderAndEpoch) extends EventQueue.FailureLoggingEvent(log) { override def run(): Unit = { val imageBuilder = MetadataImageBuilder(brokerId, log, metadataCache.currentImage()) - if (leader.nodeId() < 0) { - imageBuilder.controllerId(None) - } else { - imageBuilder.controllerId(Some(leader.nodeId())) - } + imageBuilder.controllerId(leaderAndEpoch.leaderId.asScala) metadataCache.image(imageBuilder.build()) } } - override def handleNewLeader(leader: MetaLogLeader): Unit = { + override def handleLeaderChange(leader: LeaderAndEpoch): Unit = { eventQueue.append(new HandleNewLeaderEvent(leader)) } diff --git a/core/src/main/scala/kafka/tools/ClusterTool.scala b/core/src/main/scala/kafka/tools/ClusterTool.scala index a5e2a82cad698..b868f72fc7f90 100644 --- a/core/src/main/scala/kafka/tools/ClusterTool.scala +++ b/core/src/main/scala/kafka/tools/ClusterTool.scala @@ -40,7 +40,7 @@ object ClusterTool extends Logging { val clusterIdParser = subparsers.addParser("cluster-id"). help("Get information about the ID of a cluster.") val unregisterParser = subparsers.addParser("unregister"). - help("Unregister a broker..") + help("Unregister a broker.") List(clusterIdParser, unregisterParser).foreach(parser => { parser.addArgument("--bootstrap-server", "-b"). action(store()). diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 75796ad13f57c..c495dbca5fc79 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.metadata.{MetadataJsonConverters, MetadataRecordT import org.apache.kafka.common.protocol.ByteBufferAccessor import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils -import org.apache.kafka.raft.metadata.MetadataRecordSerde +import org.apache.kafka.metadata.MetadataRecordSerde import scala.jdk.CollectionConverters._ import scala.collection.mutable diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index c21f4b7e86c63..611b9268bb41d 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -19,6 +19,7 @@ package kafka.tools import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit} + import joptsimple.OptionException import kafka.network.SocketServer import kafka.raft.{KafkaRaftManager, RaftManager} @@ -35,7 +36,7 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{TopicPartition, Uuid, protocol} -import org.apache.kafka.raft.{Batch, BatchReader, RaftClient, RaftConfig} +import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient, RaftConfig} import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.snapshot.SnapshotReader @@ -165,12 +166,12 @@ class TestRaftServer( raftManager.register(this) - override def handleClaim(epoch: Int): Unit = { - eventQueue.offer(HandleClaim(epoch)) - } - - override def handleResign(epoch: Int): Unit = { - eventQueue.offer(HandleResign) + override def handleLeaderChange(newLeaderAndEpoch: LeaderAndEpoch): Unit = { + if (newLeaderAndEpoch.isLeader(config.nodeId)) { + eventQueue.offer(HandleClaim(newLeaderAndEpoch.epoch)) + } else if (claimedEpoch.isDefined) { + eventQueue.offer(HandleResign) + } } override def handleCommit(reader: BatchReader[Array[Byte]]): Unit = { diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 203f8971dcaaf..4b7bc466280b1 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -1943,7 +1943,23 @@ object KafkaZkClient { metricGroup: String = "kafka.server", metricType: String = "SessionExpireListener", name: Option[String] = None, - zkClientConfig: Option[ZKClientConfig] = None) = { + zkClientConfig: Option[ZKClientConfig] = None, + createChrootIfNecessary: Boolean = false + ): KafkaZkClient = { + if (createChrootIfNecessary) { + val chrootIndex = connectString.indexOf("/") + if (chrootIndex > 0) { + val zkConnWithoutChrootForChrootCreation = connectString.substring(0, chrootIndex) + val zkClientForChrootCreation = KafkaZkClient(zkConnWithoutChrootForChrootCreation, isSecure, sessionTimeoutMs, + connectionTimeoutMs, maxInFlightRequests, time, metricGroup, metricType, name, zkClientConfig) + try { + val chroot = connectString.substring(chrootIndex) + zkClientForChrootCreation.makeSurePersistentPathExists(chroot) + } finally { + zkClientForChrootCreation.close() + } + } + } val zooKeeperClient = new ZooKeeperClient(connectString, sessionTimeoutMs, connectionTimeoutMs, maxInFlightRequests, time, metricGroup, metricType, name, zkClientConfig) new KafkaZkClient(zooKeeperClient, isSecure, time) diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 4941ece5163cc..0f6db4a5d3dc1 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -19,7 +19,6 @@ package kafka.zk import java.nio.charset.StandardCharsets.UTF_8 import java.util import java.util.Properties - import com.fasterxml.jackson.annotation.JsonProperty import com.fasterxml.jackson.core.JsonProcessingException import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_2_7_IV0, LeaderAndIsr} @@ -40,6 +39,7 @@ import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceT import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{SecurityUtils, Time} +import org.apache.kafka.server.common.ProducerIdsBlock import org.apache.zookeeper.ZooDefs import org.apache.zookeeper.data.{ACL, Stat} @@ -766,7 +766,33 @@ object BrokerSequenceIdZNode { } object ProducerIdBlockZNode { + val CurrentVersion: Long = 1L + def path = "/latest_producer_id_block" + + def generateProducerIdBlockJson(producerIdBlock: ProducerIdsBlock): Array[Byte] = { + Json.encodeAsBytes(Map("version" -> CurrentVersion, + "broker" -> producerIdBlock.brokerId, + "block_start" -> producerIdBlock.producerIdStart.toString, + "block_end" -> producerIdBlock.producerIdEnd.toString).asJava + ) + } + + def parseProducerIdBlockData(jsonData: Array[Byte]): ProducerIdsBlock = { + val jsonDataAsString = jsonData.map(_.toChar).mkString + try { + Json.parseBytes(jsonData).map(_.asJsonObject).flatMap { js => + val brokerId = js("broker").to[Int] + val blockStart = js("block_start").to[String].toLong + val blockEnd = js("block_end").to[String].toLong + Some(new ProducerIdsBlock(brokerId, blockStart, Math.toIntExact(blockEnd - blockStart + 1))) + }.getOrElse(throw new KafkaException(s"Failed to parse the producerId block json $jsonDataAsString")) + } catch { + case e: java.lang.NumberFormatException => + // this should never happen: the written data has exceeded long type limit + throw new KafkaException(s"Read jason data $jsonDataAsString contains producerIds that have exceeded long type limit", e) + } + } } object DelegationTokenAuthZNode { diff --git a/core/src/test/java/kafka/test/ClusterConfig.java b/core/src/test/java/kafka/test/ClusterConfig.java index e7e8bf5a0234a..20b74cf43244b 100644 --- a/core/src/test/java/kafka/test/ClusterConfig.java +++ b/core/src/test/java/kafka/test/ClusterConfig.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import java.io.File; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; @@ -40,6 +41,7 @@ public class ClusterConfig { private final SecurityProtocol securityProtocol; private final String listenerName; private final File trustStoreFile; + private final String ibp; private final Properties serverProperties = new Properties(); private final Properties producerProperties = new Properties(); @@ -47,9 +49,11 @@ public class ClusterConfig { private final Properties adminClientProperties = new Properties(); private final Properties saslServerProperties = new Properties(); private final Properties saslClientProperties = new Properties(); + private final Map perBrokerOverrideProperties = new HashMap<>(); ClusterConfig(Type type, int brokers, int controllers, String name, boolean autoStart, - SecurityProtocol securityProtocol, String listenerName, File trustStoreFile) { + SecurityProtocol securityProtocol, String listenerName, File trustStoreFile, + String ibp) { this.type = type; this.brokers = brokers; this.controllers = controllers; @@ -58,6 +62,7 @@ public class ClusterConfig { this.securityProtocol = securityProtocol; this.listenerName = listenerName; this.trustStoreFile = trustStoreFile; + this.ibp = ibp; } public Type clusterType() { @@ -116,16 +121,25 @@ public Optional trustStoreFile() { return Optional.ofNullable(trustStoreFile); } + public Optional ibp() { + return Optional.ofNullable(ibp); + } + + public Properties brokerServerProperties(int brokerId) { + return perBrokerOverrideProperties.computeIfAbsent(brokerId, __ -> new Properties()); + } + public Map nameTags() { Map tags = new LinkedHashMap<>(3); name().ifPresent(name -> tags.put("Name", name)); + ibp().ifPresent(ibp -> tags.put("IBP", ibp)); tags.put("Security", securityProtocol.name()); listenerName().ifPresent(listener -> tags.put("Listener", listener)); return tags; } public ClusterConfig copyOf() { - ClusterConfig copy = new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile); + ClusterConfig copy = new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile, ibp); copy.serverProperties.putAll(serverProperties); copy.producerProperties.putAll(producerProperties); copy.consumerProperties.putAll(consumerProperties); @@ -151,6 +165,7 @@ public static class Builder { private SecurityProtocol securityProtocol; private String listenerName; private File trustStoreFile; + private String ibp; Builder(Type type, int brokers, int controllers, boolean autoStart, SecurityProtocol securityProtocol) { this.type = type; @@ -200,8 +215,13 @@ public Builder trustStoreFile(File trustStoreFile) { return this; } + public Builder ibp(String ibp) { + this.ibp = ibp; + return this; + } + public ClusterConfig build() { - return new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile); + return new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile, ibp); } } } diff --git a/core/src/test/java/kafka/test/ClusterInstance.java b/core/src/test/java/kafka/test/ClusterInstance.java index cac986fdc317c..021db5a34381e 100644 --- a/core/src/test/java/kafka/test/ClusterInstance.java +++ b/core/src/test/java/kafka/test/ClusterInstance.java @@ -94,4 +94,6 @@ default Admin createAdminClient() { void start(); void stop(); + + void rollingBrokerRestart(); } diff --git a/core/src/test/java/kafka/test/annotation/ClusterTest.java b/core/src/test/java/kafka/test/annotation/ClusterTest.java index 687255c3c4705..11336ab87a15f 100644 --- a/core/src/test/java/kafka/test/annotation/ClusterTest.java +++ b/core/src/test/java/kafka/test/annotation/ClusterTest.java @@ -40,5 +40,6 @@ String name() default ""; SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT; String listener() default ""; + String ibp() default ""; ClusterConfigProperty[] serverProperties() default {}; } diff --git a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java index 94eefbd91cb81..293f00b035ca5 100644 --- a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java +++ b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java @@ -183,7 +183,7 @@ private void processClusterTest(ExtensionContext context, ClusterTest annot, Clu if (!annot.name().isEmpty()) { builder.name(annot.name()); } else { - builder.name(context.getDisplayName()); + builder.name(context.getRequiredTestMethod().getName()); } if (!annot.listener().isEmpty()) { builder.listenerName(annot.listener()); @@ -194,6 +194,10 @@ private void processClusterTest(ExtensionContext context, ClusterTest annot, Clu properties.put(property.key(), property.value()); } + if (!annot.ibp().isEmpty()) { + builder.ibp(annot.ibp()); + } + ClusterConfig config = builder.build(); config.serverProperties().putAll(properties); type.invocationContexts(config, testInvocations); diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index b4e50c59bfe98..fc6b55736c3fc 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -66,8 +66,8 @@ public RaftClusterInvocationContext(ClusterConfig clusterConfig) { @Override public String getDisplayName(int invocationIndex) { String clusterDesc = clusterConfig.nameTags().entrySet().stream() - .map(Object::toString) - .collect(Collectors.joining(", ")); + .map(Object::toString) + .collect(Collectors.joining(", ")); return String.format("[%d] Type=Raft, %s", invocationIndex, clusterDesc); } @@ -75,10 +75,14 @@ public String getDisplayName(int invocationIndex) { public List getAdditionalExtensions() { return Arrays.asList( (BeforeTestExecutionCallback) context -> { - KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder(). + TestKitNodes nodes = new TestKitNodes.Builder(). setNumBrokerNodes(clusterConfig.numBrokers()). - setNumControllerNodes(clusterConfig.numControllers()).build()); + setNumControllerNodes(clusterConfig.numControllers()).build(); + nodes.brokerNodes().forEach((brokerId, brokerNode) -> { + clusterConfig.brokerServerProperties(brokerId).forEach( + (key, value) -> brokerNode.propertyOverrides().put(key.toString(), value.toString())); + }); + KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder(nodes); // Copy properties into the TestKit builder clusterConfig.serverProperties().forEach((key, value) -> builder.setConfigProp(key.toString(), value.toString())); @@ -192,5 +196,10 @@ public void stop() { } } } + + @Override + public void rollingBrokerRestart() { + throw new UnsupportedOperationException("Restarting Raft servers is not yet supported."); + } } -} \ No newline at end of file +} diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index 8d4660b77382e..cd8cdc11a800d 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -19,6 +19,7 @@ import kafka.api.IntegrationTestHarness; import kafka.network.SocketServer; +import kafka.server.KafkaConfig; import kafka.server.KafkaServer; import kafka.utils.TestUtils; import org.apache.kafka.clients.admin.Admin; @@ -32,6 +33,7 @@ import org.junit.jupiter.api.extension.TestTemplateInvocationContext; import scala.Option; import scala.collection.JavaConverters; +import scala.collection.Seq; import scala.compat.java8.OptionConverters; import java.io.File; @@ -89,9 +91,20 @@ public List getAdditionalExtensions() { // This is what tests normally extend from to start a cluster, here we create it anonymously and // configure the cluster using values from ClusterConfig IntegrationTestHarness cluster = new IntegrationTestHarness() { + + @Override + public void modifyConfigs(Seq props) { + super.modifyConfigs(props); + for (int i = 0; i < props.length(); i++) { + props.apply(i).putAll(clusterConfig.brokerServerProperties(i)); + } + } + @Override public Properties serverConfig() { - return clusterConfig.serverProperties(); + Properties props = clusterConfig.serverProperties(); + clusterConfig.ibp().ifPresent(ibp -> props.put(KafkaConfig.InterBrokerProtocolVersionProp(), ibp)); + return props; } @Override @@ -248,5 +261,16 @@ public void stop() { clusterReference.get().tearDown(); } } + + @Override + public void rollingBrokerRestart() { + if (!started.get()) { + throw new IllegalStateException("Tried to restart brokers but the cluster has not been started!"); + } + for (int i = 0; i < clusterReference.get().brokerCount(); i++) { + clusterReference.get().killBroker(i); + } + clusterReference.get().restartDeadBrokers(true); + } } } diff --git a/core/src/test/java/kafka/testkit/BrokerNode.java b/core/src/test/java/kafka/testkit/BrokerNode.java index 0b5859404b346..32bd51b4b56dc 100644 --- a/core/src/test/java/kafka/testkit/BrokerNode.java +++ b/core/src/test/java/kafka/testkit/BrokerNode.java @@ -21,7 +21,9 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; public class BrokerNode implements TestKitNode { public static class Builder { @@ -68,6 +70,7 @@ public BrokerNode build() { private final Uuid incarnationId; private final String metadataDirectory; private final List logDataDirectories; + private final Map propertyOverrides; BrokerNode(int id, Uuid incarnationId, @@ -77,6 +80,7 @@ public BrokerNode build() { this.incarnationId = incarnationId; this.metadataDirectory = metadataDirectory; this.logDataDirectories = new ArrayList<>(logDataDirectories); + this.propertyOverrides = new HashMap<>(); } @Override @@ -96,4 +100,8 @@ public String metadataDirectory() { public List logDataDirectories() { return logDataDirectories; } + + public Map propertyOverrides() { + return propertyOverrides; + } } diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 87083a240c2d0..79d7bda1accbd 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -36,11 +36,9 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.controller.Controller; -import org.apache.kafka.server.common.ApiMessageAndVersion; -import org.apache.kafka.metalog.MetaLogManager; +import org.apache.kafka.metadata.MetadataRecordSerde; import org.apache.kafka.raft.RaftConfig; -import org.apache.kafka.raft.metadata.MetaLogRaftShim; -import org.apache.kafka.raft.metadata.MetadataRecordSerde; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.test.TestUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -175,11 +173,9 @@ public KafkaClusterTestKit build() throws Exception { KafkaRaftManager raftManager = new KafkaRaftManager<>( metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftServer.MetadataTopicId(), Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future); - MetaLogManager metaLogShim = new MetaLogRaftShim(raftManager.kafkaRaftClient(), config.nodeId()); ControllerServer controller = new ControllerServer( nodes.controllerProperties(node.id()), config, - metaLogShim, raftManager, Time.SYSTEM, new Metrics(), @@ -220,6 +216,7 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS // Just like above, we set a placeholder voter list here until we // find out what ports the controllers picked. props.put(RaftConfig.QUORUM_VOTERS_CONFIG, uninitializedQuorumVotersString); + props.putAll(node.propertyOverrides()); KafkaConfig config = new KafkaConfig(props, false, Option.empty()); String threadNamePrefix = String.format("broker%d_", node.id()); @@ -228,11 +225,10 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS KafkaRaftManager raftManager = new KafkaRaftManager<>( metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftServer.MetadataTopicId(), Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future); - MetaLogManager metaLogShim = new MetaLogRaftShim(raftManager.kafkaRaftClient(), config.nodeId()); BrokerServer broker = new BrokerServer( config, nodes.brokerProperties(node.id()), - metaLogShim, + raftManager, Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala new file mode 100644 index 0000000000000..6c7c248e63188 --- /dev/null +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.coordinator.transaction + +import kafka.network.SocketServer +import kafka.server.{IntegrationTestUtils, KafkaConfig} +import kafka.test.annotation.{AutoStart, ClusterTest, ClusterTests, Type} +import kafka.test.junit.ClusterTestExtensions +import kafka.test.{ClusterConfig, ClusterInstance} +import org.apache.kafka.common.message.InitProducerIdRequestData +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.record.RecordBatch +import org.apache.kafka.common.requests.{InitProducerIdRequest, InitProducerIdResponse} +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.extension.ExtendWith + +import java.util.stream.{Collectors, IntStream} +import scala.jdk.CollectionConverters._ + +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +class ProducerIdsIntegrationTest { + + @BeforeEach + def setup(clusterConfig: ClusterConfig): Unit = { + clusterConfig.serverProperties().put(KafkaConfig.TransactionsTopicPartitionsProp, "1") + clusterConfig.serverProperties().put(KafkaConfig.TransactionsTopicReplicationFactorProp, "3") + } + + @ClusterTests(Array( + new ClusterTest(clusterType = Type.ZK, brokers = 3, ibp = "2.8"), + new ClusterTest(clusterType = Type.ZK, brokers = 3, ibp = "3.0-IV0") + )) + def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = { + verifyUniqueIds(clusterInstance) + } + + @ClusterTest(clusterType = Type.ZK, brokers = 3, autoStart = AutoStart.NO) + def testUniqueProducerIdsBumpIBP(clusterInstance: ClusterInstance): Unit = { + clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "2.8") + clusterInstance.config().brokerServerProperties(0).put(KafkaConfig.InterBrokerProtocolVersionProp, "3.0-IV0") + clusterInstance.start() + verifyUniqueIds(clusterInstance) + clusterInstance.stop() + } + + private def verifyUniqueIds(clusterInstance: ClusterInstance): Unit = { + // Request enough PIDs from each broker to ensure each broker generates two PID blocks + val ids = clusterInstance.brokerSocketServers().stream().flatMap( broker => { + IntStream.range(0, 1001).parallel().mapToObj( _ => nextProducerId(broker, clusterInstance.clientListener())) + }).collect(Collectors.toList[Long]).asScala.toSeq + + assertEquals(3003, ids.size, "Expected exactly 3003 IDs") + assertEquals(ids.size, ids.distinct.size, "Found duplicate producer IDs") + } + + private def nextProducerId(broker: SocketServer, listener: ListenerName): Long = { + val data = new InitProducerIdRequestData() + .setProducerEpoch(RecordBatch.NO_PRODUCER_EPOCH) + .setProducerId(RecordBatch.NO_PRODUCER_ID) + .setTransactionalId(null) + .setTransactionTimeoutMs(10) + val request = new InitProducerIdRequest.Builder(data).build() + + val response = IntegrationTestUtils.connectAndReceive[InitProducerIdResponse](request, + destination = broker, + listenerName = listener) + response.data().producerId() + } +} diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index f9ec73e453f78..c76ccaa11be83 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -99,9 +99,7 @@ final class KafkaMetadataLogTest { snapshot.freeze() } - TestUtils.resource(log.readSnapshot(snapshotId).get()) { snapshot => - assertEquals(0, snapshot.sizeInBytes()) - } + assertEquals(0, log.readSnapshot(snapshotId).get().sizeInBytes()) } @Test diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 893562439b561..0062723bb5c97 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -282,8 +282,9 @@ class PartitionLockTest extends Logging { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None) val logDirFailureChannel = new LogDirFailureChannel(1) val segments = new LogSegments(log.topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion) - val producerStateManager = new ProducerStateManager(log.topicPartition, log.dir, log.maxProducerIdExpirationMs) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion, "") + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val producerStateManager = new ProducerStateManager(log.topicPartition, log.dir, maxProducerIdExpirationMs) val offsets = LogLoader.load(LoadLogParams( log.dir, log.topicPartition, @@ -295,7 +296,7 @@ class PartitionLockTest extends Logging { segments, 0L, 0L, - log.maxProducerIdExpirationMs, + maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) new SlowLog(log, segments, offsets, leaderEpochCache, producerStateManager, mockTime, logDirFailureChannel, appendSemaphore) @@ -379,7 +380,6 @@ class PartitionLockTest extends Logging { mockTime.scheduler, new BrokerTopicStats, mockTime, - log.maxProducerIdExpirationMs, log.producerIdExpirationCheckIntervalMs, log.topicPartition, leaderEpochCache, diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 52dea3807f186..5098ee50c7dda 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -236,8 +236,9 @@ class PartitionTest extends AbstractPartitionTest { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None) val logDirFailureChannel = new LogDirFailureChannel(1) val segments = new LogSegments(log.topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion) - val producerStateManager = new ProducerStateManager(log.topicPartition, log.dir, log.maxProducerIdExpirationMs) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion, "") + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val producerStateManager = new ProducerStateManager(log.topicPartition, log.dir, maxProducerIdExpirationMs) val offsets = LogLoader.load(LoadLogParams( log.dir, log.topicPartition, @@ -249,7 +250,7 @@ class PartitionTest extends AbstractPartitionTest { segments, 0L, 0L, - log.maxProducerIdExpirationMs, + maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) new SlowLog(log, segments, offsets, leaderEpochCache, producerStateManager, mockTime, logDirFailureChannel, appendSemaphore) @@ -1288,6 +1289,66 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(0L, partition.localLogOrException.highWatermark) } + @Test + def testAlterIsrLeaderAndIsrRace(): Unit = { + val log = logManager.getOrCreateLog(topicPartition, topicId = None) + seedLogData(log, numRecords = 10, leaderEpoch = 4) + + val controllerEpoch = 0 + val leaderEpoch = 5 + val remoteBrokerId = brokerId + 1 + val replicas = List(brokerId, remoteBrokerId) + val isr = List[Integer](brokerId, remoteBrokerId).asJava + + val initializeTimeMs = time.milliseconds() + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) + assertTrue(partition.makeLeader( + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(true), + offsetCheckpoints, None), "Expected become leader transition to succeed") + assertEquals(Set(brokerId, remoteBrokerId), partition.isrState.isr) + assertEquals(0L, partition.localLogOrException.highWatermark) + + val remoteReplica = partition.getReplica(remoteBrokerId).get + assertEquals(initializeTimeMs, remoteReplica.lastCaughtUpTimeMs) + assertEquals(LogOffsetMetadata.UnknownOffsetMetadata.messageOffset, remoteReplica.logEndOffset) + assertEquals(Log.UnknownOffset, remoteReplica.logStartOffset) + + // Shrink the ISR + time.sleep(partition.replicaLagTimeMaxMs + 1) + partition.maybeShrinkIsr() + assertTrue(partition.isrState.isInflight) + + // Become leader again, reset the ISR state + assertFalse(partition.makeLeader( + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(2) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(false), + offsetCheckpoints, None)) + assertFalse(partition.isrState.isInflight, "ISR should be committed and not inflight") + + // Try the shrink again, should not submit until AlterIsr response arrives + time.sleep(partition.replicaLagTimeMaxMs + 1) + partition.maybeShrinkIsr() + assertFalse(partition.isrState.isInflight, "ISR should still be committed and not inflight") + + // Complete the AlterIsr update and now we can make modifications again + alterIsrManager.completeIsrUpdate(10) + partition.maybeShrinkIsr() + assertTrue(partition.isrState.isInflight, "ISR should be pending a shrink") + } + @Test def testShouldNotShrinkIsrIfPreviousFetchIsCaughtUp(): Unit = { val log = logManager.getOrCreateLog(topicPartition, topicId = None) @@ -1635,6 +1696,13 @@ class PartitionTest extends AbstractPartitionTest { case _ => fail("Expected a committed ISR following Zk expansion") } + // Verify duplicate request. In-flight state should be reset even though version hasn't changed. + doAnswer(_ => (true, 2)) + .when(kafkaZkClient) + .conditionalUpdatePath(anyString(), any(), ArgumentMatchers.eq(2), any()) + partition.expandIsr(follower3) + TestUtils.waitUntilTrue(() => !partition.isrState.isInflight, "Expected ISR state to be committed", 100) + scheduler.shutdown() } @@ -1976,7 +2044,6 @@ class PartitionTest extends AbstractPartitionTest { mockTime.scheduler, new BrokerTopicStats, mockTime, - log.maxProducerIdExpirationMs, log.producerIdExpirationCheckIntervalMs, log.topicPartition, leaderEpochCache, diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 6892e2637a449..f34acd9b2c29e 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -18,10 +18,10 @@ package kafka.controller import java.util.Properties -import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue} - +import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingQueue, TimeUnit} import com.yammer.metrics.core.Timer import kafka.api.{ApiVersion, KAFKA_2_6_IV0, KAFKA_2_7_IV0, LeaderAndIsr} +import kafka.controller.KafkaController.AlterIsrCallback import kafka.metrics.KafkaYammerMetrics import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.{LogCaptureAppender, TestUtils} @@ -849,6 +849,66 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { latch.await() } + @Test + def testAlterIsrErrors(): Unit = { + servers = makeServers(1) + val controllerId = TestUtils.waitUntilControllerElected(zkClient) + val tp = new TopicPartition("t", 0) + val assignment = Map(tp.partition -> Seq(controllerId)) + TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) + val controller = getController().kafkaController + var future = captureAlterIsrError(controllerId, controller.brokerEpoch - 1, + Map(tp -> LeaderAndIsr(controllerId, List(controllerId)))) + var capturedError = future.get(5, TimeUnit.SECONDS) + assertEquals(Errors.STALE_BROKER_EPOCH, capturedError) + + future = captureAlterIsrError(99, controller.brokerEpoch, + Map(tp -> LeaderAndIsr(controllerId, List(controllerId)))) + capturedError = future.get(5, TimeUnit.SECONDS) + assertEquals(Errors.STALE_BROKER_EPOCH, capturedError) + + val unknownTopicPartition = new TopicPartition("unknown", 99) + future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch, + Map(unknownTopicPartition -> LeaderAndIsr(controllerId, List(controllerId))), unknownTopicPartition) + capturedError = future.get(5, TimeUnit.SECONDS) + assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, capturedError) + + future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch, + Map(tp -> LeaderAndIsr(controllerId, 1, List(controllerId), 99)), tp) + capturedError = future.get(5, TimeUnit.SECONDS) + assertEquals(Errors.INVALID_UPDATE_VERSION, capturedError) + } + + def captureAlterIsrError(brokerId: Int, brokerEpoch: Long, isrsToAlter: Map[TopicPartition, LeaderAndIsr]): CompletableFuture[Errors] = { + val future = new CompletableFuture[Errors]() + val controller = getController().kafkaController + val callback: AlterIsrCallback = { + case Left(_: Map[TopicPartition, Either[Errors, LeaderAndIsr]]) => + future.completeExceptionally(new AssertionError(s"Should have seen top-level error")) + case Right(error: Errors) => + future.complete(error) + } + controller.eventManager.put(AlterIsrReceived(brokerId, brokerEpoch, isrsToAlter, callback)) + future + } + + def captureAlterIsrPartitionError(brokerId: Int, brokerEpoch: Long, isrsToAlter: Map[TopicPartition, LeaderAndIsr], tp: TopicPartition): CompletableFuture[Errors] = { + val future = new CompletableFuture[Errors]() + val controller = getController().kafkaController + val callback: AlterIsrCallback = { + case Left(partitionResults: Map[TopicPartition, Either[Errors, LeaderAndIsr]]) => + partitionResults.get(tp) match { + case Some(Left(error: Errors)) => future.complete(error) + case Some(Right(_: LeaderAndIsr)) => future.completeExceptionally(new AssertionError(s"Should have seen an error for $tp in result")) + case None => future.completeExceptionally(new AssertionError(s"Should have seen $tp in result")) + } + case Right(_: Errors) => + future.completeExceptionally(new AssertionError(s"Should not seen top-level error")) + } + controller.eventManager.put(AlterIsrReceived(brokerId, brokerEpoch, isrsToAlter, callback)) + future + } + @Test def testTopicIdsAreAdded(): Unit = { servers = makeServers(1) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala index be3e8144f6859..9232bf03c0e0e 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala @@ -16,23 +16,44 @@ */ package kafka.coordinator.transaction -import kafka.zk.KafkaZkClient +import kafka.server.BrokerToControllerChannelManager +import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode} import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.message.AllocateProducerIdsResponseData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.AllocateProducerIdsResponse +import org.apache.kafka.server.common.ProducerIdsBlock import org.easymock.{Capture, EasyMock} -import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{EnumSource, ValueSource} + +import java.util.stream.IntStream class ProducerIdManagerTest { - private val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) + var brokerToController: BrokerToControllerChannelManager = EasyMock.niceMock(classOf[BrokerToControllerChannelManager]) + val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) - @AfterEach - def tearDown(): Unit = { - EasyMock.reset(zkClient) + // Mutable test implementation that lets us easily set the idStart and error + class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: Int, var error: Errors = Errors.NONE) + extends RPCProducerIdManager(brokerId, () => 1, brokerToController, 100) { + + override private[transaction] def sendRequest(): Unit = { + if (error == Errors.NONE) { + handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse( + new AllocateProducerIdsResponseData().setProducerIdStart(idStart).setProducerIdLen(idLen))) + idStart += idLen + } else { + handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse( + new AllocateProducerIdsResponseData().setErrorCode(error.code))) + } + } } @Test - def testGetProducerId(): Unit = { + def testGetProducerIdZk(): Unit = { var zkVersion: Option[Int] = None var data: Array[Byte] = null EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(() => @@ -53,34 +74,71 @@ class ProducerIdManagerTest { EasyMock.replay(zkClient) - val manager1 = new ProducerIdManager(0, zkClient) - val manager2 = new ProducerIdManager(1, zkClient) + val manager1 = new ZkProducerIdManager(0, zkClient) + val manager2 = new ZkProducerIdManager(1, zkClient) val pid1 = manager1.generateProducerId() val pid2 = manager2.generateProducerId() assertEquals(0, pid1) - assertEquals(ProducerIdManager.PidBlockSize, pid2) + assertEquals(ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE, pid2) - for (i <- 1L until ProducerIdManager.PidBlockSize) + for (i <- 1L until ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) assertEquals(pid1 + i, manager1.generateProducerId()) - for (i <- 1L until ProducerIdManager.PidBlockSize) + for (i <- 1L until ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) assertEquals(pid2 + i, manager2.generateProducerId()) - assertEquals(pid2 + ProducerIdManager.PidBlockSize, manager1.generateProducerId()) - assertEquals(pid2 + ProducerIdManager.PidBlockSize * 2, manager2.generateProducerId()) + assertEquals(pid2 + ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE, manager1.generateProducerId()) + assertEquals(pid2 + ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE * 2, manager2.generateProducerId()) + + EasyMock.reset(zkClient) } @Test - def testExceedProducerIdLimit(): Unit = { + def testExceedProducerIdLimitZk(): Unit = { EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(() => { - val json = ProducerIdManager.generateProducerIdBlockJson( - ProducerIdBlock(0, Long.MaxValue - ProducerIdManager.PidBlockSize, Long.MaxValue)) + val json = ProducerIdBlockZNode.generateProducerIdBlockJson( + new ProducerIdsBlock(0, Long.MaxValue - ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE)) (Some(json), 0) }).anyTimes() EasyMock.replay(zkClient) - assertThrows(classOf[KafkaException], () => new ProducerIdManager(0, zkClient)) + assertThrows(classOf[KafkaException], () => new ZkProducerIdManager(0, zkClient)) + } + + @ParameterizedTest + @ValueSource(ints = Array(1, 2, 10)) + def testContiguousIds(idBlockLen: Int): Unit = { + val manager = new MockProducerIdManager(0, 0, idBlockLen) + + IntStream.range(0, idBlockLen * 3).forEach { i => + assertEquals(i, manager.generateProducerId()) + } + } + + @ParameterizedTest + @EnumSource(value = classOf[Errors], names = Array("UNKNOWN_SERVER_ERROR", "INVALID_REQUEST")) + def testUnrecoverableErrors(error: Errors): Unit = { + val manager = new MockProducerIdManager(0, 0, 1) + assertEquals(0, manager.generateProducerId()) + + manager.error = error + assertThrows(classOf[Throwable], () => manager.generateProducerId()) + + manager.error = Errors.NONE + assertEquals(1, manager.generateProducerId()) + } + + @Test + def testInvalidRanges(): Unit = { + var manager = new MockProducerIdManager(0, -1, 10) + assertThrows(classOf[KafkaException], () => manager.generateProducerId()) + + manager = new MockProducerIdManager(0, 0, -1) + assertThrows(classOf[KafkaException], () => manager.generateProducerId()) + + manager = new MockProducerIdManager(0, Long.MaxValue-1, 10) + assertThrows(classOf[KafkaException], () => manager.generateProducerId()) } } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index e3cc4463771d1..e02c2fe3c320b 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -76,7 +76,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren for (i <- 0 until numPartitions) txnStateManager.addLoadedTransactionsToCache(i, coordinatorEpoch, new Pool[String, TransactionMetadata]()) - val pidGenerator: ProducerIdGenerator = EasyMock.createNiceMock(classOf[ProducerIdGenerator]) + val pidGenerator: ProducerIdManager = EasyMock.createNiceMock(classOf[ProducerIdManager]) EasyMock.expect(pidGenerator.generateProducerId()) .andAnswer(() => if (bumpProducerId) producerId + 1 else producerId) .anyTimes() diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index ebbc8498b7b33..38e8e711975ef 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -34,7 +34,7 @@ class TransactionCoordinatorTest { val time = new MockTime() var nextPid: Long = 0L - val pidGenerator: ProducerIdGenerator = EasyMock.createNiceMock(classOf[ProducerIdGenerator]) + val pidGenerator: ProducerIdManager = EasyMock.createNiceMock(classOf[ProducerIdManager]) val transactionManager: TransactionStateManager = EasyMock.createNiceMock(classOf[TransactionStateManager]) val transactionMarkerChannelManager: TransactionMarkerChannelManager = EasyMock.createNiceMock(classOf[TransactionMarkerChannelManager]) val capturedTxn: Capture[TransactionMetadata] = EasyMock.newCapture() diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 5369fdd66fae7..6fc18f0d10c68 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -161,8 +161,19 @@ abstract class KafkaServerTestHarness extends ZooKeeperTestHarness { /** * Restart any dead brokers */ - def restartDeadBrokers(): Unit = { + def restartDeadBrokers(reconfigure: Boolean = false): Unit = { + if (reconfigure) { + instanceConfigs = null + } for(i <- servers.indices if !alive(i)) { + if (reconfigure) { + servers(i) = TestUtils.createServer( + configs(i), + time = brokerTime(configs(i).brokerId), + threadNamePrefix = None, + enableForwarding + ) + } servers(i).startup() alive(i) = true } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index cdaa89de9e34e..fbdbe207a1898 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -100,7 +100,7 @@ class LogCleanerManagerTest extends Logging { val config = createLowRetentionLogConfig(logSegmentSize, LogConfig.Compact) val maxProducerIdExpirationMs = 60 * 60 * 1000 val segments = new LogSegments(tp) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(tpDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(tpDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, "") val producerStateManager = new ProducerStateManager(topicPartition, tpDir, maxProducerIdExpirationMs) val offsets = LogLoader.load(LoadLogParams( tpDir, @@ -119,7 +119,7 @@ class LogCleanerManagerTest extends Logging { // the exception should be caught and the partition that caused it marked as uncleanable class LogMock(dir: File, config: LogConfig, offsets: LoadedLogOffsets) extends Log(dir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, - offsets.nextOffsetMetadata, time.scheduler, new BrokerTopicStats, time, maxProducerIdExpirationMs, + offsets.nextOffsetMetadata, time.scheduler, new BrokerTopicStats, time, LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, producerStateManager, logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { // Throw an error in getFirstBatchTimestampForSegments since it is called in grabFilthiestLog() diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index de55724d6ba71..c137eab0a4194 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -72,7 +72,7 @@ class LogCleanerTest { // append messages to the log until we have four segments while(log.numberOfSegments < 4) log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), leaderEpoch = 0) - val keysFound = LogTest.keysInLog(log) + val keysFound = LogTestUtils.keysInLog(log) assertEquals(0L until log.logEndOffset, keysFound) // pretend we have the following keys @@ -85,8 +85,8 @@ class LogCleanerTest { val stats = new CleanerStats() val expectedBytesRead = segments.map(_.size).sum cleaner.cleanSegments(log, segments, map, 0L, stats, new CleanedTransactionMetadata) - val shouldRemain = LogTest.keysInLog(log).filter(!keys.contains(_)) - assertEquals(shouldRemain, LogTest.keysInLog(log)) + val shouldRemain = LogTestUtils.keysInLog(log).filter(!keys.contains(_)) + assertEquals(shouldRemain, LogTestUtils.keysInLog(log)) assertEquals(expectedBytesRead, stats.bytesRead) } @@ -105,7 +105,7 @@ class LogCleanerTest { val logDirFailureChannel = new LogDirFailureChannel(10) val maxProducerIdExpirationMs = 60 * 60 * 1000 val logSegments = new LogSegments(topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, "") val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) val offsets = LogLoader.load(LoadLogParams( dir, @@ -131,7 +131,6 @@ class LogCleanerTest { scheduler = time.scheduler, brokerTopicStats = new BrokerTopicStats, time, - maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, @@ -231,7 +230,7 @@ class LogCleanerTest { cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) assertEquals(List(2, 5, 7), lastOffsetsPerBatchInLog(log)) assertEquals(Map(pid1 -> 2, pid2 -> 2, pid3 -> 1), lastSequencesInLog(log)) - assertEquals(List(2, 3, 1, 4), LogTest.keysInLog(log)) + assertEquals(List(2, 3, 1, 4), LogTestUtils.keysInLog(log)) assertEquals(List(1, 3, 6, 7), offsetsInLog(log)) // we have to reload the log to validate that the cleaner maintained sequence numbers correctly @@ -263,7 +262,7 @@ class LogCleanerTest { cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) assertEquals(Map(pid1 -> 2, pid2 -> 2, pid3 -> 1, pid4 -> 0), lastSequencesInLog(log)) assertEquals(List(2, 5, 7, 8), lastOffsetsPerBatchInLog(log)) - assertEquals(List(3, 1, 4, 2), LogTest.keysInLog(log)) + assertEquals(List(3, 1, 4, 2), LogTestUtils.keysInLog(log)) assertEquals(List(3, 6, 7, 8), offsetsInLog(log)) reloadLog() @@ -300,7 +299,7 @@ class LogCleanerTest { log.roll() cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) - assertEquals(List(3, 2), LogTest.keysInLog(log)) + assertEquals(List(3, 2), LogTestUtils.keysInLog(log)) assertEquals(List(3, 6, 7, 8, 9), offsetsInLog(log)) // ensure the transaction index is still correct @@ -340,7 +339,7 @@ class LogCleanerTest { // we have only cleaned the records in the first segment val dirtyOffset = cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset))._1 - assertEquals(List(2, 3, 5, 6, 6, 7, 7, 8, 8, 9, 9, 10), LogTest.keysInLog(log)) + assertEquals(List(2, 3, 5, 6, 6, 7, 7, 8, 8, 9, 9, 10), LogTestUtils.keysInLog(log)) log.roll() @@ -350,7 +349,7 @@ class LogCleanerTest { // finally only the keys from pid3 should remain cleaner.clean(LogToClean(new TopicPartition("test", 0), log, dirtyOffset, log.activeSegment.baseOffset)) - assertEquals(List(2, 3, 6, 7, 8, 9, 11, 12), LogTest.keysInLog(log)) + assertEquals(List(2, 3, 6, 7, 8, 9, 11, 12), LogTestUtils.keysInLog(log)) } @Test @@ -374,7 +373,7 @@ class LogCleanerTest { // cannot remove the marker in this pass because there are still valid records var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) + assertEquals(List(1, 3, 2), LogTestUtils.keysInLog(log)) assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) appendProducer(Seq(1, 3)) @@ -383,17 +382,17 @@ class LogCleanerTest { // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) // delete horizon forced to 0 to verify marker is not removed early dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) } @@ -422,11 +421,11 @@ class LogCleanerTest { log.roll() cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) - assertEquals(List(2), LogTest.keysInLog(log)) + assertEquals(List(2), LogTestUtils.keysInLog(log)) assertEquals(List(1, 3, 4), offsetsInLog(log)) cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) - assertEquals(List(2), LogTest.keysInLog(log)) + assertEquals(List(2), LogTestUtils.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) } @@ -463,14 +462,14 @@ class LogCleanerTest { // first time through the records are removed // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(2, 3), LogTestUtils.keysInLog(log)) assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) // the empty batch remains if cleaned again because it still holds the last sequence // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(2, 3), LogTestUtils.keysInLog(log)) assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) @@ -484,13 +483,13 @@ class LogCleanerTest { // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) + assertEquals(List(2, 3, 1), LogTestUtils.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) + assertEquals(List(2, 3, 1), LogTestUtils.keysInLog(log)) assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) } @@ -515,14 +514,14 @@ class LogCleanerTest { // first time through the control batch is retained as an empty batch // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(2, 3), LogTestUtils.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) // the empty control batch does not cause an exception when cleaned // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(2, 3), LogTestUtils.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) } @@ -601,12 +600,12 @@ class LogCleanerTest { // delete horizon set to 0 to verify marker is not removed early val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 - assertEquals(List(3), LogTest.keysInLog(log)) + assertEquals(List(3), LogTestUtils.keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) - assertEquals(List(3), LogTest.keysInLog(log)) + assertEquals(List(3), LogTestUtils.keysInLog(log)) assertEquals(List(4, 5), offsetsInLog(log)) } @@ -680,14 +679,14 @@ class LogCleanerTest { // first time through the records are removed var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 assertAbortedTransactionIndexed() - assertEquals(List(), LogTest.keysInLog(log)) + assertEquals(List(), LogTestUtils.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is retained assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained // the empty batch remains if cleaned again because it still holds the last sequence dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 assertAbortedTransactionIndexed() - assertEquals(List(), LogTest.keysInLog(log)) + assertEquals(List(), LogTestUtils.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained @@ -697,12 +696,12 @@ class LogCleanerTest { dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 assertAbortedTransactionIndexed() - assertEquals(List(1), LogTest.keysInLog(log)) + assertEquals(List(1), LogTestUtils.keysInLog(log)) assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(1), LogTest.keysInLog(log)) + assertEquals(List(1), LogTestUtils.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // abort marker is gone assertEquals(List(3), lastOffsetsPerBatchInLog(log)) @@ -726,7 +725,7 @@ class LogCleanerTest { while(log.numberOfSegments < 2) log.appendAsLeader(record(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)), leaderEpoch = 0) - val keysFound = LogTest.keysInLog(log) + val keysFound = LogTestUtils.keysInLog(log) assertEquals(0L until log.logEndOffset, keysFound) // pretend we have the following keys @@ -737,8 +736,8 @@ class LogCleanerTest { // clean the log val stats = new CleanerStats() cleaner.cleanSegments(log, Seq(log.logSegments.head), map, 0L, stats, new CleanedTransactionMetadata) - val shouldRemain = LogTest.keysInLog(log).filter(!keys.contains(_)) - assertEquals(shouldRemain, LogTest.keysInLog(log)) + val shouldRemain = LogTestUtils.keysInLog(log).filter(!keys.contains(_)) + assertEquals(shouldRemain, LogTestUtils.keysInLog(log)) } /** @@ -750,8 +749,8 @@ class LogCleanerTest { val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) cleaner.cleanSegments(log, Seq(log.logSegments.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata) - val shouldRemain = LogTest.keysInLog(log).filter(k => !offsetMap.map.containsKey(k.toString)) - assertEquals(shouldRemain, LogTest.keysInLog(log)) + val shouldRemain = LogTestUtils.keysInLog(log).filter(k => !offsetMap.map.containsKey(k.toString)) + assertEquals(shouldRemain, LogTestUtils.keysInLog(log)) } /** @@ -798,7 +797,7 @@ class LogCleanerTest { while(log.numberOfSegments < 2) log.appendAsLeader(record(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)), leaderEpoch = 0) - val keysFound = LogTest.keysInLog(log) + val keysFound = LogTestUtils.keysInLog(log) assertEquals(0L until log.logEndOffset, keysFound) // Decrease the log's max message size @@ -835,7 +834,7 @@ class LogCleanerTest { log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), leaderEpoch = 0) cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0, log.activeSegment.baseOffset)) - val keys = LogTest.keysInLog(log).toSet + val keys = LogTestUtils.keysInLog(log).toSet assertTrue((0 until leo.toInt by 2).forall(!keys.contains(_)), "None of the keys we deleted should still exist.") } @@ -887,7 +886,7 @@ class LogCleanerTest { cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) assertEquals(List(1, 3, 4), lastOffsetsPerBatchInLog(log)) assertEquals(Map(1L -> 0, 2L -> 1, 3L -> 0), lastSequencesInLog(log)) - assertEquals(List(0, 1), LogTest.keysInLog(log)) + assertEquals(List(0, 1), LogTestUtils.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) } @@ -910,7 +909,7 @@ class LogCleanerTest { cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) assertEquals(Map(producerId -> 2), lastSequencesInLog(log)) - assertEquals(List(), LogTest.keysInLog(log)) + assertEquals(List(), LogTestUtils.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // Append a new entry from the producer and verify that the empty batch is cleaned up @@ -920,7 +919,7 @@ class LogCleanerTest { assertEquals(List(3, 5), lastOffsetsPerBatchInLog(log)) assertEquals(Map(producerId -> 4), lastSequencesInLog(log)) - assertEquals(List(1, 5), LogTest.keysInLog(log)) + assertEquals(List(1, 5), LogTestUtils.keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) } @@ -943,16 +942,16 @@ class LogCleanerTest { // clean the log with only one message removed cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 2, log.activeSegment.baseOffset)) - assertEquals(List(1,0,1,0), LogTest.keysInLog(log)) + assertEquals(List(1,0,1,0), LogTestUtils.keysInLog(log)) assertEquals(List(1,2,3,4), offsetsInLog(log)) // continue to make progress, even though we can only clean one message at a time cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 3, log.activeSegment.baseOffset)) - assertEquals(List(0,1,0), LogTest.keysInLog(log)) + assertEquals(List(0,1,0), LogTestUtils.keysInLog(log)) assertEquals(List(2,3,4), offsetsInLog(log)) cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 4, log.activeSegment.baseOffset)) - assertEquals(List(1,0), LogTest.keysInLog(log)) + assertEquals(List(1,0), LogTestUtils.keysInLog(log)) assertEquals(List(3,4), offsetsInLog(log)) } @@ -1115,7 +1114,7 @@ class LogCleanerTest { while(log.numberOfSegments < 4) log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), leaderEpoch = 0) - val keys = LogTest.keysInLog(log) + val keys = LogTestUtils.keysInLog(log) val map = new FakeOffsetMap(Int.MaxValue) keys.foreach(k => map.put(key(k), Long.MaxValue)) assertThrows(classOf[LogCleaningAbortedException], () => @@ -1313,15 +1312,15 @@ class LogCleanerTest { logProps.put(LogConfig.FileDeleteDelayMsProp, 1000: java.lang.Integer) val config = LogConfig.fromProps(logConfig.originals, logProps) - LogTest.initializeLogDirWithOverflowedSegment(dir) + LogTestUtils.initializeLogDirWithOverflowedSegment(dir) val log = makeLog(config = config, recoveryPoint = Long.MaxValue) - val segmentWithOverflow = LogTest.firstOverflowSegment(log).getOrElse { + val segmentWithOverflow = LogTestUtils.firstOverflowSegment(log).getOrElse { throw new AssertionError("Failed to create log with a segment which has overflowed offsets") } val numSegmentsInitial = log.logSegments.size - val allKeys = LogTest.keysInLog(log).toList + val allKeys = LogTestUtils.keysInLog(log).toList val expectedKeysAfterCleaning = new mutable.ArrayBuffer[Long]() // pretend we want to clean every alternate key @@ -1337,15 +1336,15 @@ class LogCleanerTest { new CleanedTransactionMetadata) ) assertEquals(numSegmentsInitial + 1, log.logSegments.size) - assertEquals(allKeys, LogTest.keysInLog(log)) - assertFalse(LogTest.hasOffsetOverflow(log)) + assertEquals(allKeys, LogTestUtils.keysInLog(log)) + assertFalse(LogTestUtils.hasOffsetOverflow(log)) // Clean each segment now that split is complete. for (segmentToClean <- log.logSegments) cleaner.cleanSegments(log, List(segmentToClean), offsetMap, 0L, new CleanerStats(), new CleanedTransactionMetadata) - assertEquals(expectedKeysAfterCleaning, LogTest.keysInLog(log)) - assertFalse(LogTest.hasOffsetOverflow(log)) + assertEquals(expectedKeysAfterCleaning, LogTestUtils.keysInLog(log)) + assertFalse(LogTestUtils.hasOffsetOverflow(log)) log.close() } @@ -1375,7 +1374,7 @@ class LogCleanerTest { log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), leaderEpoch = 0) messageCount += 1 } - val allKeys = LogTest.keysInLog(log) + val allKeys = LogTestUtils.keysInLog(log) // pretend we have odd-numbered keys val offsetMap = new FakeOffsetMap(Int.MaxValue) @@ -1387,7 +1386,7 @@ class LogCleanerTest { new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run time.scheduler.clear() - var cleanedKeys = LogTest.keysInLog(log) + var cleanedKeys = LogTestUtils.keysInLog(log) log.close() // 1) Simulate recovery just after .cleaned file is created, before rename to .swap @@ -1403,7 +1402,7 @@ class LogCleanerTest { new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run time.scheduler.clear() - cleanedKeys = LogTest.keysInLog(log) + cleanedKeys = LogTestUtils.keysInLog(log) log.close() // 2) Simulate recovery just after swap file is created, before old segment files are @@ -1425,7 +1424,7 @@ class LogCleanerTest { new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run time.scheduler.clear() - cleanedKeys = LogTest.keysInLog(log) + cleanedKeys = LogTestUtils.keysInLog(log) // 3) Simulate recovery after swap file is created and old segments files are renamed // to .deleted. Clean operation is resumed during recovery. @@ -1443,7 +1442,7 @@ class LogCleanerTest { new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run time.scheduler.clear() - cleanedKeys = LogTest.keysInLog(log) + cleanedKeys = LogTestUtils.keysInLog(log) log.close() // 4) Simulate recovery after swap is complete, but async deletion diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 0d6e7e3dfc9d4..d59ed1d0c4c63 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -94,14 +94,14 @@ class LogLoaderTest { val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) val maxProducerIdExpirationMs = 60 * 60 * 1000 val segments = new LogSegments(topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, "") val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) val loadLogParams = LoadLogParams(logDir, topicPartition, config, time.scheduler, time, logDirFailureChannel, hadCleanShutdown, segments, logStartOffset, logRecoveryPoint, maxProducerIdExpirationMs, leaderEpochCache, producerStateManager) val offsets = LogLoader.load(loadLogParams) new Log(logDir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, - offsets.nextOffsetMetadata, time.scheduler, brokerTopicStats, time, maxPidExpirationMs, + offsets.nextOffsetMetadata, time.scheduler, brokerTopicStats, time, LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, producerStateManager, logDirFailureChannel, None, true) } @@ -264,7 +264,7 @@ class LogLoaderTest { super.add(wrapper) } } - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion, "") val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) val loadLogParams = LoadLogParams( logDir, @@ -283,9 +283,8 @@ class LogLoaderTest { val offsets = LogLoader.load(loadLogParams) new Log(logDir, logConfig, interceptedLogSegments, offsets.logStartOffset, offsets.recoveryPoint, offsets.nextOffsetMetadata, mockTime.scheduler, brokerTopicStats, mockTime, - maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, - leaderEpochCache, producerStateManager, logDirFailureChannel, topicId = None, - keepPartitionMetadataFile = true) + LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, + producerStateManager, logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) } // Retain snapshots for the last 2 segments @@ -338,7 +337,7 @@ class LogLoaderTest { val config = LogConfig(new Properties()) val maxProducerIdExpirationMs = 300000 val segments = new LogSegments(topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, "") val offsets = LogLoader.load(LoadLogParams( logDir, topicPartition, @@ -362,7 +361,6 @@ class LogLoaderTest { scheduler = mockTime.scheduler, brokerTopicStats = brokerTopicStats, time = mockTime, - maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = 30000, topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, @@ -431,7 +429,8 @@ class LogLoaderTest { firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch) assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) - mockTime.sleep(log.maxProducerIdExpirationMs) + val maxProducerIdExpirationMs = 60 * 60 * 1000 + mockTime.sleep(maxProducerIdExpirationMs) assertEquals(None, log.producerStateManager.lastEntry(producerId)) val secondAppendTimestamp = mockTime.milliseconds() @@ -472,7 +471,7 @@ class LogLoaderTest { val maxProducerIdExpirationMs = 300000 val logDirFailureChannel = null val segments = new LogSegments(topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, "") val offsets = LogLoader.load(LoadLogParams( logDir, topicPartition, @@ -496,7 +495,6 @@ class LogLoaderTest { scheduler = mockTime.scheduler, brokerTopicStats = brokerTopicStats, time = mockTime, - maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = 30000, topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, @@ -534,7 +532,7 @@ class LogLoaderTest { val maxProducerIdExpirationMs = 300000 val logDirFailureChannel = null val segments = new LogSegments(topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, "") val offsets = LogLoader.load(LoadLogParams( logDir, topicPartition, @@ -558,7 +556,6 @@ class LogLoaderTest { scheduler = mockTime.scheduler, brokerTopicStats = brokerTopicStats, time = mockTime, - maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = 30000, topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, @@ -598,7 +595,7 @@ class LogLoaderTest { val maxProducerIdExpirationMs = 300000 val logDirFailureChannel = null val segments = new LogSegments(topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, "") val offsets = LogLoader.load(LoadLogParams( logDir, topicPartition, @@ -622,7 +619,6 @@ class LogLoaderTest { scheduler = mockTime.scheduler, brokerTopicStats = brokerTopicStats, time = mockTime, - maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = 30000, topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, @@ -1345,7 +1341,7 @@ class LogLoaderTest { @Test def testFullTransactionIndexRecovery(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 128 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -1388,7 +1384,7 @@ class LogLoaderTest { log.close() - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 5) val reloadedLog = createLog(logDir, reloadedLogConfig, lastShutdownClean = false) val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) @@ -1396,7 +1392,7 @@ class LogLoaderTest { @Test def testRecoverOnlyLastSegment(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 128 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -1439,7 +1435,7 @@ class LogLoaderTest { log.close() - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 5) val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) @@ -1447,7 +1443,7 @@ class LogLoaderTest { @Test def testRecoverLastSegmentWithNoSnapshots(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 128 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -1493,7 +1489,7 @@ class LogLoaderTest { log.close() - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 5) val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index c8891ebd58bc2..30541165a8289 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -22,13 +22,13 @@ import java.nio.ByteBuffer import java.nio.file.Files import java.util.concurrent.{Callable, Executors} import java.util.regex.Pattern -import java.util.{Collections, Optional, Properties} +import java.util.{Collections, Optional} import kafka.common.{OffsetsOutOfOrderException, RecordValidationException, UnexpectedAppendOffsetException} import kafka.log.Log.DeleteDirSuffix import kafka.metrics.KafkaYammerMetrics import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} -import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata, PartitionMetadataFile} +import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogOffsetMetadata, PartitionMetadataFile} import kafka.utils._ import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.errors._ @@ -43,7 +43,7 @@ import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} -import scala.collection.{Iterable, Map} +import scala.collection.Map import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -76,7 +76,7 @@ class LogTest { @Test def testHighWatermarkMetadataUpdatedAfterSegmentRoll(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) def assertFetchSizeAndOffsets(fetchOffset: Long, @@ -112,7 +112,7 @@ class LogTest { @Test def testAppendAsLeaderWithRaftLeader(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) val leaderEpoch = 0 @@ -137,7 +137,7 @@ class LogTest { @Test def testAppendInfoFirstOffset(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) val simpleRecords = List( @@ -179,7 +179,7 @@ class LogTest { // resets the producer state. Specifically we are testing the case when // the segment position of the first unstable offset is unknown. - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) val producerId = 17L @@ -217,7 +217,7 @@ class LogTest { @Test def testHighWatermarkMaintenance(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) val leaderEpoch = 0 @@ -303,7 +303,7 @@ class LogTest { @Test def testFetchUpToLogEndOffset(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List( @@ -323,7 +323,7 @@ class LogTest { @Test def testFetchUpToHighWatermark(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List( @@ -357,7 +357,7 @@ class LogTest { @Test def testActiveProducers(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) def assertProducerState( @@ -415,7 +415,7 @@ class LogTest { @Test def testFetchUpToLastStableOffset(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -501,7 +501,7 @@ class LogTest { @Test def testTimeBasedLogRoll(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTest.createLogConfig(segmentMs = 1 * 60 * 60L) + val logConfig = LogTestUtils.createLogConfig(segmentMs = 1 * 60 * 60L) // create a log val log = createLog(logDir, logConfig, maxProducerIdExpirationMs = 24 * 60) @@ -548,7 +548,7 @@ class LogTest { @Test def testRollSegmentThatAlreadyExists(): Unit = { - val logConfig = LogTest.createLogConfig(segmentMs = 1 * 60 * 60L) + val logConfig = LogTestUtils.createLogConfig(segmentMs = 1 * 60 * 60L) // create a log val log = createLog(logDir, logConfig) @@ -674,7 +674,7 @@ class LogTest { val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages // create a log - val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -701,7 +701,7 @@ class LogTest { def testInitializationOfProducerSnapshotsUpgradePath(): Unit = { // simulate the upgrade path by creating a new log with several segments, deleting the // snapshot files, and then reloading the log - val logConfig = LogTest.createLogConfig(segmentBytes = 64 * 10) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 64 * 10) var log = createLog(logDir, logConfig) assertEquals(None, log.oldestProducerSnapshotOffset) @@ -740,7 +740,7 @@ class LogTest { @Test def testLogReinitializeAfterManualDelete(): Unit = { - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() // simulate a case where log data does not exist but the start offset is non-zero val log = createLog(logDir, logConfig, logStartOffset = 500) assertEquals(500, log.logStartOffset) @@ -749,7 +749,7 @@ class LogTest { @Test def testLogEndLessThanStartAfterReopen(): Unit = { - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() var log = createLog(logDir, logConfig) for (i <- 0 until 5) { val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) @@ -782,7 +782,7 @@ class LogTest { @Test def testNonActiveSegmentsFrom(): Unit = { - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() val log = createLog(logDir, logConfig) for (i <- 0 until 5) { @@ -804,7 +804,7 @@ class LogTest { @Test def testInconsistentLogSegmentRange(): Unit = { - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() val log = createLog(logDir, logConfig) for (i <- 0 until 5) { @@ -818,7 +818,7 @@ class LogTest { @Test def testLogDelete(): Unit = { - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() val log = createLog(logDir, logConfig) for (i <- 0 to 100) { @@ -846,7 +846,7 @@ class LogTest { val scheduler = new KafkaScheduler(1) try { scheduler.startup() - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() val log = createLog(logDir, logConfig, scheduler = scheduler) val producerExpireCheck = log.producerExpireCheck @@ -875,7 +875,7 @@ class LogTest { @Test def testProducerIdMapOffsetUpdatedForNonIdempotentData(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val records = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds, "key".getBytes, "value".getBytes))) log.appendAsLeader(records, leaderEpoch = 0) @@ -885,7 +885,7 @@ class LogTest { @Test def testRebuildProducerIdMapWithCompactedData(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L val epoch = 0.toShort @@ -928,7 +928,7 @@ class LogTest { @Test def testRebuildProducerStateWithEmptyCompactedBatch(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L val epoch = 0.toShort @@ -969,7 +969,7 @@ class LogTest { @Test def testUpdateProducerIdMapWithCompactedData(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L val epoch = 0.toShort @@ -1002,7 +1002,7 @@ class LogTest { @Test def testProducerIdMapTruncateTo(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes))), leaderEpoch = 0) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes))), leaderEpoch = 0) @@ -1027,7 +1027,7 @@ class LogTest { @Test def testProducerIdMapTruncateToWithNoSnapshots(): Unit = { // This ensures that the upgrade optimization path cannot be hit after initial loading - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L val epoch = 0.toShort @@ -1051,7 +1051,7 @@ class LogTest { @Test def testRetentionDeletesProducerStateSnapshots(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0) val log = createLog(logDir, logConfig) val pid1 = 1L val epoch = 0.toShort @@ -1079,7 +1079,7 @@ class LogTest { @Test def testRetentionIdempotency(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = -1, retentionMs = 900, fileDeleteDelayMs = 0) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = -1, retentionMs = 900, fileDeleteDelayMs = 0) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds() + 100, "a".getBytes))), leaderEpoch = 0) @@ -1100,7 +1100,7 @@ class LogTest { @Test def testLogStartOffsetMovementDeletesSnapshots(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = -1, fileDeleteDelayMs = 0) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = -1, fileDeleteDelayMs = 0) val log = createLog(logDir, logConfig) val pid1 = 1L val epoch = 0.toShort @@ -1127,7 +1127,7 @@ class LogTest { @Test def testCompactionDeletesProducerStateSnapshots(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, cleanupPolicy = LogConfig.Compact, fileDeleteDelayMs = 0) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, cleanupPolicy = LogConfig.Compact, fileDeleteDelayMs = 0) val log = createLog(logDir, logConfig) val pid1 = 1L val epoch = 0.toShort @@ -1171,7 +1171,7 @@ class LogTest { def testLoadingLogDeletesProducerStateSnapshotsPastLogEndOffset(): Unit = { val straySnapshotFile = Log.producerSnapshotFile(logDir, 42).toPath Files.createFile(straySnapshotFile) - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = -1, fileDeleteDelayMs = 0) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = -1, fileDeleteDelayMs = 0) createLog(logDir, logConfig) assertEquals(0, ProducerStateManager.listSnapshotFiles(logDir).size, "expected producer state snapshots greater than the log end offset to be cleaned up") @@ -1180,7 +1180,7 @@ class LogTest { @Test def testProducerIdMapTruncateFullyAndStartAt(): Unit = { val records = TestUtils.singletonRecords("foo".getBytes) - val logConfig = LogTest.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) val log = createLog(logDir, logConfig) log.appendAsLeader(records, leaderEpoch = 0) log.takeProducerSnapshot() @@ -1203,7 +1203,7 @@ class LogTest { def testProducerIdExpirationOnSegmentDeletion(): Unit = { val pid1 = 1L val records = TestUtils.records(Seq(new SimpleRecord("foo".getBytes)), producerId = pid1, producerEpoch = 0, sequence = 0) - val logConfig = LogTest.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) val log = createLog(logDir, logConfig) log.appendAsLeader(records, leaderEpoch = 0) log.takeProducerSnapshot() @@ -1228,7 +1228,7 @@ class LogTest { @Test def testTakeSnapshotOnRollAndDeleteSnapshotOnRecoveryPointCheckpoint(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.singletonRecords("a".getBytes), leaderEpoch = 0) log.roll(Some(1L)) @@ -1256,7 +1256,7 @@ class LogTest { @Test def testProducerSnapshotAfterSegmentRollOnAppend(): Unit = { val producerId = 1L - val logConfig = LogTest.createLogConfig(segmentBytes = 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(Seq(new SimpleRecord(mockTime.milliseconds(), new Array[Byte](512))), @@ -1288,7 +1288,7 @@ class LogTest { @Test def testRebuildTransactionalState(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val pid = 137L @@ -1320,7 +1320,7 @@ class LogTest { val producerIdExpirationCheckIntervalMs = 100 val pid = 23L - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig, maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs) val records = Seq(new SimpleRecord(mockTime.milliseconds(), "foo".getBytes)) @@ -1459,7 +1459,7 @@ class LogTest { @Test def testDuplicateAppendToFollower(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val epoch: Short = 0 val pid = 1L @@ -1480,7 +1480,7 @@ class LogTest { @Test def testMultipleProducersWithDuplicatesInSingleAppend(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val pid1 = 1L @@ -1547,7 +1547,7 @@ class LogTest { @Test def testDeleteSnapshotsOnIncrementLogStartOffset(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid1 = 1L val pid2 = 2L @@ -1585,7 +1585,7 @@ class LogTest { var set = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) val maxJitter = 20 * 60L // create a log - val logConfig = LogTest.createLogConfig(segmentMs = 1 * 60 * 60L, segmentJitterMs = maxJitter) + val logConfig = LogTestUtils.createLogConfig(segmentMs = 1 * 60 * 60L, segmentJitterMs = maxJitter) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "Log begins with a single empty segment.") log.appendAsLeader(set, leaderEpoch = 0) @@ -1612,7 +1612,7 @@ class LogTest { val msgPerSeg = 10 val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages // create a log - val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -1638,7 +1638,7 @@ class LogTest { */ @Test def testAppendAndReadWithSequentialOffsets(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 71) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 71) val log = createLog(logDir, logConfig) val values = (0 until 100 by 2).map(id => id.toString.getBytes).toArray @@ -1662,7 +1662,7 @@ class LogTest { */ @Test def testAppendAndReadWithNonSequentialOffsets(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 72) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 72) val log = createLog(logDir, logConfig) val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) @@ -1686,7 +1686,7 @@ class LogTest { */ @Test def testReadAtLogGap(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 300) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 300) val log = createLog(logDir, logConfig) // keep appending until we have two segments with only a single message in the second segment @@ -1702,7 +1702,7 @@ class LogTest { @Test def testLogRollAfterLogHandlerClosed(): Unit = { - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() val log = createLog(logDir, logConfig) log.closeHandlers() assertThrows(classOf[KafkaStorageException], () => log.roll(Some(1L))) @@ -1710,7 +1710,7 @@ class LogTest { @Test def testReadWithMinMessage(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 72) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 72) val log = createLog(logDir, logConfig) val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) @@ -1735,7 +1735,7 @@ class LogTest { @Test def testReadWithTooSmallMaxLength(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 72) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 72) val log = createLog(logDir, logConfig) val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) @@ -1769,7 +1769,7 @@ class LogTest { def testReadOutOfRange(): Unit = { createEmptyLogs(logDir, 1024) // set up replica log starting with offset 1024 and with one message (at offset 1024) - val logConfig = LogTest.createLogConfig(segmentBytes = 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.singletonRecords(value = "42".getBytes), leaderEpoch = 0) @@ -1787,7 +1787,7 @@ class LogTest { @Test def testLogRolls(): Unit = { /* create a multipart log with 100 messages */ - val logConfig = LogTest.createLogConfig(segmentBytes = 100) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 100) val log = createLog(logDir, logConfig) val numMessages = 100 val messageSets = (0 until numMessages).map(i => TestUtils.singletonRecords(value = i.toString.getBytes, @@ -1824,7 +1824,7 @@ class LogTest { @Test def testCompressedMessages(): Unit = { /* this log should roll after every messageset */ - val logConfig = LogTest.createLogConfig(segmentBytes = 110) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 110) val log = createLog(logDir, logConfig) /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ @@ -1848,7 +1848,7 @@ class LogTest { for(messagesToAppend <- List(0, 1, 25)) { logDir.mkdirs() // first test a log segment starting at 0 - val logConfig = LogTest.createLogConfig(segmentBytes = 100, retentionMs = 0) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 100, retentionMs = 0) val log = createLog(logDir, logConfig) for(i <- 0 until messagesToAppend) log.appendAsLeader(TestUtils.singletonRecords(value = i.toString.getBytes, timestamp = mockTime.milliseconds - 10), leaderEpoch = 0) @@ -1886,7 +1886,7 @@ class LogTest { val messageSet = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes)) // append messages to log val configSegmentSize = messageSet.sizeInBytes - 1 - val logConfig = LogTest.createLogConfig(segmentBytes = configSegmentSize) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = configSegmentSize) val log = createLog(logDir, logConfig) assertThrows(classOf[RecordBatchTooLargeException], () => log.appendAsLeader(messageSet, leaderEpoch = 0)) @@ -1906,7 +1906,7 @@ class LogTest { val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage) val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage) - val logConfig = LogTest.createLogConfig(cleanupPolicy = LogConfig.Compact) + val logConfig = LogTestUtils.createLogConfig(cleanupPolicy = LogConfig.Compact) val log = createLog(logDir, logConfig) val errorMsgPrefix = "Compacted topic cannot accept message without key" @@ -1955,7 +1955,7 @@ class LogTest { // append messages to log val maxMessageSize = second.sizeInBytes - 1 - val logConfig = LogTest.createLogConfig(maxMessageBytes = maxMessageSize) + val logConfig = LogTestUtils.createLogConfig(maxMessageBytes = maxMessageSize) val log = createLog(logDir, logConfig) // should be able to append the small message @@ -1973,7 +1973,7 @@ class LogTest { new SimpleRecord("change (I need more bytes)... blah blah blah.".getBytes), new SimpleRecord("More padding boo hoo".getBytes)) - val log = createLog(logDir, LogTest.createLogConfig(maxMessageBytes = second.sizeInBytes - 1)) + val log = createLog(logDir, LogTestUtils.createLogConfig(maxMessageBytes = second.sizeInBytes - 1)) log.appendAsFollower(first) // the second record is larger then limit but appendAsFollower does not validate the size. @@ -1982,7 +1982,7 @@ class LogTest { @Test def testLogRecoversTopicId(): Unit = { - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() var log = createLog(logDir, logConfig) val topicId = Uuid.randomUuid() @@ -1998,7 +1998,7 @@ class LogTest { @Test def testLogFailsWhenInconsistentTopicIdSet(): Unit = { - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() var log = createLog(logDir, logConfig) val topicId = Uuid.randomUuid() @@ -2020,7 +2020,7 @@ class LogTest { @Test def testBuildTimeIndexWhenNotAssigningOffsets(): Unit = { val numMessages = 100 - val logConfig = LogTest.createLogConfig(segmentBytes = 10000, indexIntervalBytes = 1) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 10000, indexIntervalBytes = 1) val log = createLog(logDir, logConfig) val messages = (0 until numMessages).map { i => @@ -2035,7 +2035,7 @@ class LogTest { @Test def testFetchOffsetByTimestampIncludesLeaderEpoch(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) val log = createLog(logDir, logConfig) assertEquals(None, log.fetchOffsetByTimestamp(0L)) @@ -2083,7 +2083,7 @@ class LogTest { val segmentSize = msgPerSeg * setSize // each segment will be 10 messages // create a log - val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -2135,7 +2135,7 @@ class LogTest { val setSize = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds).sizeInBytes val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages - val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = setSize - 1) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = setSize - 1) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -2175,7 +2175,7 @@ class LogTest { def testAsyncDelete(): Unit = { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000L) val asyncDeleteMs = 1000 - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000, + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000, retentionMs = 999, fileDeleteDelayMs = asyncDeleteMs) val log = createLog(logDir, logConfig) @@ -2299,7 +2299,7 @@ class LogTest { @Test def testWriteLeaderEpochCheckpointAfterDirectoryRename(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) assertEquals(Some(5), log.latestEpoch) @@ -2315,7 +2315,7 @@ class LogTest { @Test def testTopicIdTransfersAfterDirectoryRename(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val log = createLog(logDir, logConfig) // Write a topic ID to the partition metadata file to ensure it is transferred correctly. @@ -2342,7 +2342,7 @@ class LogTest { @Test def testLeaderEpochCacheClearedAfterDowngradeInAppendedMessages(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) assertEquals(Some(5), log.leaderEpochCache.flatMap(_.latestEpoch)) @@ -2355,12 +2355,12 @@ class LogTest { @Test def testLeaderEpochCacheClearedAfterDynamicMessageFormatDowngrade(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) assertEquals(Some(5), log.latestEpoch) - val downgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, + val downgradedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) log.updateConfig(downgradedLogConfig) LogTestUtils.assertLeaderEpochCacheEmpty(log) @@ -2372,14 +2372,14 @@ class LogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), magicValue = RecordVersion.V1.value), leaderEpoch = 5) LogTestUtils.assertLeaderEpochCacheEmpty(log) - val upgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, + val upgradedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_11_0_IV0.shortVersion) log.updateConfig(upgradedLogConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) @@ -2391,9 +2391,9 @@ class LogTest { @Test def testSplitOnOffsetOverflow(): Unit = { // create a log such that one log segment has offsets that overflow, and call the split API on that segment - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - assertTrue(LogTest.hasOffsetOverflow(log), "At least one segment must have offset overflow") + assertTrue(LogTestUtils.hasOffsetOverflow(log), "At least one segment must have offset overflow") val allRecordsBeforeSplit = LogTest.allRecords(log) @@ -2405,7 +2405,7 @@ class LogTest { LogTest.verifyRecordsInLog(log, allRecordsBeforeSplit) // verify we do not have offset overflow anymore - assertFalse(LogTest.hasOffsetOverflow(log)) + assertFalse(LogTestUtils.hasOffsetOverflow(log)) } @Test @@ -2437,17 +2437,17 @@ class LogTest { } private def testDegenerateSplitSegmentWithOverflow(segmentBaseOffset: Long, records: List[MemoryRecords]): Unit = { - val segment = LogTest.rawSegment(logDir, segmentBaseOffset) + val segment = LogTestUtils.rawSegment(logDir, segmentBaseOffset) // Need to create the offset files explicitly to avoid triggering segment recovery to truncate segment. Log.offsetIndexFile(logDir, segmentBaseOffset).createNewFile() Log.timeIndexFile(logDir, segmentBaseOffset).createNewFile() records.foreach(segment.append _) segment.close() - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue) - val segmentWithOverflow = LogTest.firstOverflowSegment(log).getOrElse { + val segmentWithOverflow = LogTestUtils.firstOverflowSegment(log).getOrElse { throw new AssertionError("Failed to create log with a segment which has overflowed offsets") } @@ -2460,7 +2460,7 @@ class LogTest { assertEquals(firstBatchBaseOffset, log.activeSegment.baseOffset) LogTest.verifyRecordsInLog(log, allRecordsBeforeSplit) - assertFalse(LogTest.hasOffsetOverflow(log)) + assertFalse(LogTestUtils.hasOffsetOverflow(log)) } @Test @@ -2573,7 +2573,7 @@ class LogTest { @Test def testDeleteOldSegments(): Unit = { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2623,7 +2623,7 @@ class LogTest { @Test def testLogDeletionAfterClose(): Unit = { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2641,7 +2641,7 @@ class LogTest { @Test def testLogDeletionAfterDeleteRecords(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5) val log = createLog(logDir, logConfig) for (_ <- 0 until 15) @@ -2673,7 +2673,7 @@ class LogTest { @Test def shouldDeleteSizeBasedSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2688,7 +2688,7 @@ class LogTest { @Test def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 15) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 15) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2703,7 +2703,7 @@ class LogTest { @Test def shouldDeleteTimeBasedSegmentsReadyToBeDeleted(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, timestamp = 10) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2718,7 +2718,7 @@ class LogTest { @Test def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, timestamp = mockTime.milliseconds) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000000) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000000) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2733,7 +2733,7 @@ class LogTest { @Test def shouldNotDeleteSegmentsWhenPolicyDoesNotIncludeDelete(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000, cleanupPolicy = "compact") + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000, cleanupPolicy = "compact") val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2752,7 +2752,7 @@ class LogTest { @Test def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes, timestamp = 10L) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000, cleanupPolicy = "compact,delete") + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000, cleanupPolicy = "compact,delete") val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2768,7 +2768,7 @@ class LogTest { def shouldDeleteStartOffsetBreachedSegmentsWhenPolicyDoesNotIncludeDelete(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes, timestamp = 10L) val recordsPerSegment = 5 - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * recordsPerSegment, retentionMs = 10000, cleanupPolicy = "compact") + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * recordsPerSegment, retentionMs = 10000, cleanupPolicy = "compact") val log = createLog(logDir, logConfig, brokerTopicStats) // append some messages to create some segments @@ -2840,7 +2840,7 @@ class LogTest { @Test def shouldTruncateLeaderEpochsWhenDeletingSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -2865,7 +2865,7 @@ class LogTest { @Test def shouldUpdateOffsetForLeaderEpochsWhenDeletingSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -2894,7 +2894,7 @@ class LogTest { baseOffset = startOffset, partitionLeaderEpoch = epoch) } - val logConfig = LogTest.createLogConfig(segmentBytes = 10 * createRecords(0, 0).sizeInBytes) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 10 * createRecords(0, 0).sizeInBytes) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -2938,7 +2938,7 @@ class LogTest { @Test def testFirstUnstableOffsetNoTransactionalData(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val records = MemoryRecords.withRecords(CompressionType.NONE, @@ -2952,7 +2952,7 @@ class LogTest { @Test def testFirstUnstableOffsetWithTransactionalData(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val pid = 137L @@ -2989,7 +2989,7 @@ class LogTest { @Test def testReadCommittedWithConcurrentHighWatermarkUpdates(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val lastOffset = 50L @@ -3042,7 +3042,7 @@ class LogTest { @Test def testTransactionIndexUpdated(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -3098,7 +3098,7 @@ class LogTest { @Test def testTransactionIndexUpdatedThroughReplication(): Unit = { val epoch = 0.toShort - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val buffer = ByteBuffer.allocate(2048) @@ -3189,7 +3189,7 @@ class LogTest { def testZombieCoordinatorFenced(): Unit = { val pid = 1L val epoch = 0.toShort - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) @@ -3209,7 +3209,7 @@ class LogTest { def testZombieCoordinatorFencedEmptyTransaction(): Unit = { val pid = 1L val epoch = 0.toShort - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val buffer = ByteBuffer.allocate(256) @@ -3230,7 +3230,7 @@ class LogTest { def testEndTxnWithFencedProducerEpoch(): Unit = { val producerId = 1L val epoch = 5.toShort - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1) @@ -3240,7 +3240,7 @@ class LogTest { @Test def testLastStableOffsetDoesNotExceedLogStartOffsetMidSegment(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort val pid = 1L @@ -3265,7 +3265,7 @@ class LogTest { @Test def testLastStableOffsetDoesNotExceedLogStartOffsetAfterSegmentDeletion(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort val pid = 1L @@ -3295,7 +3295,7 @@ class LogTest { def testAppendToTransactionIndexFailure(): Unit = { val pid = 1L val epoch = 0.toShort - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) @@ -3335,7 +3335,7 @@ class LogTest { @Test def testOffsetSnapshot(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) // append a few records @@ -3357,7 +3357,7 @@ class LogTest { @Test def testLastStableOffsetWithMixedProducerData(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) // for convenience, both producers share the same epoch @@ -3416,7 +3416,7 @@ class LogTest { new SimpleRecord("b".getBytes), new SimpleRecord("c".getBytes)) - val logConfig = LogTest.createLogConfig(segmentBytes = records.sizeInBytes) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes) val log = createLog(logDir, logConfig) val firstAppendInfo = log.appendAsLeader(records, leaderEpoch = 0) @@ -3453,7 +3453,7 @@ class LogTest { val dirName = Log.logDeleteDirName(new TopicPartition("foo", 3)) val logDir = new File(tmpDir, dirName) logDir.mkdirs() - val logConfig = LogTest.createLogConfig() + val logConfig = LogTestUtils.createLogConfig() val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments) } @@ -3511,15 +3511,15 @@ class LogTest { producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, lastShutdownClean: Boolean = true, topicId: Option[Uuid] = None): Log = { - LogTest.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint, + LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint, maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs, lastShutdownClean, topicId = topicId) } private def createLogWithOffsetOverflow(logConfig: LogConfig): (Log, LogSegment) = { - LogTest.initializeLogDirWithOverflowedSegment(logDir) + LogTestUtils.initializeLogDirWithOverflowedSegment(logDir) val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue) - val segmentWithOverflow = LogTest.firstOverflowSegment(log).getOrElse { + val segmentWithOverflow = LogTestUtils.firstOverflowSegment(log).getOrElse { throw new AssertionError("Failed to create log with a segment which has overflowed offsets") } @@ -3528,126 +3528,6 @@ class LogTest { } object LogTest { - def createLogConfig(segmentMs: Long = Defaults.SegmentMs, - segmentBytes: Int = Defaults.SegmentSize, - retentionMs: Long = Defaults.RetentionMs, - retentionBytes: Long = Defaults.RetentionSize, - segmentJitterMs: Long = Defaults.SegmentJitterMs, - cleanupPolicy: String = Defaults.CleanupPolicy, - maxMessageBytes: Int = Defaults.MaxMessageSize, - indexIntervalBytes: Int = Defaults.IndexInterval, - segmentIndexBytes: Int = Defaults.MaxIndexSize, - messageFormatVersion: String = Defaults.MessageFormatVersion, - fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs): LogConfig = { - val logProps = new Properties() - - logProps.put(LogConfig.SegmentMsProp, segmentMs: java.lang.Long) - logProps.put(LogConfig.SegmentBytesProp, segmentBytes: Integer) - logProps.put(LogConfig.RetentionMsProp, retentionMs: java.lang.Long) - logProps.put(LogConfig.RetentionBytesProp, retentionBytes: java.lang.Long) - logProps.put(LogConfig.SegmentJitterMsProp, segmentJitterMs: java.lang.Long) - logProps.put(LogConfig.CleanupPolicyProp, cleanupPolicy) - logProps.put(LogConfig.MaxMessageBytesProp, maxMessageBytes: Integer) - logProps.put(LogConfig.IndexIntervalBytesProp, indexIntervalBytes: Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, segmentIndexBytes: Integer) - logProps.put(LogConfig.MessageFormatVersionProp, messageFormatVersion) - logProps.put(LogConfig.FileDeleteDelayMsProp, fileDeleteDelayMs: java.lang.Long) - LogConfig(logProps) - } - - def createLog(dir: File, - config: LogConfig, - brokerTopicStats: BrokerTopicStats, - scheduler: Scheduler, - time: Time, - logStartOffset: Long = 0L, - recoveryPoint: Long = 0L, - maxProducerIdExpirationMs: Int = 60 * 60 * 1000, - producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, - lastShutdownClean: Boolean = true, - topicId: Option[Uuid] = None): Log = { - Log(dir = dir, - config = config, - logStartOffset = logStartOffset, - recoveryPoint = recoveryPoint, - scheduler = scheduler, - brokerTopicStats = brokerTopicStats, - time = time, - maxProducerIdExpirationMs = maxProducerIdExpirationMs, - producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs, - logDirFailureChannel = new LogDirFailureChannel(10), - lastShutdownClean = lastShutdownClean, - topicId = topicId, - keepPartitionMetadataFile = true) - } - - /** - * Check if the given log contains any segment with records that cause offset overflow. - * @param log Log to check - * @return true if log contains at least one segment with offset overflow; false otherwise - */ - def hasOffsetOverflow(log: Log): Boolean = firstOverflowSegment(log).isDefined - - def firstOverflowSegment(log: Log): Option[LogSegment] = { - def hasOverflow(baseOffset: Long, batch: RecordBatch): Boolean = - batch.lastOffset > baseOffset + Int.MaxValue || batch.baseOffset < baseOffset - - for (segment <- log.logSegments) { - val overflowBatch = segment.log.batches.asScala.find(batch => hasOverflow(segment.baseOffset, batch)) - if (overflowBatch.isDefined) - return Some(segment) - } - None - } - - private def rawSegment(logDir: File, baseOffset: Long): FileRecords = - FileRecords.open(Log.logFile(logDir, baseOffset)) - - /** - * Initialize the given log directory with a set of segments, one of which will have an - * offset which overflows the segment - */ - def initializeLogDirWithOverflowedSegment(logDir: File): Unit = { - def writeSampleBatches(baseOffset: Long, segment: FileRecords): Long = { - def record(offset: Long) = { - val data = offset.toString.getBytes - new SimpleRecord(data, data) - } - - segment.append(MemoryRecords.withRecords(baseOffset, CompressionType.NONE, 0, - record(baseOffset))) - segment.append(MemoryRecords.withRecords(baseOffset + 1, CompressionType.NONE, 0, - record(baseOffset + 1), - record(baseOffset + 2))) - segment.append(MemoryRecords.withRecords(baseOffset + Int.MaxValue - 1, CompressionType.NONE, 0, - record(baseOffset + Int.MaxValue - 1))) - // Need to create the offset files explicitly to avoid triggering segment recovery to truncate segment. - Log.offsetIndexFile(logDir, baseOffset).createNewFile() - Log.timeIndexFile(logDir, baseOffset).createNewFile() - baseOffset + Int.MaxValue - } - - def writeNormalSegment(baseOffset: Long): Long = { - val segment = rawSegment(logDir, baseOffset) - try writeSampleBatches(baseOffset, segment) - finally segment.close() - } - - def writeOverflowSegment(baseOffset: Long): Long = { - val segment = rawSegment(logDir, baseOffset) - try { - val nextOffset = writeSampleBatches(baseOffset, segment) - writeSampleBatches(nextOffset, segment) - } finally segment.close() - } - - // We create three segments, the second of which contains offsets which overflow - var nextOffset = 0L - nextOffset = writeNormalSegment(nextOffset) - nextOffset = writeOverflowSegment(nextOffset) - writeNormalSegment(nextOffset) - } - def allRecords(log: Log): List[Record] = { val recordsFound = ListBuffer[Record]() for (logSegment <- log.logSegments) { @@ -3661,12 +3541,4 @@ object LogTest { def verifyRecordsInLog(log: Log, expectedRecords: List[Record]): Unit = { assertEquals(expectedRecords, allRecords(log)) } - - /* extract all the keys from a log */ - def keysInLog(log: Log): Iterable[Long] = { - for (logSegment <- log.logSegments; - batch <- logSegment.log.batches.asScala if !batch.isControlBatch; - record <- batch.asScala if record.hasValue && record.hasKey) - yield TestUtils.readString(record.key).toLong - } } diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 29777b2dec900..0169cd4b3d586 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -119,7 +119,7 @@ object LogTestUtils { None } - private def rawSegment(logDir: File, baseOffset: Long): FileRecords = + def rawSegment(logDir: File, baseOffset: Long): FileRecords = FileRecords.open(Log.logFile(logDir, baseOffset)) /** @@ -185,8 +185,8 @@ object LogTestUtils { assertFalse(file.getName.endsWith(Log.CleanedFileSuffix), "Unexpected .cleaned file after recovery") assertFalse(file.getName.endsWith(Log.SwapFileSuffix), "Unexpected .swap file after recovery") } - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertFalse(LogTest.hasOffsetOverflow(recoveredLog)) + assertEquals(expectedKeys, keysInLog(recoveredLog)) + assertFalse(hasOffsetOverflow(recoveredLog)) recoveredLog } @@ -214,10 +214,10 @@ object LogTestUtils { } def readLog(log: Log, - startOffset: Long, - maxLength: Int, - isolation: FetchIsolation = FetchLogEnd, - minOneMessage: Boolean = true): FetchDataInfo = { + startOffset: Long, + maxLength: Int, + isolation: FetchIsolation = FetchLogEnd, + minOneMessage: Boolean = true): FetchDataInfo = { log.read(startOffset, maxLength, isolation, minOneMessage) } diff --git a/core/src/test/scala/unit/kafka/server/AlterIsrManagerTest.scala b/core/src/test/scala/unit/kafka/server/AlterIsrManagerTest.scala index 1074fd3157c2f..1c8c81471fd91 100644 --- a/core/src/test/scala/unit/kafka/server/AlterIsrManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterIsrManagerTest.scala @@ -70,8 +70,10 @@ class AlterIsrManagerTest { @Test def testOverwriteWithinBatch(): Unit = { val capture = EasyMock.newCapture[AbstractRequest.Builder[AlterIsrRequest]]() + val callbackCapture = EasyMock.newCapture[ControllerRequestCompletionHandler]() + EasyMock.expect(brokerToController.start()) - EasyMock.expect(brokerToController.sendRequest(EasyMock.capture(capture), EasyMock.anyObject())).once() + EasyMock.expect(brokerToController.sendRequest(EasyMock.capture(capture), EasyMock.capture(callbackCapture))).times(2) EasyMock.replay(brokerToController) val scheduler = new MockScheduler(time) @@ -81,11 +83,21 @@ class AlterIsrManagerTest { // Only send one ISR update for a given topic+partition assertTrue(alterIsrManager.submit(AlterIsrItem(tp0, new LeaderAndIsr(1, 1, List(1,2,3), 10), _ => {}, 0))) assertFalse(alterIsrManager.submit(AlterIsrItem(tp0, new LeaderAndIsr(1, 1, List(1,2), 10), _ => {}, 0))) + + // Simulate response + val alterIsrResp = partitionResponse(tp0, Errors.NONE) + val resp = new ClientResponse(null, null, "", 0L, 0L, + false, null, null, alterIsrResp) + callbackCapture.getValue.onComplete(resp) + + // Now we can submit this partition again + assertTrue(alterIsrManager.submit(AlterIsrItem(tp0, new LeaderAndIsr(1, 1, List(1), 10), _ => {}, 0))) EasyMock.verify(brokerToController) + // Make sure we sent the right request ISR={1} val request = capture.getValue.build() assertEquals(request.data().topics().size(), 1) - assertEquals(request.data().topics().get(0).partitions().get(0).newIsr().size(), 3) + assertEquals(request.data().topics().get(0).partitions().get(0).newIsr().size(), 1) } @Test diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 1ccd98f63a15c..46f4c1bd51e64 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -139,6 +139,7 @@ class KafkaApisTest { overrideProperties.foreach( p => properties.put(p._1, p._2)) properties.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerProtocolVersion.toString) properties.put(KafkaConfig.LogMessageFormatVersionProp, interBrokerProtocolVersion.toString) + val config = new KafkaConfig(properties) val forwardingManagerOpt = if (enableForwarding) Some(this.forwardingManager) @@ -176,7 +177,7 @@ class KafkaApisTest { txnCoordinator, autoTopicCreationManager, brokerId, - new KafkaConfig(properties), + config, configRepository, metadataCache, metrics, @@ -1104,7 +1105,6 @@ class KafkaApisTest { 15L, 0.toShort, Map(invalidTopicPartition -> partitionOffsetCommitData).asJava, - false ).build() val request = buildRequest(offsetCommitRequest) @@ -1144,7 +1144,6 @@ class KafkaApisTest { producerId, epoch, Map(topicPartition -> partitionOffsetCommitData).asJava, - false ).build(version.toShort) val request = buildRequest(offsetCommitRequest) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 77e4816725ec2..2a0ccca7a0217 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -1486,7 +1486,7 @@ class ReplicaManagerTest { val tp = new TopicPartition(topic, topicPartition) val maxProducerIdExpirationMs = 30000 val segments = new LogSegments(tp) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, tp, mockLogDirFailureChannel, logConfig.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, tp, mockLogDirFailureChannel, logConfig.messageFormatVersion.recordVersion, "") val producerStateManager = new ProducerStateManager(tp, logDir, maxProducerIdExpirationMs) val offsets = LogLoader.load(LoadLogParams( logDir, @@ -1512,7 +1512,6 @@ class ReplicaManagerTest { scheduler = mockScheduler, brokerTopicStats = mockBrokerTopicStats, time = time, - maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = 30000, topicPartition = tp, leaderEpochCache = leaderEpochCache, diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 35f6ff8b8c953..757b82c13dbe8 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -131,7 +131,7 @@ class RequestQuotaTest extends BaseRequestTest { @Test def testResponseThrottleTime(): Unit = { - for (apiKey <- RequestQuotaTest.ClientActions) + for (apiKey <- RequestQuotaTest.ClientActions ++ RequestQuotaTest.ClusterActionsWithThrottle) submitTest(apiKey, () => checkRequestThrottleTime(apiKey)) waitAndCheckResults() @@ -160,7 +160,7 @@ class RequestQuotaTest extends BaseRequestTest { @Test def testExemptRequestTime(): Unit = { - for (apiKey <- RequestQuotaTest.ClusterActions) { + for (apiKey <- RequestQuotaTest.ClusterActions -- RequestQuotaTest.ClusterActionsWithThrottle) { submitTest(apiKey, () => checkExemptRequestMetric(apiKey)) } @@ -454,8 +454,8 @@ class RequestQuotaTest extends BaseRequestTest { "test-txn-group", 2, 0, - Map.empty[TopicPartition, TxnOffsetCommitRequest.CommittedOffset].asJava, - false) + Map.empty[TopicPartition, TxnOffsetCommitRequest.CommittedOffset].asJava + ) case ApiKeys.DESCRIBE_ACLS => new DescribeAclsRequest.Builder(AclBindingFilter.ANY) @@ -642,6 +642,8 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.LIST_TRANSACTIONS => new ListTransactionsRequest.Builder(new ListTransactionsRequestData()) + case ApiKeys.ALLOCATE_PRODUCER_IDS => + new AllocateProducerIdsRequest.Builder(new AllocateProducerIdsRequestData()) case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) } @@ -762,6 +764,7 @@ class RequestQuotaTest extends BaseRequestTest { object RequestQuotaTest { val ClusterActions = ApiKeys.zkBrokerApis.asScala.filter(_.clusterAction).toSet + val ClusterActionsWithThrottle = Set(ApiKeys.ALLOCATE_PRODUCER_IDS) val SaslActions = Set(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE) val ClientActions = ApiKeys.zkBrokerApis.asScala.toSet -- ClusterActions -- SaslActions diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala index 4c5c6993fd724..ccccb8bd28ede 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala @@ -25,9 +25,11 @@ import kafka.server.RaftReplicaManager import kafka.utils.Implicits._ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.metadata.{ConfigRecord, PartitionRecord, RemoveTopicRecord, TopicRecord} -import org.apache.kafka.common.protocol.ApiMessage import org.apache.kafka.common.utils.MockTime import org.apache.kafka.common.{TopicPartition, Uuid} +import org.apache.kafka.raft.Batch +import org.apache.kafka.raft.internals.MemoryBatchReader; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.mockito.ArgumentMatchers._ @@ -39,6 +41,7 @@ import scala.jdk.CollectionConverters._ class BrokerMetadataListenerTest { + private val leaderEpoch = 5 private val brokerId = 1 private val time = new MockTime() private val configRepository = new CachedConfigRepository @@ -82,11 +85,10 @@ class BrokerMetadataListenerTest { ): Unit = { val deleteRecord = new RemoveTopicRecord() .setTopicId(topicId) - lastMetadataOffset += 1 - listener.execCommits(lastOffset = lastMetadataOffset, List[ApiMessage]( - deleteRecord, - ).asJava) + applyBatch(List[ApiMessageAndVersion]( + new ApiMessageAndVersion(deleteRecord, 0.toShort), + )) assertFalse(metadataCache.contains(topic)) assertEquals(new Properties, configRepository.topicConfig(topic)) @@ -108,6 +110,25 @@ class BrokerMetadataListenerTest { assertEquals(localPartitions, localRemoved.map(_.toTopicPartition).toSet) } + private def applyBatch( + records: List[ApiMessageAndVersion] + ): Unit = { + val baseOffset = lastMetadataOffset + 1 + lastMetadataOffset += records.size + listener.execCommits( + new MemoryBatchReader( + List( + Batch.of( + baseOffset, + leaderEpoch, + records.asJava + ) + ).asJava, + reader => () + ) + ) + } + private def createAndAssert( topicId: Uuid, topic: String, @@ -115,11 +136,10 @@ class BrokerMetadataListenerTest { numPartitions: Int, numBrokers: Int ): Set[TopicPartition] = { - val records = new java.util.ArrayList[ApiMessage] - records.add(new TopicRecord() + val records = mutable.ListBuffer.empty[ApiMessageAndVersion] + records += new ApiMessageAndVersion(new TopicRecord() .setName(topic) - .setTopicId(topicId) - ) + .setTopicId(topicId), 0) val localTopicPartitions = mutable.Set.empty[TopicPartition] (0 until numPartitions).map { partitionId => @@ -134,28 +154,25 @@ class BrokerMetadataListenerTest { localTopicPartitions.add(new TopicPartition(topic, partitionId)) } - records.add(new PartitionRecord() + records += new ApiMessageAndVersion(new PartitionRecord() .setTopicId(topicId) .setPartitionId(partitionId) .setLeader(preferredLeaderId) .setLeaderEpoch(0) .setPartitionEpoch(0) .setReplicas(replicas) - .setIsr(replicas) - ) + .setIsr(replicas), 0) } topicConfig.forKeyValue { (key, value) => - records.add(new ConfigRecord() + records += new ApiMessageAndVersion(new ConfigRecord() .setResourceName(topic) .setResourceType(ConfigResource.Type.TOPIC.id()) .setName(key) - .setValue(value) - ) + .setValue(value), 0) } - lastMetadataOffset += records.size() - listener.execCommits(lastOffset = lastMetadataOffset, records) + applyBatch(records.toList) assertTrue(metadataCache.contains(topic)) assertEquals(Some(numPartitions), metadataCache.numPartitions(topic)) assertEquals(topicConfig, configRepository.topicConfig(topic).asScala) diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 965dd2d64fb7b..bcfe75438c2e3 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -21,7 +21,7 @@ import java.io.{ByteArrayOutputStream, File, PrintWriter} import java.nio.ByteBuffer import java.util import java.util.Properties -import kafka.log.{Log, LogConfig, LogManager, LogTest} +import kafka.log.{Log, LogConfig, LogManager, LogTestUtils} import kafka.server.{BrokerTopicStats, LogDirFailureChannel} import kafka.tools.DumpLogSegments.TimeIndexDumpErrors import kafka.utils.{MockTime, TestUtils} @@ -30,7 +30,7 @@ import org.apache.kafka.common.metadata.{PartitionChangeRecord, RegisterBrokerRe import org.apache.kafka.common.protocol.{ByteBufferAccessor, ObjectSerializationCache} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.raft.metadata.MetadataRecordSerde +import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.server.common.ApiMessageAndVersion import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -181,8 +181,8 @@ class DumpLogSegmentsTest { @Test def testDumpMetadataRecords(): Unit = { val mockTime = new MockTime - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) - val log = LogTest.createLog(logDir, logConfig, new BrokerTopicStats, mockTime.scheduler, mockTime) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024) + val log = LogTestUtils.createLog(logDir, logConfig, new BrokerTopicStats, mockTime.scheduler, mockTime) val metadataRecords = Seq( new ApiMessageAndVersion( diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index 159ace8103e5a..93362e25a9d8c 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -123,7 +123,7 @@ class SchedulerTest { val topicPartition = Log.parseTopicPartitionName(logDir) val logDirFailureChannel = new LogDirFailureChannel(10) val segments = new LogSegments(topicPartition) - val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion, "") val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) val offsets = LogLoader.load(LoadLogParams( logDir, @@ -141,7 +141,7 @@ class SchedulerTest { producerStateManager)) val log = new Log(logDir, logConfig, segments = segments, logStartOffset = offsets.logStartOffset, recoveryPoint = offsets.recoveryPoint, nextOffsetMetadata = offsets.nextOffsetMetadata, scheduler, - brokerTopicStats, mockTime, maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, + brokerTopicStats, mockTime, LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, producerStateManager, logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) assertTrue(scheduler.taskRunning(log.producerExpireCheck)) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 7114d5dda99bd..53bc88e249874 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -1128,10 +1128,6 @@ object TestUtils extends Logging { } } - override def clearPending(topicPartition: TopicPartition): Unit = { - inFlight.set(false); - } - def completeIsrUpdate(newZkVersion: Int): Unit = { if (inFlight.compareAndSet(true, false)) { val item = isrUpdates.head diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index 04aaacf4294df..27c15a9dc1f79 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -19,7 +19,6 @@ package kafka.zk import java.util.{Collections, Properties} import java.nio.charset.StandardCharsets.UTF_8 import java.util.concurrent.{CountDownLatch, TimeUnit} - import kafka.api.{ApiVersion, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} import kafka.log.LogConfig @@ -53,6 +52,8 @@ import org.apache.kafka.common.security.JaasUtils import org.apache.zookeeper.ZooDefs import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.data.Stat +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource class KafkaZkClientTest extends ZooKeeperTestHarness { @@ -96,7 +97,7 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { def testConnectionViaNettyClient(): Unit = { // Confirm that we can explicitly set client connection configuration, which is necessary for TLS. // TLS connectivity itself is tested in system tests rather than here to avoid having to add TLS support - // to kafka.zk.EmbeddedZoopeeper + // to kafka.zk.EmbeddedZookeeper val clientConfig = new ZKClientConfig() val propKey = KafkaConfig.ZkClientCnxnSocketProp val propVal = "org.apache.zookeeper.ClientCnxnSocketNetty" @@ -116,6 +117,29 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { } } + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testChroot(createChrootIfNecessary: Boolean): Unit = { + val chroot = "/chroot" + val clientConfig = new ZKClientConfig() + val client = KafkaZkClient(zkConnect + chroot, zkAclsEnabled.getOrElse(JaasUtils.isZkSaslEnabled), zkSessionTimeout, + zkConnectionTimeout, zkMaxInFlightRequests, Time.SYSTEM, zkClientConfig = Some(clientConfig), createChrootIfNecessary = createChrootIfNecessary) + try { + client.createTopLevelPaths() + if (!createChrootIfNecessary) { + fail("We should not have been able to create top-level paths with a chroot when not explicitly creating the chroot path, but we were able to do so") + } + } catch { + case e: Exception => + if (createChrootIfNecessary) { + fail("We should have been able to create top-level paths with a chroot when explicitly creating the chroot path, but we failed to do so", + e) + } + } finally { + client.close() + } + } + @Test def testSetAndGetConsumerOffset(): Unit = { val offset = 123L diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index 599d9c8659c43..0392386a18e69 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -103,7 +103,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { def testConnectionViaNettyClient(): Unit = { // Confirm that we can explicitly set client connection configuration, which is necessary for TLS. // TLS connectivity itself is tested in system tests rather than here to avoid having to add TLS support - // to kafka.zk.EmbeddedZoopeeper + // to kafka.zk.EmbeddedZookeeper val clientConfig = new ZKClientConfig() val propKey = KafkaConfig.ZkClientCnxnSocketProp val propVal = "org.apache.zookeeper.ClientCnxnSocketNetty" diff --git a/docs/streams/developer-guide/app-reset-tool.html b/docs/streams/developer-guide/app-reset-tool.html index d6d07c27c97f2..597b662a5296f 100644 --- a/docs/streams/developer-guide/app-reset-tool.html +++ b/docs/streams/developer-guide/app-reset-tool.html @@ -78,44 +78,43 @@

Step 1: Run the application reset tool

Invoke the application reset tool from the command line

Warning! This tool makes irreversible changes to your application. It is strongly recommended that you run this once with --dry-run to preview your changes before making them.

-
<path-to-kafka>/bin/kafka-streams-application-reset
-
+
<path-to-kafka>/bin/kafka-streams-application-reset

The tool accepts the following parameters:

-
Option (* = required)                 Description
+            
Option (* = required)                 Description
 ---------------------                 -----------
 * --application-id <String: id>       The Kafka Streams application ID
-                                        (application.id).
+                                        (application.id).
 --bootstrap-servers <String: urls>    Comma-separated list of broker urls with
                                         format: HOST1:PORT1,HOST2:PORT2
-                                        (default: localhost:9092)
---by-duration <String: urls>      Reset offsets to offset by duration from
-                                        current timestamp. Format: 'PnDTnHnMnS'
+                                        (default: localhost:9092)
+--by-duration <String: urls>          Reset offsets to offset by duration from
+                                        current timestamp. Format: 'PnDTnHnMnS'
 --config-file <String: file name>     Property file containing configs to be
                                         passed to admin clients and embedded
                                         consumer.
 --dry-run                             Display the actions that would be
                                         performed without executing the reset
                                         commands.
---from-file <String: urls>        Reset offsets to values defined in CSV
+--from-file <String: urls>            Reset offsets to values defined in CSV
                                         file.
 --input-topics <String: list>         Comma-separated list of user input
                                         topics. For these topics, the tool will
                                         reset the offset to the earliest
                                         available offset.
 --intermediate-topics <String: list>  Comma-separated list of intermediate user
-                                        topics (topics used in the through()
-                                        method). For these topics, the tool
+                                        topics (topics used in the through()
+                                        method). For these topics, the tool
                                         will skip to the end.
 --internal-topics <String: list>      Comma-separated list of internal topics
                                         to delete. Must be a subset of the
                                         internal topics marked for deletion by
                                         the default behaviour (do a dry-run without
                                         this option to view these topics).
---shift-by <Long: number-of-offsets> Reset offsets shifting current offset by
-                                        'n', where 'n' can be positive or
+--shift-by <Long: number-of-offsets>  Reset offsets shifting current offset by
+                                        'n', where 'n' can be positive or
                                         negative
 --to-datetime <String>                Reset offsets to offset from datetime.
-                                        Format: 'YYYY-MM-DDTHH:mm:SS.sss'
+                                        Format: 'YYYY-MM-DDTHH:mm:SS.sss'
 --to-earliest                         Reset offsets to earliest offset.
 --to-latest                           Reset offsets to latest offset.
 --to-offset <Long>                    Reset offsets to a specific offset.
@@ -125,8 +124,7 @@ 

Step 1: Run the application reset toolCreate a java.util.Properties instance.

  • Set the parameters. For example:

    -
    import java.util.Properties;
    -import org.apache.kafka.streams.StreamsConfig;
    -
    -Properties settings = new Properties();
    -// Set a few key parameters
    -settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-first-streams-application");
    -settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
    -// Any further settings
    -settings.put(... , ...);
    -
    +
    import java.util.Properties;
    +import org.apache.kafka.streams.StreamsConfig;
    +
    +Properties settings = new Properties();
    +// Set a few key parameters
    +settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-first-streams-application");
    +settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
    +// Any further settings
    +settings.put(... , ...);
  • @@ -396,31 +395,31 @@

    acceptable.recovery.lag -
                  public class SendToDeadLetterQueueExceptionHandler implements DeserializationExceptionHandler {
    -                  KafkaProducer<byte[], byte[]> dlqProducer;
    -                  String dlqTopic;
    +              
    public class SendToDeadLetterQueueExceptionHandler implements DeserializationExceptionHandler {
    +    KafkaProducer<byte[], byte[]> dlqProducer;
    +    String dlqTopic;
     
    -                  @Override
    -                  public DeserializationHandlerResponse handle(final ProcessorContext context,
    -                                                               final ConsumerRecord<byte[], byte[]> record,
    -                                                               final Exception exception) {
    +    @Override
    +    public DeserializationHandlerResponse handle(final ProcessorContext context,
    +                                                 final ConsumerRecord<byte[], byte[]> record,
    +                                                 final Exception exception) {
     
    -                      log.warn("Exception caught during Deserialization, sending to the dead queue topic; " +
    -                          "taskId: {}, topic: {}, partition: {}, offset: {}",
    -                          context.taskId(), record.topic(), record.partition(), record.offset(),
    -                          exception);
    +        log.warn("Exception caught during Deserialization, sending to the dead queue topic; " +
    +            "taskId: {}, topic: {}, partition: {}, offset: {}",
    +            context.taskId(), record.topic(), record.partition(), record.offset(),
    +            exception);
     
    -                      dlqProducer.send(new ProducerRecord<>(dlqTopic, record.timestamp(), record.key(), record.value(), record.headers())).get();
    +        dlqProducer.send(new ProducerRecord<>(dlqTopic, record.timestamp(), record.key(), record.value(), record.headers())).get();
     
    -                      return DeserializationHandlerResponse.CONTINUE;
    -                  }
    +        return DeserializationHandlerResponse.CONTINUE;
    +    }
     
    -                  @Override
    -                  public void configure(final Map<String, ?> configs) {
    -                      dlqProducer = .. // get a producer from the configs map
    -                      dlqTopic = .. // get the topic name from the configs map
    -                  }
    -              }
    + @Override + public void configure(final Map<String, ?> configs) { + dlqProducer = .. // get a producer from the configs map + dlqTopic = .. // get the topic name from the configs map + } +}

    @@ -434,32 +433,31 @@

    acceptable.recovery.lag

    acceptable.recovery.lagpreviousTimestamp (i.e., a Kafka Streams timestamp estimation). Here is an example of a custom TimestampExtractor implementation:

    -
    import org.apache.kafka.clients.consumer.ConsumerRecord;
    -import org.apache.kafka.streams.processor.TimestampExtractor;
    -
    -// Extracts the embedded timestamp of a record (giving you "event-time" semantics).
    -public class MyEventTimeExtractor implements TimestampExtractor {
    -
    -  @Override
    -  public long extract(final ConsumerRecord<Object, Object> record, final long previousTimestamp) {
    -    // `Foo` is your own custom class, which we assume has a method that returns
    -    // the embedded timestamp (milliseconds since midnight, January 1, 1970 UTC).
    -    long timestamp = -1;
    -    final Foo myPojo = (Foo) record.value();
    -    if (myPojo != null) {
    -      timestamp = myPojo.getTimestampInMillis();
    -    }
    -    if (timestamp < 0) {
    -      // Invalid timestamp!  Attempt to estimate a new timestamp,
    -      // otherwise fall back to wall-clock time (processing-time).
    -      if (previousTimestamp >= 0) {
    -        return previousTimestamp;
    -      } else {
    -        return System.currentTimeMillis();
    -      }
    -    }
    -  }
    -
    -}
    -
    -
    +
    import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.streams.processor.TimestampExtractor;
    +
    +// Extracts the embedded timestamp of a record (giving you "event-time" semantics).
    +public class MyEventTimeExtractor implements TimestampExtractor {
    +
    +  @Override
    +  public long extract(final ConsumerRecord<Object, Object> record, final long previousTimestamp) {
    +    // `Foo` is your own custom class, which we assume has a method that returns
    +    // the embedded timestamp (milliseconds since midnight, January 1, 1970 UTC).
    +    long timestamp = -1;
    +    final Foo myPojo = (Foo) record.value();
    +    if (myPojo != null) {
    +      timestamp = myPojo.getTimestampInMillis();
    +    }
    +    if (timestamp < 0) {
    +      // Invalid timestamp!  Attempt to estimate a new timestamp,
    +      // otherwise fall back to wall-clock time (processing-time).
    +      if (previousTimestamp >= 0) {
    +        return previousTimestamp;
    +      } else {
    +        return System.currentTimeMillis();
    +      }
    +    }
    +  }
    +
    +}

    You would then define the custom timestamp extractor in your Streams configuration as follows:

    -
    import java.util.Properties;
    -import org.apache.kafka.streams.StreamsConfig;
    +              
    import java.util.Properties;
    +import org.apache.kafka.streams.StreamsConfig;
     
    -Properties streamsConfiguration = new Properties();
    -streamsConfiguration.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MyEventTimeExtractor.class);
    -
    +Properties streamsConfiguration = new Properties(); +streamsConfiguration.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MyEventTimeExtractor.class);

    @@ -707,38 +702,33 @@

    probing.rebalance.interval.ms

    The RocksDB configuration. Kafka Streams uses RocksDB as the default storage engine for persistent stores. To change the default configuration for RocksDB, you can implement RocksDBConfigSetter and provide your custom class via rocksdb.config.setter.

    Here is an example that adjusts the memory size consumed by RocksDB.

    -
    -
    -public static class CustomRocksDBConfig implements RocksDBConfigSetter {
    -    // This object should be a member variable so it can be closed in RocksDBConfigSetter#close.
    -    private org.rocksdb.Cache cache = new org.rocksdb.LRUCache(16 * 1024L * 1024L);
    -
    -    @Override
    -    public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) {
    -        // See #1 below.
    -        BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();
    -        tableConfig.setBlockCache(cache);
    -        // See #2 below.
    -        tableConfig.setBlockSize(16 * 1024L);
    -        // See #3 below.
    -        tableConfig.setCacheIndexAndFilterBlocks(true);
    -        options.setTableFormatConfig(tableConfig);
    -        // See #4 below.
    -        options.setMaxWriteBufferNumber(2);
    -    }
    -
    -    @Override
    -    public void close(final String storeName, final Options options) {
    -        // See #5 below.
    -        cache.close();
    -    }
    -}
    -
    -Properties streamsSettings = new Properties();
    -streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksDBConfig.class);
    -
    -
    -
    +
    public static class CustomRocksDBConfig implements RocksDBConfigSetter {
    +    // This object should be a member variable so it can be closed in RocksDBConfigSetter#close.
    +    private org.rocksdb.Cache cache = new org.rocksdb.LRUCache(16 * 1024L * 1024L);
    +
    +    @Override
    +    public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) {
    +        // See #1 below.
    +        BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();
    +        tableConfig.setBlockCache(cache);
    +        // See #2 below.
    +        tableConfig.setBlockSize(16 * 1024L);
    +        // See #3 below.
    +        tableConfig.setCacheIndexAndFilterBlocks(true);
    +        options.setTableFormatConfig(tableConfig);
    +        // See #4 below.
    +        options.setMaxWriteBufferNumber(2);
    +    }
    +
    +    @Override
    +    public void close(final String storeName, final Options options) {
    +        // See #5 below.
    +        cache.close();
    +    }
    +}
    +
    +Properties streamsSettings = new Properties();
    +streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksDBConfig.class);
    Notes for example:
      @@ -798,12 +788,12 @@

      Kafka consumers, producer and admin clie and admin client that are used internally. The consumer, producer and admin client settings are defined by specifying parameters in a StreamsConfig instance.

      In this example, the Kafka consumer session timeout is configured to be 60000 milliseconds in the Streams settings:

      -
      Properties streamsSettings = new Properties();
      -// Example of a "normal" setting for Kafka Streams
      -streamsSettings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker-01:9092");
      -// Customize the Kafka consumer settings of your Streams application
      -streamsSettings.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 60000);
      -
      +
      Properties streamsSettings = new Properties();
      +// Example of a "normal" setting for Kafka Streams
      +streamsSettings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker-01:9092");
      +// Customize the Kafka consumer settings of your Streams application
      +streamsSettings.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 60000);
      +

      Naming

      @@ -811,18 +801,17 @@

      Namingreceive.buffer.bytes are used to configure TCP buffers; request.timeout.ms and retry.backoff.ms control retries for client request; retries are used to configure how many retries are allowed when handling retriable errors from broker request responses. You can avoid duplicate names by prefix parameter names with consumer., producer., or admin. (e.g., consumer.send.buffer.bytes and producer.send.buffer.bytes).

      -
      Properties streamsSettings = new Properties();
      -// same value for consumer, producer, and admin client
      -streamsSettings.put("PARAMETER_NAME", "value");
      -// different values for consumer and producer
      -streamsSettings.put("consumer.PARAMETER_NAME", "consumer-value");
      -streamsSettings.put("producer.PARAMETER_NAME", "producer-value");
      -streamsSettings.put("admin.PARAMETER_NAME", "admin-value");
      -// alternatively, you can use
      -streamsSettings.put(StreamsConfig.consumerPrefix("PARAMETER_NAME"), "consumer-value");
      -streamsSettings.put(StreamsConfig.producerPrefix("PARAMETER_NAME"), "producer-value");
      -streamsSettings.put(StreamsConfig.adminClientPrefix("PARAMETER_NAME"), "admin-value");
      -
      +
      Properties streamsSettings = new Properties();
      +// same value for consumer, producer, and admin client
      +streamsSettings.put("PARAMETER_NAME", "value");
      +// different values for consumer and producer
      +streamsSettings.put("consumer.PARAMETER_NAME", "consumer-value");
      +streamsSettings.put("producer.PARAMETER_NAME", "producer-value");
      +streamsSettings.put("admin.PARAMETER_NAME", "admin-value");
      +// alternatively, you can use
      +streamsSettings.put(StreamsConfig.consumerPrefix("PARAMETER_NAME"), "consumer-value");
      +streamsSettings.put(StreamsConfig.producerPrefix("PARAMETER_NAME"), "producer-value");
      +streamsSettings.put(StreamsConfig.adminClientPrefix("PARAMETER_NAME"), "admin-value");

      You could further separate consumer configuration by adding different prefixes:

      For example, if you only want to set restore consumer config without touching other consumers' settings, you could simply use restore.consumer. to set the config.

      -
      Properties streamsSettings = new Properties();
      -// same config value for all consumer types
      -streamsSettings.put("consumer.PARAMETER_NAME", "general-consumer-value");
      -// set a different restore consumer config. This would make restore consumer take restore-consumer-value,
      -// while main consumer and global consumer stay with general-consumer-value
      -streamsSettings.put("restore.consumer.PARAMETER_NAME", "restore-consumer-value");
      -// alternatively, you can use
      -streamsSettings.put(StreamsConfig.restoreConsumerPrefix("PARAMETER_NAME"), "restore-consumer-value");
      -
      -
      +
      Properties streamsSettings = new Properties();
      +// same config value for all consumer types
      +streamsSettings.put("consumer.PARAMETER_NAME", "general-consumer-value");
      +// set a different restore consumer config. This would make restore consumer take restore-consumer-value,
      +// while main consumer and global consumer stay with general-consumer-value
      +streamsSettings.put("restore.consumer.PARAMETER_NAME", "restore-consumer-value");
      +// alternatively, you can use
      +streamsSettings.put(StreamsConfig.restoreConsumerPrefix("PARAMETER_NAME"), "restore-consumer-value");

      Same applied to main.consumer. and main.consumer., if you only want to specify one consumer type config.

      Additionally, to configure the internal repartition/changelog topics, you could use the topic. prefix, followed by any of the standard topic configs.

      -
      Properties streamsSettings = new Properties();
      -// Override default for both changelog and repartition topics
      -streamsSettings.put("topic.PARAMETER_NAME", "topic-value");
      -// alternatively, you can use
      -streamsSettings.put(StreamsConfig.topicPrefix("PARAMETER_NAME"), "topic-value");
      -
      -
      +
      Properties streamsSettings = new Properties();
      +// Override default for both changelog and repartition topics
      +streamsSettings.put("topic.PARAMETER_NAME", "topic-value");
      +// alternatively, you can use
      +streamsSettings.put(StreamsConfig.topicPrefix("PARAMETER_NAME"), "topic-value");

      @@ -977,11 +962,11 @@

      acksreplication.factor

      -
      Properties streamsSettings = new Properties();
      -streamsSettings.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3);
      -streamsSettings.put(StreamsConfig.topicPrefix(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG), 2);
      -streamsSettings.put(StreamsConfig.producerPrefix(ProducerConfig.ACKS_CONFIG), "all");
      -
      +
      Properties streamsSettings = new Properties();
      +streamsSettings.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3);
      +streamsSettings.put(StreamsConfig.topicPrefix(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG), 2);
      +streamsSettings.put(StreamsConfig.producerPrefix(ProducerConfig.ACKS_CONFIG), "all");
      +
      diff --git a/docs/streams/developer-guide/datatypes.html b/docs/streams/developer-guide/datatypes.html index 2201b5b69d35a..f527021dc4911 100644 --- a/docs/streams/developer-guide/datatypes.html +++ b/docs/streams/developer-guide/datatypes.html @@ -55,40 +55,37 @@

      Configuring SerDes

      SerDes specified in the Streams configuration are used as the default in your Kafka Streams application.

      -
      import org.apache.kafka.common.serialization.Serdes;
      -import org.apache.kafka.streams.StreamsConfig;
      -
      -Properties settings = new Properties();
      -// Default serde for keys of data records (here: built-in serde for String type)
      -settings.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
      -// Default serde for values of data records (here: built-in serde for Long type)
      -settings.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass().getName());
      -
      +
      import org.apache.kafka.common.serialization.Serdes;
      +import org.apache.kafka.streams.StreamsConfig;
      +
      +Properties settings = new Properties();
      +// Default serde for keys of data records (here: built-in serde for String type)
      +settings.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
      +// Default serde for values of data records (here: built-in serde for Long type)
      +settings.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass().getName());

      Overriding default SerDes

      You can also specify SerDes explicitly by passing them to the appropriate API methods, which overrides the default serde settings:

      -
      import org.apache.kafka.common.serialization.Serde;
      -import org.apache.kafka.common.serialization.Serdes;
      +      
      import org.apache.kafka.common.serialization.Serde;
      +import org.apache.kafka.common.serialization.Serdes;
       
      -final Serde<String> stringSerde = Serdes.String();
      -final Serde<Long> longSerde = Serdes.Long();
      +final Serde<String> stringSerde = Serdes.String();
      +final Serde<Long> longSerde = Serdes.Long();
       
      -// The stream userCountByRegion has type `String` for record keys (for region)
      -// and type `Long` for record values (for user counts).
      -KStream<String, Long> userCountByRegion = ...;
      -userCountByRegion.to("RegionCountsTopic", Produced.with(stringSerde, longSerde));
      -
      +// The stream userCountByRegion has type `String` for record keys (for region) +// and type `Long` for record values (for user counts). +KStream<String, Long> userCountByRegion = ...; +userCountByRegion.to("RegionCountsTopic", Produced.with(stringSerde, longSerde));

      If you want to override serdes selectively, i.e., keep the defaults for some fields, then don’t specify the serde whenever you want to leverage the default settings:

      -
      import org.apache.kafka.common.serialization.Serde;
      -import org.apache.kafka.common.serialization.Serdes;
      -
      -// Use the default serializer for record keys (here: region as String) by not specifying the key serde,
      -// but override the default serializer for record values (here: userCount as Long).
      -final Serde<Long> longSerde = Serdes.Long();
      -KStream<String, Long> userCountByRegion = ...;
      -userCountByRegion.to("RegionCountsTopic", Produced.valueSerde(Serdes.Long()));
      -
      +
      import org.apache.kafka.common.serialization.Serde;
      +import org.apache.kafka.common.serialization.Serdes;
      +
      +// Use the default serializer for record keys (here: region as String) by not specifying the key serde,
      +// but override the default serializer for record values (here: userCount as Long).
      +final Serde<Long> longSerde = Serdes.Long();
      +KStream<String, Long> userCountByRegion = ...;
      +userCountByRegion.to("RegionCountsTopic", Produced.valueSerde(Serdes.Long()));

      If some of your incoming records are corrupted or ill-formatted, they will cause the deserializer class to report an error. Since 1.0.x we have introduced an DeserializationExceptionHandler interface which allows you to customize how to handle such records. The customized implementation of the interface can be specified via the StreamsConfig. @@ -101,12 +98,11 @@

      Overriding default SerDes

      Apache Kafka includes several built-in serde implementations for Java primitives and basic types such as byte[] in its kafka-clients Maven artifact:

      -
      <dependency>
      -    <groupId>org.apache.kafka</groupId>
      -    <artifactId>kafka-clients</artifactId>
      -    <version>{{fullDotVersion}}</version>
      -</dependency>
      -
      +
      <dependency>
      +    <groupId>org.apache.kafka</groupId>
      +    <artifactId>kafka-clients</artifactId>
      +    <version>2.8.0</version>
      +</dependency>

      This artifact provides the following serde implementations under the package org.apache.kafka.common.serialization, which you can leverage when e.g., defining default serializers in your Streams configuration.

      diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index 2add551d61137..d2bce047868c7 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -242,19 +242,18 @@

      import org.apache.kafka.common.serialization.Serdes;
      -import org.apache.kafka.streams.StreamsBuilder;
      -import org.apache.kafka.streams.kstream.KStream;
      -
      -StreamsBuilder builder = new StreamsBuilder();
      -
      -KStream<String, Long> wordCounts = builder.stream(
      -    "word-counts-input-topic", /* input topic */
      -    Consumed.with(
      -      Serdes.String(), /* key serde */
      -      Serdes.Long()   /* value serde */
      -    );
      - +
      import org.apache.kafka.common.serialization.Serdes;
      +import org.apache.kafka.streams.StreamsBuilder;
      +import org.apache.kafka.streams.kstream.KStream;
      +
      +StreamsBuilder builder = new StreamsBuilder();
      +
      +KStream<String, Long> wordCounts = builder.stream(
      +    "word-counts-input-topic", /* input topic */
      +    Consumed.with(
      +      Serdes.String(), /* key serde */
      +      Serdes.Long()   /* value serde */
      +    );

      If you do not specify SerDes explicitly, the default SerDes from the configuration are used.

      You must specify SerDes explicitly if the key or value types of the records in the Kafka input @@ -303,20 +302,19 @@

      state store that backs the table). This is required for supporting interactive queries against the table. When a name is not provided the table will not be queryable and an internal name will be provided for the state store.

      -
      import org.apache.kafka.common.serialization.Serdes;
      -import org.apache.kafka.streams.StreamsBuilder;
      -import org.apache.kafka.streams.kstream.GlobalKTable;
      -
      -StreamsBuilder builder = new StreamsBuilder();
      -
      -GlobalKTable<String, Long> wordCounts = builder.globalTable(
      -    "word-counts-input-topic",
      -    Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as(
      -      "word-counts-global-store" /* table/store name */)
      -      .withKeySerde(Serdes.String()) /* key serde */
      -      .withValueSerde(Serdes.Long()) /* value serde */
      -    );
      -
      +
      import org.apache.kafka.common.serialization.Serdes;
      +import org.apache.kafka.streams.StreamsBuilder;
      +import org.apache.kafka.streams.kstream.GlobalKTable;
      +
      +StreamsBuilder builder = new StreamsBuilder();
      +
      +GlobalKTable<String, Long> wordCounts = builder.globalTable(
      +    "word-counts-input-topic",
      +    Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as(
      +      "word-counts-global-store" /* table/store name */)
      +      .withKeySerde(Serdes.String()) /* key serde */
      +      .withValueSerde(Serdes.Long()) /* value serde */
      +    );

      You must specify SerDes explicitly if the key or value types of the records in the Kafka input topics do not match the configured default SerDes. For information about configuring default SerDes, available SerDes, and implementing your own custom SerDes see Data Types and Serialization.

      @@ -371,23 +369,21 @@

      -KStream stream = ...; -Map> branches = - stream.split(Named.as("Branch-")) - .branch((key, value) -> key.startsWith("A"), /* first predicate */ - Branched.as("A")) - .branch((key, value) -> key.startsWith("B"), /* second predicate */ - Branched.as("B")) -.defaultBranch(Branched.as("C")) +
      KStream<String, Long> stream = ...;
      +Map<String, KStream<String, Long>> branches =
      +    stream.split(Named.as("Branch-"))
      +        .branch((key, value) -> key.startsWith("A"),  /* first predicate  */
      +             Branched.as("A"))
      +        .branch((key, value) -> key.startsWith("B"),  /* second predicate */
      +             Branched.as("B"))
      +.defaultBranch(Branched.as("C"))
       );
       
      -// KStream branches.get("Branch-A") contains all records whose keys start with "A"
      -// KStream branches.get("Branch-B") contains all records whose keys start with "B"
      -// KStream branches.get("Branch-C") contains all other records
      +// KStream branches.get("Branch-A") contains all records whose keys start with "A"
      +// KStream branches.get("Branch-B") contains all records whose keys start with "B"
      +// KStream branches.get("Branch-C") contains all other records
       
      -// Java 7 example: cf. `filter` for how to create `Predicate` instances     
      -                            
      +// Java 7 example: cf. `filter` for how to create `Predicate` instances

      @@ -861,9 +838,8 @@

      repartition() operation always triggers repartitioning of the stream, as a result it can be used with embedded Processor API methods (like transform() et al.) that do not trigger auto repartitioning when key changing operation is performed beforehand. -
      KStream<byte[], String> stream = ... ;
      -KStream<byte[], String> repartitionedStream = stream.repartition(Repartitioned.numberOfPartitions(10));
      -
      +
      KStream<byte[], String> stream = ... ;
      +KStream<byte[], String> repartitionedStream = stream.repartition(Repartitioned.numberOfPartitions(10));

      @@ -898,45 +874,43 @@

      // Assume the record values represent lines of text.  For the sake of this example, you can ignore
      -// whatever may be stored in the record keys.
      -KStream<String, String> textLines = ...;
      -
      -KStream<String, Long> wordCounts = textLines
      -    // Split each text line, by whitespace, into words.  The text lines are the record
      -    // values, i.e. you can ignore whatever data is in the record keys and thus invoke
      -    // `flatMapValues` instead of the more generic `flatMap`.
      -    .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
      -    // Group the stream by word to ensure the key of the record is the word.
      -    .groupBy((key, word) -> word)
      -    // Count the occurrences of each word (record key).
      -    //
      -    // This will change the stream type from `KGroupedStream<String, String>` to
      -    // `KTable<String, Long>` (word -> count).
      -    .count()
      -    // Convert the `KTable<String, Long>` into a `KStream<String, Long>`.
      -    .toStream();
      - +
      // Assume the record values represent lines of text.  For the sake of this example, you can ignore
      +// whatever may be stored in the record keys.
      +KStream<String, String> textLines = ...;
      +
      +KStream<String, Long> wordCounts = textLines
      +    // Split each text line, by whitespace, into words.  The text lines are the record
      +    // values, i.e. you can ignore whatever data is in the record keys and thus invoke
      +    // `flatMapValues` instead of the more generic `flatMap`.
      +    .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
      +    // Group the stream by word to ensure the key of the record is the word.
      +    .groupBy((key, word) -> word)
      +    // Count the occurrences of each word (record key).
      +    //
      +    // This will change the stream type from `KGroupedStream<String, String>` to
      +    // `KTable<String, Long>` (word -> count).
      +    .count()
      +    // Convert the `KTable<String, Long>` into a `KStream<String, Long>`.
      +    .toStream();

      WordCount example in Java 7:

      -
      // Code below is equivalent to the previous Java 8+ example above.
      -KStream<String, String> textLines = ...;
      -
      -KStream<String, Long> wordCounts = textLines
      -    .flatMapValues(new ValueMapper<String, Iterable<String>>() {
      -        @Override
      -        public Iterable<String> apply(String value) {
      -            return Arrays.asList(value.toLowerCase().split("\\W+"));
      -        }
      -    })
      -    .groupBy(new KeyValueMapper<String, String, String>>() {
      -        @Override
      -        public String apply(String key, String word) {
      -            return word;
      -        }
      -    })
      -    .count()
      -    .toStream();
      -
      +
      // Code below is equivalent to the previous Java 8+ example above.
      +KStream<String, String> textLines = ...;
      +
      +KStream<String, Long> wordCounts = textLines
      +    .flatMapValues(new ValueMapper<String, Iterable<String>>() {
      +        @Override
      +        public Iterable<String> apply(String value) {
      +            return Arrays.asList(value.toLowerCase().split("\\W+"));
      +        }
      +    })
      +    .groupBy(new KeyValueMapper<String, String, String>>() {
      +        @Override
      +        public String apply(String key, String word) {
      +            return word;
      +        }
      +    })
      +    .count()
      +    .toStream();

      Aggregating

      After records are grouped by key via groupByKey or @@ -973,69 +947,68 @@

      aggValue = 0)

      Several variants of aggregate exist, see Javadocs for details.

      -
      KGroupedStream<byte[], String> groupedStream = ...;
      -KGroupedTable<byte[], String> groupedTable = ...;
      -
      -// Java 8+ examples, using lambda expressions
      -
      -// Aggregating a KGroupedStream (note how the value type changes from String to Long)
      -KTable<byte[], Long> aggregatedStream = groupedStream.aggregate(
      -    () -> 0L, /* initializer */
      -    (aggKey, newValue, aggValue) -> aggValue + newValue.length(), /* adder */
      -    Materialized.as("aggregated-stream-store") /* state store name */
      -        .withValueSerde(Serdes.Long()); /* serde for aggregate value */
      -
      -// Aggregating a KGroupedTable (note how the value type changes from String to Long)
      -KTable<byte[], Long> aggregatedTable = groupedTable.aggregate(
      -    () -> 0L, /* initializer */
      -    (aggKey, newValue, aggValue) -> aggValue + newValue.length(), /* adder */
      -    (aggKey, oldValue, aggValue) -> aggValue - oldValue.length(), /* subtractor */
      -    Materialized.as("aggregated-table-store") /* state store name */
      -	.withValueSerde(Serdes.Long()) /* serde for aggregate value */
      -
      -
      -// Java 7 examples
      -
      -// Aggregating a KGroupedStream (note how the value type changes from String to Long)
      -KTable<byte[], Long> aggregatedStream = groupedStream.aggregate(
      -    new Initializer<Long>() { /* initializer */
      -      @Override
      -      public Long apply() {
      -        return 0L;
      -      }
      -    },
      -    new Aggregator<byte[], String, Long>() { /* adder */
      -      @Override
      -      public Long apply(byte[] aggKey, String newValue, Long aggValue) {
      -        return aggValue + newValue.length();
      -      }
      -    },
      -    Materialized.as("aggregated-stream-store")
      -        .withValueSerde(Serdes.Long());
      -
      -// Aggregating a KGroupedTable (note how the value type changes from String to Long)
      -KTable<byte[], Long> aggregatedTable = groupedTable.aggregate(
      -    new Initializer<Long>() { /* initializer */
      -      @Override
      -      public Long apply() {
      -        return 0L;
      -      }
      -    },
      -    new Aggregator<byte[], String, Long>() { /* adder */
      -      @Override
      -      public Long apply(byte[] aggKey, String newValue, Long aggValue) {
      -        return aggValue + newValue.length();
      -      }
      -    },
      -    new Aggregator<byte[], String, Long>() { /* subtractor */
      -      @Override
      -      public Long apply(byte[] aggKey, String oldValue, Long aggValue) {
      -        return aggValue - oldValue.length();
      -      }
      -    },
      -    Materialized.as("aggregated-stream-store")
      -        .withValueSerde(Serdes.Long());
      -
      +
      KGroupedStream<byte[], String> groupedStream = ...;
      +KGroupedTable<byte[], String> groupedTable = ...;
      +
      +// Java 8+ examples, using lambda expressions
      +
      +// Aggregating a KGroupedStream (note how the value type changes from String to Long)
      +KTable<byte[], Long> aggregatedStream = groupedStream.aggregate(
      +    () -> 0L, /* initializer */
      +    (aggKey, newValue, aggValue) -> aggValue + newValue.length(), /* adder */
      +    Materialized.as("aggregated-stream-store") /* state store name */
      +        .withValueSerde(Serdes.Long()); /* serde for aggregate value */
      +
      +// Aggregating a KGroupedTable (note how the value type changes from String to Long)
      +KTable<byte[], Long> aggregatedTable = groupedTable.aggregate(
      +    () -> 0L, /* initializer */
      +    (aggKey, newValue, aggValue) -> aggValue + newValue.length(), /* adder */
      +    (aggKey, oldValue, aggValue) -> aggValue - oldValue.length(), /* subtractor */
      +    Materialized.as("aggregated-table-store") /* state store name */
      +	.withValueSerde(Serdes.Long()) /* serde for aggregate value */
      +
      +
      +// Java 7 examples
      +
      +// Aggregating a KGroupedStream (note how the value type changes from String to Long)
      +KTable<byte[], Long> aggregatedStream = groupedStream.aggregate(
      +    new Initializer<Long>() { /* initializer */
      +      @Override
      +      public Long apply() {
      +        return 0L;
      +      }
      +    },
      +    new Aggregator<byte[], String, Long>() { /* adder */
      +      @Override
      +      public Long apply(byte[] aggKey, String newValue, Long aggValue) {
      +        return aggValue + newValue.length();
      +      }
      +    },
      +    Materialized.as("aggregated-stream-store")
      +        .withValueSerde(Serdes.Long());
      +
      +// Aggregating a KGroupedTable (note how the value type changes from String to Long)
      +KTable<byte[], Long> aggregatedTable = groupedTable.aggregate(
      +    new Initializer<Long>() { /* initializer */
      +      @Override
      +      public Long apply() {
      +        return 0L;
      +      }
      +    },
      +    new Aggregator<byte[], String, Long>() { /* adder */
      +      @Override
      +      public Long apply(byte[] aggKey, String newValue, Long aggValue) {
      +        return aggValue + newValue.length();
      +      }
      +    },
      +    new Aggregator<byte[], String, Long>() { /* subtractor */
      +      @Override
      +      public Long apply(byte[] aggKey, String oldValue, Long aggValue) {
      +        return aggValue - oldValue.length();
      +      }
      +    },
      +    Materialized.as("aggregated-stream-store")
      +        .withValueSerde(Serdes.Long());

      Detailed behavior of KGroupedStream:

      Filter

      @@ -399,21 +395,20 @@

      Evaluates a boolean function for each element and retains those for which the function returns true. (KStream details, KTable details)

      -
      KStream<String, Long> stream = ...;
      -
      -// A filter that selects (keeps) only positive numbers
      -// Java 8+ example, using lambda expressions
      -KStream<String, Long> onlyPositives = stream.filter((key, value) -> value > 0);
      -
      -// Java 7 example
      -KStream<String, Long> onlyPositives = stream.filter(
      -    new Predicate<String, Long>() {
      -      @Override
      -      public boolean test(String key, Long value) {
      -        return value > 0;
      -      }
      -    });
      -
      +
      KStream<String, Long> stream = ...;
      +
      +// A filter that selects (keeps) only positive numbers
      +// Java 8+ example, using lambda expressions
      +KStream<String, Long> onlyPositives = stream.filter((key, value) -> value > 0);
      +
      +// Java 7 example
      +KStream<String, Long> onlyPositives = stream.filter(
      +    new Predicate<String, Long>() {
      +      @Override
      +      public boolean test(String key, Long value) {
      +        return value > 0;
      +      }
      +    });

      Inverse Filter

      @@ -425,21 +420,20 @@

      Evaluates a boolean function for each element and drops those for which the function returns true. (KStream details, KTable details)

      -
      KStream<String, Long> stream = ...;
      -
      -// An inverse filter that discards any negative numbers or zero
      -// Java 8+ example, using lambda expressions
      -KStream<String, Long> onlyPositives = stream.filterNot((key, value) -> value <= 0);
      -
      -// Java 7 example
      -KStream<String, Long> onlyPositives = stream.filterNot(
      -    new Predicate<String, Long>() {
      -      @Override
      -      public boolean test(String key, Long value) {
      -        return value <= 0;
      -      }
      -    });
      -
      +
      KStream<String, Long> stream = ...;
      +
      +// An inverse filter that discards any negative numbers or zero
      +// Java 8+ example, using lambda expressions
      +KStream<String, Long> onlyPositives = stream.filterNot((key, value) -> value <= 0);
      +
      +// Java 7 example
      +KStream<String, Long> onlyPositives = stream.filterNot(
      +    new Predicate<String, Long>() {
      +      @Override
      +      public boolean test(String key, Long value) {
      +        return value <= 0;
      +      }
      +    });

      FlatMap

      @@ -453,21 +447,20 @@

      flatMap will result in re-partitioning of the records. If possible use flatMapValues instead, which will not cause data re-partitioning.

      -
      KStream<Long, String> stream = ...;
      -KStream<String, Integer> transformed = stream.flatMap(
      -     // Here, we generate two output records for each input record.
      -     // We also change the key and value types.
      -     // Example: (345L, "Hello") -> ("HELLO", 1000), ("hello", 9000)
      -    (key, value) -> {
      -      List<KeyValue<String, Integer>> result = new LinkedList<>();
      -      result.add(KeyValue.pair(value.toUpperCase(), 1000));
      -      result.add(KeyValue.pair(value.toLowerCase(), 9000));
      -      return result;
      -    }
      -  );
      -
      -// Java 7 example: cf. `map` for how to create `KeyValueMapper` instances
      -
      +
      KStream<Long, String> stream = ...;
      +KStream<String, Integer> transformed = stream.flatMap(
      +     // Here, we generate two output records for each input record.
      +     // We also change the key and value types.
      +     // Example: (345L, "Hello") -> ("HELLO", 1000), ("hello", 9000)
      +    (key, value) -> {
      +      List<KeyValue<String, Integer>> result = new LinkedList<>();
      +      result.add(KeyValue.pair(value.toUpperCase(), 1000));
      +      result.add(KeyValue.pair(value.toLowerCase(), 9000));
      +      return result;
      +    }
      +  );
      +
      +// Java 7 example: cf. `map` for how to create `KeyValueMapper` instances

      FlatMap (values only)

      @@ -480,12 +473,11 @@

      details)

      flatMapValues is preferable to flatMap because it will not cause data re-partitioning. However, you cannot modify the key or key type like flatMap does.

      -
      // Split a sentence into words.
      -KStream<byte[], String> sentences = ...;
      -KStream<byte[], String> words = sentences.flatMapValues(value -> Arrays.asList(value.split("\\s+")));
      +                            
      // Split a sentence into words.
      +KStream<byte[], String> sentences = ...;
      +KStream<byte[], String> words = sentences.flatMapValues(value -> Arrays.asList(value.split("\\s+")));
       
      -// Java 7 example: cf. `mapValues` for how to create `ValueMapper` instances
      -
      +// Java 7 example: cf. `mapValues` for how to create `ValueMapper` instances

      Foreach

      @@ -501,21 +493,20 @@

      peek, which is not a terminal operation).

      Note on processing guarantees: Any side effects of an action (such as writing to external systems) are not trackable by Kafka, which means they will typically not benefit from Kafka’s processing guarantees.

      -
      KStream<String, Long> stream = ...;
      -
      -// Print the contents of the KStream to the local console.
      -// Java 8+ example, using lambda expressions
      -stream.foreach((key, value) -> System.out.println(key + " => " + value));
      -
      -// Java 7 example
      -stream.foreach(
      -    new ForeachAction<String, Long>() {
      -      @Override
      -      public void apply(String key, Long value) {
      -        System.out.println(key + " => " + value);
      -      }
      -    });
      -
      +
      KStream<String, Long> stream = ...;
      +
      +// Print the contents of the KStream to the local console.
      +// Java 8+ example, using lambda expressions
      +stream.foreach((key, value) -> System.out.println(key + " => " + value));
      +
      +// Java 7 example
      +stream.foreach(
      +    new ForeachAction<String, Long>() {
      +      @Override
      +      public void apply(String key, Long value) {
      +        System.out.println(key + " => " + value);
      +      }
      +    });

      GroupByKey

      @@ -543,20 +534,19 @@

      groupByKey is preferable to groupBy because it re-partitions data only if the stream was already marked for re-partitioning. However, groupByKey does not allow you to modify the key or key type like groupBy does.

      -
      KStream<byte[], String> stream = ...;
      -
      -// Group by the existing key, using the application's configured
      -// default serdes for keys and values.
      -KGroupedStream<byte[], String> groupedStream = stream.groupByKey();
      -
      -// When the key and/or value types do not match the configured
      -// default serdes, we must explicitly specify serdes.
      -KGroupedStream<byte[], String> groupedStream = stream.groupByKey(
      -    Grouped.with(
      -      Serdes.ByteArray(), /* key */
      -      Serdes.String())     /* value */
      -  );
      -
      +
      KStream<byte[], String> stream = ...;
      +
      +// Group by the existing key, using the application's configured
      +// default serdes for keys and values.
      +KGroupedStream<byte[], String> groupedStream = stream.groupByKey();
      +
      +// When the key and/or value types do not match the configured
      +// default serdes, we must explicitly specify serdes.
      +KGroupedStream<byte[], String> groupedStream = stream.groupByKey(
      +    Grouped.with(
      +      Serdes.ByteArray(), /* key */
      +      Serdes.String())     /* value */
      +  );

      GroupBy

      @@ -586,56 +576,55 @@

      groupBy always causes data re-partitioning. If possible use groupByKey instead, which will re-partition data only if required.

      -
      KStream<byte[], String> stream = ...;
      -KTable<byte[], String> table = ...;
      -
      -// Java 8+ examples, using lambda expressions
      -
      -// Group the stream by a new key and key type
      -KGroupedStream<String, String> groupedStream = stream.groupBy(
      -    (key, value) -> value,
      -    Grouped.with(
      -      Serdes.String(), /* key (note: type was modified) */
      -      Serdes.String())  /* value */
      -  );
      -
      -// Group the table by a new key and key type, and also modify the value and value type.
      -KGroupedTable<String, Integer> groupedTable = table.groupBy(
      -    (key, value) -> KeyValue.pair(value, value.length()),
      -    Grouped.with(
      -      Serdes.String(), /* key (note: type was modified) */
      -      Serdes.Integer()) /* value (note: type was modified) */
      -  );
      -
      -
      -// Java 7 examples
      -
      -// Group the stream by a new key and key type
      -KGroupedStream<String, String> groupedStream = stream.groupBy(
      -    new KeyValueMapper<byte[], String, String>>() {
      -      @Override
      -      public String apply(byte[] key, String value) {
      -        return value;
      -      }
      -    },
      -    Grouped.with(
      -      Serdes.String(), /* key (note: type was modified) */
      -      Serdes.String())  /* value */
      -  );
      -
      -// Group the table by a new key and key type, and also modify the value and value type.
      -KGroupedTable<String, Integer> groupedTable = table.groupBy(
      -    new KeyValueMapper<byte[], String, KeyValue<String, Integer>>() {
      -      @Override
      -      public KeyValue<String, Integer> apply(byte[] key, String value) {
      -        return KeyValue.pair(value, value.length());
      -      }
      -    },
      -    Grouped.with(
      -      Serdes.String(), /* key (note: type was modified) */
      -      Serdes.Integer()) /* value (note: type was modified) */
      -  );
      -
      +
      KStream<byte[], String> stream = ...;
      +KTable<byte[], String> table = ...;
      +
      +// Java 8+ examples, using lambda expressions
      +
      +// Group the stream by a new key and key type
      +KGroupedStream<String, String> groupedStream = stream.groupBy(
      +    (key, value) -> value,
      +    Grouped.with(
      +      Serdes.String(), /* key (note: type was modified) */
      +      Serdes.String())  /* value */
      +  );
      +
      +// Group the table by a new key and key type, and also modify the value and value type.
      +KGroupedTable<String, Integer> groupedTable = table.groupBy(
      +    (key, value) -> KeyValue.pair(value, value.length()),
      +    Grouped.with(
      +      Serdes.String(), /* key (note: type was modified) */
      +      Serdes.Integer()) /* value (note: type was modified) */
      +  );
      +
      +
      +// Java 7 examples
      +
      +// Group the stream by a new key and key type
      +KGroupedStream<String, String> groupedStream = stream.groupBy(
      +    new KeyValueMapper<byte[], String, String>>() {
      +      @Override
      +      public String apply(byte[] key, String value) {
      +        return value;
      +      }
      +    },
      +    Grouped.with(
      +      Serdes.String(), /* key (note: type was modified) */
      +      Serdes.String())  /* value */
      +  );
      +
      +// Group the table by a new key and key type, and also modify the value and value type.
      +KGroupedTable<String, Integer> groupedTable = table.groupBy(
      +    new KeyValueMapper<byte[], String, KeyValue<String, Integer>>() {
      +      @Override
      +      public KeyValue<String, Integer> apply(byte[] key, String value) {
      +        return KeyValue.pair(value, value.length());
      +      }
      +    },
      +    Grouped.with(
      +      Serdes.String(), /* key (note: type was modified) */
      +      Serdes.Integer()) /* value (note: type was modified) */
      +  );

      Cogroup

      @@ -650,19 +639,18 @@

      windowed before it is aggregated.

      Cogroup does not cause a repartition as it has the prerequisite that the input streams are grouped. In the process of creating these groups they will have already been repartitioned if the stream was already marked for repartitioning.

      -
      KStream<byte[], String> stream = ...;
      -                        KStream<byte[], String> stream2 = ...;
      +                            
      KStream<byte[], String> stream = ...;
      +                        KStream<byte[], String> stream2 = ...;
       
      -// Group by the existing key, using the application's configured
      -// default serdes for keys and values.
      -KGroupedStream<byte[], String> groupedStream = stream.groupByKey();
      -KGroupedStream<byte[], String> groupedStream2 = stream2.groupByKey();
      -CogroupedKStream<byte[], String> cogroupedStream = groupedStream.cogroup(aggregator1).cogroup(groupedStream2, aggregator2);
      +// Group by the existing key, using the application's configured
      +// default serdes for keys and values.
      +KGroupedStream<byte[], String> groupedStream = stream.groupByKey();
      +KGroupedStream<byte[], String> groupedStream2 = stream2.groupByKey();
      +CogroupedKStream<byte[], String> cogroupedStream = groupedStream.cogroup(aggregator1).cogroup(groupedStream2, aggregator2);
       
      -KTable<byte[], String> table = cogroupedStream.aggregate(initializer);
      +KTable<byte[], String> table = cogroupedStream.aggregate(initializer);
       
      -KTable<byte[], String> table2 = cogroupedStream.windowedBy(TimeWindows.duration(500ms)).aggregate(initializer);
      -
      +KTable<byte[], String> table2 = cogroupedStream.windowedBy(TimeWindows.duration(500ms)).aggregate(initializer);

      Map

      @@ -675,23 +663,22 @@

      map will result in re-partitioning of the records. If possible use mapValues instead, which will not cause data re-partitioning.

      -
      KStream<byte[], String> stream = ...;
      -
      -// Java 8+ example, using lambda expressions
      -// Note how we change the key and the key type (similar to `selectKey`)
      -// as well as the value and the value type.
      -KStream<String, Integer> transformed = stream.map(
      -    (key, value) -> KeyValue.pair(value.toLowerCase(), value.length()));
      -
      -// Java 7 example
      -KStream<String, Integer> transformed = stream.map(
      -    new KeyValueMapper<byte[], String, KeyValue<String, Integer>>() {
      -      @Override
      -      public KeyValue<String, Integer> apply(byte[] key, String value) {
      -        return new KeyValue<>(value.toLowerCase(), value.length());
      -      }
      -    });
      -
      +
      KStream<byte[], String> stream = ...;
      +
      +// Java 8+ example, using lambda expressions
      +// Note how we change the key and the key type (similar to `selectKey`)
      +// as well as the value and the value type.
      +KStream<String, Integer> transformed = stream.map(
      +    (key, value) -> KeyValue.pair(value.toLowerCase(), value.length()));
      +
      +// Java 7 example
      +KStream<String, Integer> transformed = stream.map(
      +    new KeyValueMapper<byte[], String, KeyValue<String, Integer>>() {
      +      @Override
      +      public KeyValue<String, Integer> apply(byte[] key, String value) {
      +        return new KeyValue<>(value.toLowerCase(), value.length());
      +      }
      +    });

      Map (values only)

      @@ -706,20 +693,19 @@

      KTable details)

      mapValues is preferable to map because it will not cause data re-partitioning. However, it does not allow you to modify the key or key type like map does.

      -
      KStream<byte[], String> stream = ...;
      -
      -// Java 8+ example, using lambda expressions
      -KStream<byte[], String> uppercased = stream.mapValues(value -> value.toUpperCase());
      -
      -// Java 7 example
      -KStream<byte[], String> uppercased = stream.mapValues(
      -    new ValueMapper<String>() {
      -      @Override
      -      public String apply(String s) {
      -        return s.toUpperCase();
      -      }
      -    });
      -
      +
      KStream<byte[], String> stream = ...;
      +
      +// Java 8+ example, using lambda expressions
      +KStream<byte[], String> uppercased = stream.mapValues(value -> value.toUpperCase());
      +
      +// Java 7 example
      +KStream<byte[], String> uppercased = stream.mapValues(
      +    new ValueMapper<String>() {
      +      @Override
      +      public String apply(String s) {
      +        return s.toUpperCase();
      +      }
      +    });

      Merge

      @@ -732,15 +718,11 @@

      details)

      There is no ordering guarantee between records from different streams in the merged stream. Relative order is preserved within each input stream though (ie, records within the same input stream are processed in order)

      -
      -
      -
      KStream<byte[], String> stream1 = ...;
      +                            
      KStream<byte[], String> stream1 = ...;
       
      -KStream<byte[], String> stream2 = ...;
      +KStream<byte[], String> stream2 = ...;
       
      -KStream<byte[], String> merged = stream1.merge(stream2);
      -
      -
      +KStream<byte[], String> merged = stream1.merge(stream2);

      Peek

      @@ -756,21 +738,20 @@

      peek is helpful for use cases such as logging or tracking metrics or for debugging and troubleshooting.

      Note on processing guarantees: Any side effects of an action (such as writing to external systems) are not trackable by Kafka, which means they will typically not benefit from Kafka’s processing guarantees.

      -
      KStream<byte[], String> stream = ...;
      -
      -// Java 8+ example, using lambda expressions
      -KStream<byte[], String> unmodifiedStream = stream.peek(
      -    (key, value) -> System.out.println("key=" + key + ", value=" + value));
      -
      -// Java 7 example
      -KStream<byte[], String> unmodifiedStream = stream.peek(
      -    new ForeachAction<byte[], String>() {
      -      @Override
      -      public void apply(byte[] key, String value) {
      -        System.out.println("key=" + key + ", value=" + value);
      -      }
      -    });
      -
      +
      KStream<byte[], String> stream = ...;
      +
      +// Java 8+ example, using lambda expressions
      +KStream<byte[], String> unmodifiedStream = stream.peek(
      +    (key, value) -> System.out.println("key=" + key + ", value=" + value));
      +
      +// Java 7 example
      +KStream<byte[], String> unmodifiedStream = stream.peek(
      +    new ForeachAction<byte[], String>() {
      +      @Override
      +      public void apply(byte[] key, String value) {
      +        System.out.println("key=" + key + ", value=" + value);
      +      }
      +    });

      Print

      @@ -783,13 +764,12 @@

      details)

      Calling print() is the same as calling foreach((key, value) -> System.out.println(key + ", " + value))

      print is mainly for debugging/testing purposes, and it will try to flush on each record print. Hence it should not be used for production usage if performance requirements are concerned.

      -
      KStream<byte[], String> stream = ...;
      -// print to sysout
      -stream.print();
      +                            
      KStream<byte[], String> stream = ...;
      +// print to sysout
      +stream.print();
       
      -// print to file with a custom label
      -stream.print(Printed.toFile("streams.out").withLabel("streams"));
      -
      +// print to file with a custom label +stream.print(Printed.toFile("streams.out").withLabel("streams"));

      SelectKey

      @@ -802,21 +782,20 @@

      selectKey(mapper) is the same as calling map((key, value) -> mapper(key, value), value).

      Marks the stream for data re-partitioning: Applying a grouping or a join after selectKey will result in re-partitioning of the records.

      -
      KStream<byte[], String> stream = ...;
      -
      -// Derive a new record key from the record's value.  Note how the key type changes, too.
      -// Java 8+ example, using lambda expressions
      -KStream<String, String> rekeyed = stream.selectKey((key, value) -> value.split(" ")[0])
      -
      -// Java 7 example
      -KStream<String, String> rekeyed = stream.selectKey(
      -    new KeyValueMapper<byte[], String, String>() {
      -      @Override
      -      public String apply(byte[] key, String value) {
      -        return value.split(" ")[0];
      -      }
      -    });
      -
      +
      KStream<byte[], String> stream = ...;
      +
      +// Derive a new record key from the record's value.  Note how the key type changes, too.
      +// Java 8+ example, using lambda expressions
      +KStream<String, String> rekeyed = stream.selectKey((key, value) -> value.split(" ")[0])
      +
      +// Java 7 example
      +KStream<String, String> rekeyed = stream.selectKey(
      +    new KeyValueMapper<byte[], String, String>() {
      +      @Override
      +      public String apply(byte[] key, String value) {
      +        return value.split(" ")[0];
      +      }
      +    });

      Table to Stream

      @@ -826,12 +805,11 @@

      Get the changelog stream of this table. (details)

      -
      KTable<byte[], String> table = ...;
      +                            
      KTable<byte[], String> table = ...;
       
      -// Also, a variant of `toStream` exists that allows you
      -// to select a new key for the resulting stream.
      -KStream<byte[], String> stream = table.toStream();
      -
      +// Also, a variant of `toStream` exists that allows you +// to select a new key for the resulting stream. +KStream<byte[], String> stream = table.toStream();

      Stream to Table

      @@ -841,10 +819,9 @@

      Convert an event stream into a table, or say a changelog stream. (details)

      -
      KStream<byte[], String> stream = ...;
      +                            
      KStream<byte[], String> stream = ...;
       
      -KTable<byte[], String> table = stream.toTable();
      -
      +KTable<byte[], String> table = stream.toTable();

      @@ -1827,35 +1792,34 @@

      co-partitioned.

      Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).

      Several variants of join exists, see the Javadocs for details.

      -
      import java.time.Duration;
      -KStream<String, Long> left = ...;
      -KStream<String, Double> right = ...;
      -
      -// Java 8+ example, using lambda expressions
      -KStream<String, String> joined = left.join(right,
      -    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
      -    JoinWindows.of(Duration.ofMinutes(5)),
      -    Joined.with(
      -      Serdes.String(), /* key */
      -      Serdes.Long(),   /* left value */
      -      Serdes.Double())  /* right value */
      -  );
      -
      -// Java 7 example
      -KStream<String, String> joined = left.join(right,
      -    new ValueJoiner<Long, Double, String>() {
      -      @Override
      -      public String apply(Long leftValue, Double rightValue) {
      -        return "left=" + leftValue + ", right=" + rightValue;
      -      }
      -    },
      -    JoinWindows.of(Duration.ofMinutes(5)),
      -    Joined.with(
      -      Serdes.String(), /* key */
      -      Serdes.Long(),   /* left value */
      -      Serdes.Double())  /* right value */
      -  );
      -
      +
      import java.time.Duration;
      +KStream<String, Long> left = ...;
      +KStream<String, Double> right = ...;
      +
      +// Java 8+ example, using lambda expressions
      +KStream<String, String> joined = left.join(right,
      +    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
      +    JoinWindows.of(Duration.ofMinutes(5)),
      +    Joined.with(
      +      Serdes.String(), /* key */
      +      Serdes.Long(),   /* left value */
      +      Serdes.Double())  /* right value */
      +  );
      +
      +// Java 7 example
      +KStream<String, String> joined = left.join(right,
      +    new ValueJoiner<Long, Double, String>() {
      +      @Override
      +      public String apply(Long leftValue, Double rightValue) {
      +        return "left=" + leftValue + ", right=" + rightValue;
      +      }
      +    },
      +    JoinWindows.of(Duration.ofMinutes(5)),
      +    Joined.with(
      +      Serdes.String(), /* key */
      +      Serdes.Long(),   /* left value */
      +      Serdes.Double())  /* right value */
      +  );

      Detailed behavior:

      • The join is key-based, i.e. with the join predicate leftRecord.key == rightRecord.key, and window-based, i.e. two input records are joined if and only if their @@ -1885,35 +1849,34 @@

        co-partitioned.

        Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).

        Several variants of leftJoin exists, see the Javadocs for details.

        -
        import java.time.Duration;
        -KStream<String, Long> left = ...;
        -KStream<String, Double> right = ...;
        -
        -// Java 8+ example, using lambda expressions
        -KStream<String, String> joined = left.leftJoin(right,
        -    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
        -    JoinWindows.of(Duration.ofMinutes(5)),
        -    Joined.with(
        -      Serdes.String(), /* key */
        -      Serdes.Long(),   /* left value */
        -      Serdes.Double())  /* right value */
        -  );
        -
        -// Java 7 example
        -KStream<String, String> joined = left.leftJoin(right,
        -    new ValueJoiner<Long, Double, String>() {
        -      @Override
        -      public String apply(Long leftValue, Double rightValue) {
        -        return "left=" + leftValue + ", right=" + rightValue;
        -      }
        -    },
        -    JoinWindows.of(Duration.ofMinutes(5)),
        -    Joined.with(
        -      Serdes.String(), /* key */
        -      Serdes.Long(),   /* left value */
        -      Serdes.Double())  /* right value */
        -  );
        -
        +
        import java.time.Duration;
        +KStream<String, Long> left = ...;
        +KStream<String, Double> right = ...;
        +
        +// Java 8+ example, using lambda expressions
        +KStream<String, String> joined = left.leftJoin(right,
        +    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
        +    JoinWindows.of(Duration.ofMinutes(5)),
        +    Joined.with(
        +      Serdes.String(), /* key */
        +      Serdes.Long(),   /* left value */
        +      Serdes.Double())  /* right value */
        +  );
        +
        +// Java 7 example
        +KStream<String, String> joined = left.leftJoin(right,
        +    new ValueJoiner<Long, Double, String>() {
        +      @Override
        +      public String apply(Long leftValue, Double rightValue) {
        +        return "left=" + leftValue + ", right=" + rightValue;
        +      }
        +    },
        +    JoinWindows.of(Duration.ofMinutes(5)),
        +    Joined.with(
        +      Serdes.String(), /* key */
        +      Serdes.Long(),   /* left value */
        +      Serdes.Double())  /* right value */
        +  );

        Detailed behavior:

        • The join is key-based, i.e. with the join predicate leftRecord.key == rightRecord.key, and window-based, i.e. two input records are joined if and only if their @@ -1946,35 +1909,34 @@

          co-partitioned.

          Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).

          Several variants of outerJoin exists, see the Javadocs for details.

          -
          import java.time.Duration;
          -KStream<String, Long> left = ...;
          -KStream<String, Double> right = ...;
          -
          -// Java 8+ example, using lambda expressions
          -KStream<String, String> joined = left.outerJoin(right,
          -    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
          -    JoinWindows.of(Duration.ofMinutes(5)),
          -    Joined.with(
          -      Serdes.String(), /* key */
          -      Serdes.Long(),   /* left value */
          -      Serdes.Double())  /* right value */
          -  );
          -
          -// Java 7 example
          -KStream<String, String> joined = left.outerJoin(right,
          -    new ValueJoiner<Long, Double, String>() {
          -      @Override
          -      public String apply(Long leftValue, Double rightValue) {
          -        return "left=" + leftValue + ", right=" + rightValue;
          -      }
          -    },
          -    JoinWindows.of(Duration.ofMinutes(5)),
          -    Joined.with(
          -      Serdes.String(), /* key */
          -      Serdes.Long(),   /* left value */
          -      Serdes.Double())  /* right value */
          -  );
          -
          +
          import java.time.Duration;
          +KStream<String, Long> left = ...;
          +KStream<String, Double> right = ...;
          +
          +// Java 8+ example, using lambda expressions
          +KStream<String, String> joined = left.outerJoin(right,
          +    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
          +    JoinWindows.of(Duration.ofMinutes(5)),
          +    Joined.with(
          +      Serdes.String(), /* key */
          +      Serdes.Long(),   /* left value */
          +      Serdes.Double())  /* right value */
          +  );
          +
          +// Java 7 example
          +KStream<String, String> joined = left.outerJoin(right,
          +    new ValueJoiner<Long, Double, String>() {
          +      @Override
          +      public String apply(Long leftValue, Double rightValue) {
          +        return "left=" + leftValue + ", right=" + rightValue;
          +      }
          +    },
          +    JoinWindows.of(Duration.ofMinutes(5)),
          +    Joined.with(
          +      Serdes.String(), /* key */
          +      Serdes.Long(),   /* left value */
          +      Serdes.Double())  /* right value */
          +  );

          Detailed behavior:

          • The join is key-based, i.e. with the join predicate leftRecord.key == rightRecord.key, and window-based, i.e. two input records are joined if and only if their @@ -2131,15 +2093,14 @@

            table duals. The join result is a new KTable that represents the changelog stream of the join operation.

            Join output records are effectively created as follows, leveraging the user-supplied ValueJoiner:

            -
            KeyValue<K, LV> leftRecord = ...;
            -KeyValue<K, RV> rightRecord = ...;
            -ValueJoiner<LV, RV, JV> joiner = ...;
            -
            -KeyValue<K, JV> joinOutputRecord = KeyValue.pair(
            -    leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */
            -    joiner.apply(leftRecord.value, rightRecord.value)
            -  );
            -
            +
            KeyValue<K, LV> leftRecord = ...;
            +KeyValue<K, RV> rightRecord = ...;
            +ValueJoiner<LV, RV, JV> joiner = ...;
            +
            +KeyValue<K, JV> joinOutputRecord = KeyValue.pair(
            +    leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */
            +    joiner.apply(leftRecord.value, rightRecord.value)
            +  );

      @@ -2161,23 +2122,22 @@

      (details)

      Data must be co-partitioned: The input data for both sides must be co-partitioned.

      -
      KTable<String, Long> left = ...;
      -KTable<String, Double> right = ...;
      -
      -// Java 8+ example, using lambda expressions
      -KTable<String, String> joined = left.join(right,
      -    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
      -  );
      -
      -// Java 7 example
      -KTable<String, String> joined = left.join(right,
      -    new ValueJoiner<Long, Double, String>() {
      -      @Override
      -      public String apply(Long leftValue, Double rightValue) {
      -        return "left=" + leftValue + ", right=" + rightValue;
      -      }
      -    });
      -
      +
      KTable<String, Long> left = ...;
      +KTable<String, Double> right = ...;
      +
      +// Java 8+ example, using lambda expressions
      +KTable<String, String> joined = left.join(right,
      +    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
      +  );
      +
      +// Java 7 example
      +KTable<String, String> joined = left.join(right,
      +    new ValueJoiner<Long, Double, String>() {
      +      @Override
      +      public String apply(Long leftValue, Double rightValue) {
      +        return "left=" + leftValue + ", right=" + rightValue;
      +      }
      +    });

      Detailed behavior:

      @@ -2770,28 +2727,27 @@
      KTable-KTable Foreign-Key

      Data must be co-partitioned: The input data for both sides must be co-partitioned.

      Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.

      Several variants of join exists, see the Javadocs for details.

      -
      KStream<String, Long> left = ...;
      -KTable<String, Double> right = ...;
      -
      -// Java 8+ example, using lambda expressions
      -KStream<String, String> joined = left.join(right,
      -    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
      -    Joined.keySerde(Serdes.String()) /* key */
      -      .withValueSerde(Serdes.Long()) /* left value */
      -  );
      -
      -// Java 7 example
      -KStream<String, String> joined = left.join(right,
      -    new ValueJoiner<Long, Double, String>() {
      -      @Override
      -      public String apply(Long leftValue, Double rightValue) {
      -        return "left=" + leftValue + ", right=" + rightValue;
      -      }
      -    },
      -    Joined.keySerde(Serdes.String()) /* key */
      -      .withValueSerde(Serdes.Long()) /* left value */
      -  );
      -
      +
      KStream<String, Long> left = ...;
      +KTable<String, Double> right = ...;
      +
      +// Java 8+ example, using lambda expressions
      +KStream<String, String> joined = left.join(right,
      +    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
      +    Joined.keySerde(Serdes.String()) /* key */
      +      .withValueSerde(Serdes.Long()) /* left value */
      +  );
      +
      +// Java 7 example
      +KStream<String, String> joined = left.join(right,
      +    new ValueJoiner<Long, Double, String>() {
      +      @Override
      +      public String apply(Long leftValue, Double rightValue) {
      +        return "left=" + leftValue + ", right=" + rightValue;
      +      }
      +    },
      +    Joined.keySerde(Serdes.String()) /* key */
      +      .withValueSerde(Serdes.Long()) /* left value */
      +  );

      Detailed behavior:

      @@ -3038,30 +2992,29 @@
      KTable-KTable Foreign-Key

      The GlobalKTable is fully bootstrapped upon (re)start of a KafkaStreams instance, which means the table is fully populated with all the data in the underlying topic that is available at the time of the startup. The actual data processing begins only once the bootstrapping has completed.

      Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.

      -
      KStream<String, Long> left = ...;
      -GlobalKTable<Integer, Double> right = ...;
      -
      -// Java 8+ example, using lambda expressions
      -KStream<String, String> joined = left.join(right,
      -    (leftKey, leftValue) -> leftKey.length(), /* derive a (potentially) new key by which to lookup against the table */
      -    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
      -  );
      -
      -// Java 7 example
      -KStream<String, String> joined = left.join(right,
      -    new KeyValueMapper<String, Long, Integer>() { /* derive a (potentially) new key by which to lookup against the table */
      -      @Override
      -      public Integer apply(String key, Long value) {
      -        return key.length();
      -      }
      -    },
      -    new ValueJoiner<Long, Double, String>() {
      -      @Override
      -      public String apply(Long leftValue, Double rightValue) {
      -        return "left=" + leftValue + ", right=" + rightValue;
      -      }
      -    });
      -
      +
      KStream<String, Long> left = ...;
      +GlobalKTable<Integer, Double> right = ...;
      +
      +// Java 8+ example, using lambda expressions
      +KStream<String, String> joined = left.join(right,
      +    (leftKey, leftValue) -> leftKey.length(), /* derive a (potentially) new key by which to lookup against the table */
      +    (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
      +  );
      +
      +// Java 7 example
      +KStream<String, String> joined = left.join(right,
      +    new KeyValueMapper<String, Long, Integer>() { /* derive a (potentially) new key by which to lookup against the table */
      +      @Override
      +      public Integer apply(String key, Long value) {
      +        return key.length();
      +      }
      +    },
      +    new ValueJoiner<Long, Double, String>() {
      +      @Override
      +      public String apply(Long leftValue, Double rightValue) {
      +        return "left=" + leftValue + ", right=" + rightValue;
      +      }
      +    });

      Detailed behavior:

      The class/interface hierarchy for your custom store might look something like:

      -
      public class MyCustomStore<K,V> implements StateStore, MyWriteableCustomStore<K,V> {
      -  // implementation of the actual store
      -}
      -
      -// Read-write interface for MyCustomStore
      -public interface MyWriteableCustomStore<K,V> extends MyReadableCustomStore<K,V> {
      -  void write(K Key, V value);
      -}
      -
      -// Read-only interface for MyCustomStore
      -public interface MyReadableCustomStore<K,V> {
      -  V read(K key);
      -}
      -
      -public class MyCustomStoreBuilder implements StoreBuilder {
      -  // implementation of the supplier for MyCustomStore
      -}
      -
      +
      public class MyCustomStore<K,V> implements StateStore, MyWriteableCustomStore<K,V> {
      +  // implementation of the actual store
      +}
      +
      +// Read-write interface for MyCustomStore
      +public interface MyWriteableCustomStore<K,V> extends MyReadableCustomStore<K,V> {
      +  void write(K Key, V value);
      +}
      +
      +// Read-only interface for MyCustomStore
      +public interface MyReadableCustomStore<K,V> {
      +  V read(K key);
      +}
      +
      +public class MyCustomStoreBuilder implements StoreBuilder {
      +  // implementation of the supplier for MyCustomStore
      +}

      To make this store queryable you must:

      • Provide an implementation of QueryableStoreType.
      • Provide a wrapper class that has access to all of the underlying instances of the store and is used for querying.

      Here is how to implement QueryableStoreType:

      -
      public class MyCustomStoreType<K,V> implements QueryableStoreType<MyReadableCustomStore<K,V>> {
      +                
      public class MyCustomStoreType<K,V> implements QueryableStoreType<MyReadableCustomStore<K,V>> {
       
      -  // Only accept StateStores that are of type MyCustomStore
      -  public boolean accepts(final StateStore stateStore) {
      -    return stateStore instanceOf MyCustomStore;
      -  }
      +  // Only accept StateStores that are of type MyCustomStore
      +  public boolean accepts(final StateStore stateStore) {
      +    return stateStore instanceOf MyCustomStore;
      +  }
       
      -  public MyReadableCustomStore<K,V> create(final StateStoreProvider storeProvider, final String storeName) {
      -      return new MyCustomStoreTypeWrapper(storeProvider, storeName, this);
      -  }
      +  public MyReadableCustomStore<K,V> create(final StateStoreProvider storeProvider, final String storeName) {
      +      return new MyCustomStoreTypeWrapper(storeProvider, storeName, this);
      +  }
       
      -}
      -
      +}

      A wrapper class is required because each instance of a Kafka Streams application may run multiple stream tasks and manage multiple local instances of a particular state store. The wrapper class hides this complexity and lets you query a “logical” state store by name without having to know about all of the underlying local instances of that state store.

      @@ -279,56 +269,53 @@ StateStoreProvider#stores(String storeName, QueryableStoreType<T> queryableStoreType) returns a List of state stores with the given storeName and of the type as defined by queryableStoreType.

      Here is an example implementation of the wrapper follows (Java 8+):

      -
      // We strongly recommended implementing a read-only interface
      -// to restrict usage of the store to safe read operations!
      -public class MyCustomStoreTypeWrapper<K,V> implements MyReadableCustomStore<K,V> {
      -
      -  private final QueryableStoreType<MyReadableCustomStore<K, V>> customStoreType;
      -  private final String storeName;
      -  private final StateStoreProvider provider;
      -
      -  public CustomStoreTypeWrapper(final StateStoreProvider provider,
      -                              final String storeName,
      -                              final QueryableStoreType<MyReadableCustomStore<K, V>> customStoreType) {
      -
      -    // ... assign fields ...
      -  }
      -
      -  // Implement a safe read method
      -  @Override
      -  public V read(final K key) {
      -    // Get all the stores with storeName and of customStoreType
      -    final List<MyReadableCustomStore<K, V>> stores = provider.getStores(storeName, customStoreType);
      -    // Try and find the value for the given key
      -    final Optional<V> value = stores.stream().filter(store -> store.read(key) != null).findFirst();
      -    // Return the value if it exists
      -    return value.orElse(null);
      -  }
      -
      -}
      -
      +
      // We strongly recommended implementing a read-only interface
      +// to restrict usage of the store to safe read operations!
      +public class MyCustomStoreTypeWrapper<K,V> implements MyReadableCustomStore<K,V> {
      +
      +  private final QueryableStoreType<MyReadableCustomStore<K, V>> customStoreType;
      +  private final String storeName;
      +  private final StateStoreProvider provider;
      +
      +  public CustomStoreTypeWrapper(final StateStoreProvider provider,
      +                              final String storeName,
      +                              final QueryableStoreType<MyReadableCustomStore<K, V>> customStoreType) {
      +
      +    // ... assign fields ...
      +  }
      +
      +  // Implement a safe read method
      +  @Override
      +  public V read(final K key) {
      +    // Get all the stores with storeName and of customStoreType
      +    final List<MyReadableCustomStore<K, V>> stores = provider.getStores(storeName, customStoreType);
      +    // Try and find the value for the given key
      +    final Optional<V> value = stores.stream().filter(store -> store.read(key) != null).findFirst();
      +    // Return the value if it exists
      +    return value.orElse(null);
      +  }
      +
      +}

      You can now find and query your custom store:

      -
      
      -Topology topology = ...;
      -ProcessorSupplier processorSuppler = ...;
      -
      -// Create CustomStoreSupplier for store name the-custom-store
      -MyCustomStoreBuilder customStoreBuilder = new MyCustomStoreBuilder("the-custom-store") //...;
      -// Add the source topic
      -topology.addSource("input", "inputTopic");
      -// Add a custom processor that reads from the source topic
      -topology.addProcessor("the-processor", processorSupplier, "input");
      -// Connect your custom state store to the custom processor above
      -topology.addStateStore(customStoreBuilder, "the-processor");
      -
      -KafkaStreams streams = new KafkaStreams(topology, config);
      -streams.start();
      -
      -// Get access to the custom store
      -MyReadableCustomStore<String,String> store = streams.store("the-custom-store", new MyCustomStoreType<String,String>());
      -// Query the store
      -String value = store.read("key");
      -
      +
      Topology topology = ...;
      +ProcessorSupplier processorSuppler = ...;
      +
      +// Create CustomStoreSupplier for store name the-custom-store
      +MyCustomStoreBuilder customStoreBuilder = new MyCustomStoreBuilder("the-custom-store") //...;
      +// Add the source topic
      +topology.addSource("input", "inputTopic");
      +// Add a custom processor that reads from the source topic
      +topology.addProcessor("the-processor", processorSupplier, "input");
      +// Connect your custom state store to the custom processor above
      +topology.addStateStore(customStoreBuilder, "the-processor");
      +
      +KafkaStreams streams = new KafkaStreams(topology, config);
      +streams.start();
      +
      +// Get access to the custom store
      +MyReadableCustomStore<String,String> store = streams.store("the-custom-store", new MyCustomStoreType<String,String>());
      +// Query the store
      +String value = store.read("key");
      @@ -369,41 +356,39 @@ piggybacking additional inter-application communication that goes beyond interactive queries.

      This example shows how to configure and run a Kafka Streams application that supports the discovery of its state stores.

      -
      Properties props = new Properties();
      -// Set the unique RPC endpoint of this application instance through which it
      -// can be interactively queried.  In a real application, the value would most
      -// probably not be hardcoded but derived dynamically.
      -String rpcEndpoint = "host1:4460";
      -props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, rpcEndpoint);
      -// ... further settings may follow here ...
      -
      -StreamsBuilder builder = new StreamsBuilder();
      -
      -KStream<String, String> textLines = builder.stream(stringSerde, stringSerde, "word-count-input");
      -
      -final KGroupedStream<String, String> groupedByWord = textLines
      -    .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
      -    .groupBy((key, word) -> word, Grouped.with(stringSerde, stringSerde));
      -
      -// This call to `count()` creates a state store named "word-count".
      -// The state store is discoverable and can be queried interactively.
      -groupedByWord.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>as("word-count"));
      -
      -// Start an instance of the topology
      -KafkaStreams streams = new KafkaStreams(builder, props);
      -streams.start();
      -
      -// Then, create and start the actual RPC service for remote access to this
      -// application instance's local state stores.
      -//
      -// This service should be started on the same host and port as defined above by
      -// the property `StreamsConfig.APPLICATION_SERVER_CONFIG`.  The example below is
      -// fictitious, but we provide end-to-end demo applications (such as KafkaMusicExample)
      -// that showcase how to implement such a service to get you started.
      -MyRPCService rpcService = ...;
      -rpcService.listenAt(rpcEndpoint);
      -
      - +
      Properties props = new Properties();
      +// Set the unique RPC endpoint of this application instance through which it
      +// can be interactively queried.  In a real application, the value would most
      +// probably not be hardcoded but derived dynamically.
      +String rpcEndpoint = "host1:4460";
      +props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, rpcEndpoint);
      +// ... further settings may follow here ...
      +
      +StreamsBuilder builder = new StreamsBuilder();
      +
      +KStream<String, String> textLines = builder.stream(stringSerde, stringSerde, "word-count-input");
      +
      +final KGroupedStream<String, String> groupedByWord = textLines
      +    .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
      +    .groupBy((key, word) -> word, Grouped.with(stringSerde, stringSerde));
      +
      +// This call to `count()` creates a state store named "word-count".
      +// The state store is discoverable and can be queried interactively.
      +groupedByWord.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>as("word-count"));
      +
      +// Start an instance of the topology
      +KafkaStreams streams = new KafkaStreams(builder, props);
      +streams.start();
      +
      +// Then, create and start the actual RPC service for remote access to this
      +// application instance's local state stores.
      +//
      +// This service should be started on the same host and port as defined above by
      +// the property `StreamsConfig.APPLICATION_SERVER_CONFIG`.  The example below is
      +// fictitious, but we provide end-to-end demo applications (such as KafkaMusicExample)
      +// that showcase how to implement such a service to get you started.
      +MyRPCService rpcService = ...;
      +rpcService.listenAt(rpcEndpoint);

      Discovering and accessing application instances and their local state stores

      The following methods return StreamsMetadata objects, which provide meta-information about application instances such as their RPC endpoint and locally available state stores.

      @@ -419,39 +404,38 @@

      For example, we can now find the StreamsMetadata for the state store named “word-count” that we defined in the code example shown in the previous section:

      -
      KafkaStreams streams = ...;
      -// Find all the locations of local instances of the state store named "word-count"
      -Collection<StreamsMetadata> wordCountHosts = streams.allMetadataForStore("word-count");
      -
      -// For illustrative purposes, we assume using an HTTP client to talk to remote app instances.
      -HttpClient http = ...;
      -
      -// Get the word count for word (aka key) 'alice': Approach 1
      -//
      -// We first find the one app instance that manages the count for 'alice' in its local state stores.
      -StreamsMetadata metadata = streams.metadataForKey("word-count", "alice", Serdes.String().serializer());
      -// Then, we query only that single app instance for the latest count of 'alice'.
      -// Note: The RPC URL shown below is fictitious and only serves to illustrate the idea.  Ultimately,
      -// the URL (or, in general, the method of communication) will depend on the RPC layer you opted to
      -// implement.  Again, we provide end-to-end demo applications (such as KafkaMusicExample) that showcase
      -// how to implement such an RPC layer.
      -Long result = http.getLong("http://" + metadata.host() + ":" + metadata.port() + "/word-count/alice");
      -
      -// Get the word count for word (aka key) 'alice': Approach 2
      -//
      -// Alternatively, we could also choose (say) a brute-force approach where we query every app instance
      -// until we find the one that happens to know about 'alice'.
      -Optional<Long> result = streams.allMetadataForStore("word-count")
      -    .stream()
      -    .map(streamsMetadata -> {
      -        // Construct the (fictituous) full endpoint URL to query the current remote application instance
      -        String url = "http://" + streamsMetadata.host() + ":" + streamsMetadata.port() + "/word-count/alice";
      -        // Read and return the count for 'alice', if any.
      -        return http.getLong(url);
      -    })
      -    .filter(s -> s != null)
      -    .findFirst();
      -
      +
      KafkaStreams streams = ...;
      +// Find all the locations of local instances of the state store named "word-count"
      +Collection<StreamsMetadata> wordCountHosts = streams.allMetadataForStore("word-count");
      +
      +// For illustrative purposes, we assume using an HTTP client to talk to remote app instances.
      +HttpClient http = ...;
      +
      +// Get the word count for word (aka key) 'alice': Approach 1
      +//
      +// We first find the one app instance that manages the count for 'alice' in its local state stores.
      +StreamsMetadata metadata = streams.metadataForKey("word-count", "alice", Serdes.String().serializer());
      +// Then, we query only that single app instance for the latest count of 'alice'.
      +// Note: The RPC URL shown below is fictitious and only serves to illustrate the idea.  Ultimately,
      +// the URL (or, in general, the method of communication) will depend on the RPC layer you opted to
      +// implement.  Again, we provide end-to-end demo applications (such as KafkaMusicExample) that showcase
      +// how to implement such an RPC layer.
      +Long result = http.getLong("http://" + metadata.host() + ":" + metadata.port() + "/word-count/alice");
      +
      +// Get the word count for word (aka key) 'alice': Approach 2
      +//
      +// Alternatively, we could also choose (say) a brute-force approach where we query every app instance
      +// until we find the one that happens to know about 'alice'.
      +Optional<Long> result = streams.allMetadataForStore("word-count")
      +    .stream()
      +    .map(streamsMetadata -> {
      +        // Construct the (fictituous) full endpoint URL to query the current remote application instance
      +        String url = "http://" + streamsMetadata.host() + ":" + streamsMetadata.port() + "/word-count/alice";
      +        // Read and return the count for 'alice', if any.
      +        return http.getLong(url);
      +    })
      +    .filter(s -> s != null)
      +    .findFirst();

      At this point the full state of the application is interactively queryable:

      • You can discover the running instances of the application and the state stores they manage locally.
      • diff --git a/docs/streams/developer-guide/memory-mgmt.html b/docs/streams/developer-guide/memory-mgmt.html index 9aa382e089a20..9a39ce1e78a5c 100644 --- a/docs/streams/developer-guide/memory-mgmt.html +++ b/docs/streams/developer-guide/memory-mgmt.html @@ -80,10 +80,9 @@

      The cache size is specified through the cache.max.bytes.buffering parameter, which is a global setting per processing topology:

      -
      // Enable record cache of size 10 MB.
      -Properties props = new Properties();
      -props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);
      -
      +
      // Enable record cache of size 10 MB.
      +Properties props = new Properties();
      +props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);

      This parameter controls the number of bytes allocated for caching. Specifically, for a processor topology instance with T threads and C bytes allocated for caching, each thread will have an even C/T bytes to construct its own cache and use as it sees fit among its tasks. This means that there are as many caches as there are threads, but no sharing of @@ -103,27 +102,16 @@

      Here are example settings for both parameters based on desired scenarios.

      • To turn off caching the cache size can be set to zero:

        -
        -
        // Disable record cache
        -Properties props = new Properties();
        -props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
        -
        -

        Turning off caching might result in high write traffic for the underlying RocksDB store. - With default settings caching is enabled within Kafka Streams but RocksDB caching is disabled. - Thus, to avoid high write traffic it is recommended to enable RocksDB caching if Kafka Streams caching is turned off.

        -

        For example, the RocksDB Block Cache could be set to 100MB and Write Buffer size to 32 MB. For more information, see - the RocksDB config.

        -
        +
        // Disable record cache
        +Properties props = new Properties();
        +props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
      • To enable caching but still have an upper bound on how long records will be cached, you can set the commit interval. In this example, it is set to 1000 milliseconds:

        -
        -
        Properties props = new Properties();
        -// Enable record cache of size 10 MB.
        -props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);
        -// Set commit interval to 1 second.
        -props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
        -
        -
        +
        Properties props = new Properties();
        +// Enable record cache of size 10 MB.
        +props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);
        +// Set commit interval to 1 second.
        +props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);

      The effect of these two configurations is described in the figure below. The records are shown using 4 keys: blue, red, yellow, and green. Assume the cache has space for only 3 keys.

      @@ -156,13 +144,12 @@

      Following from the example first shown in section State Stores, to disable caching, you can add the withCachingDisabled call (note that caches are enabled by default, however there is an explicit withCachingEnabled call).

      -
      StoreBuilder countStoreBuilder =
      -  Stores.keyValueStoreBuilder(
      -    Stores.persistentKeyValueStore("Counts"),
      -    Serdes.String(),
      -    Serdes.Long())
      -  .withCachingEnabled()
      -
      +
      StoreBuilder countStoreBuilder =
      +  Stores.keyValueStoreBuilder(
      +    Stores.persistentKeyValueStore("Counts"),
      +    Serdes.String(),
      +    Serdes.Long())
      +  .withCachingEnabled();

      RocksDB

      @@ -171,44 +158,42 @@

      RocksDBrocksdb.config.setter configuration.

      Also, we recommend changing RocksDB's default memory allocator, because the default allocator may lead to increased memory consumption. To change the memory allocator to jemalloc, you need to set the environment variable LD_PRELOADbefore you start your Kafka Streams application:

      -
      -# example: install jemalloc (on Debian)
      +      
      # example: install jemalloc (on Debian)
       $ apt install -y libjemalloc-dev
       # set LD_PRELOAD before you start your Kafka Streams application
       $ export LD_PRELOAD="/usr/lib/x86_64-linux-gnu/libjemalloc.so”
      -      
      +

      As of 2.3.0 the memory usage across all instances can be bounded, limiting the total off-heap memory of your Kafka Streams application. To do so you must configure RocksDB to cache the index and filter blocks in the block cache, limit the memtable memory through a shared WriteBufferManager and count its memory against the block cache, and then pass the same Cache object to each instance. See RocksDB Memory Usage for details. An example RocksDBConfigSetter implementing this is shown below:

      +
      public static class BoundedMemoryRocksDBConfig implements RocksDBConfigSetter {
       
      -      
          public static class BoundedMemoryRocksDBConfig implements RocksDBConfigSetter {
      -
      -       private static org.rocksdb.Cache cache = new org.rocksdb.LRUCache(TOTAL_OFF_HEAP_MEMORY, -1, false, INDEX_FILTER_BLOCK_RATIO);1
      -       private static org.rocksdb.WriteBufferManager writeBufferManager = new org.rocksdb.WriteBufferManager(TOTAL_MEMTABLE_MEMORY, cache);
      +   private static org.rocksdb.Cache cache = new org.rocksdb.LRUCache(TOTAL_OFF_HEAP_MEMORY, -1, false, INDEX_FILTER_BLOCK_RATIO);1
      +   private static org.rocksdb.WriteBufferManager writeBufferManager = new org.rocksdb.WriteBufferManager(TOTAL_MEMTABLE_MEMORY, cache);
       
      -       @Override
      -       public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) {
      +   @Override
      +   public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) {
       
      -         BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();
      +     BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();
       
      -          // These three options in combination will limit the memory used by RocksDB to the size passed to the block cache (TOTAL_OFF_HEAP_MEMORY)
      -         tableConfig.setBlockCache(cache);
      -         tableConfig.setCacheIndexAndFilterBlocks(true);
      -         options.setWriteBufferManager(writeBufferManager);
      +      // These three options in combination will limit the memory used by RocksDB to the size passed to the block cache (TOTAL_OFF_HEAP_MEMORY)
      +     tableConfig.setBlockCache(cache);
      +     tableConfig.setCacheIndexAndFilterBlocks(true);
      +     options.setWriteBufferManager(writeBufferManager);
       
      -          // These options are recommended to be set when bounding the total memory
      -         tableConfig.setCacheIndexAndFilterBlocksWithHighPriority(true);2
      -         tableConfig.setPinTopLevelIndexAndFilter(true);
      -         tableConfig.setBlockSize(BLOCK_SIZE);3
      -         options.setMaxWriteBufferNumber(N_MEMTABLES);
      -         options.setWriteBufferSize(MEMTABLE_SIZE);
      +      // These options are recommended to be set when bounding the total memory
      +     tableConfig.setCacheIndexAndFilterBlocksWithHighPriority(true);2
      +     tableConfig.setPinTopLevelIndexAndFilter(true);
      +     tableConfig.setBlockSize(BLOCK_SIZE);3
      +     options.setMaxWriteBufferNumber(N_MEMTABLES);
      +     options.setWriteBufferSize(MEMTABLE_SIZE);
       
      -         options.setTableFormatConfig(tableConfig);
      -       }
      +     options.setTableFormatConfig(tableConfig);
      +   }
       
      -       @Override
      -       public void close(final String storeName, final Options options) {
      -         // Cache and WriteBufferManager should not be closed here, as the same objects are shared by every store instance.
      -       }
      -    }
      +   @Override
      +   public void close(final String storeName, final Options options) {
      +     // Cache and WriteBufferManager should not be closed here, as the same objects are shared by every store instance.
      +   }
      +}
      1. INDEX_FILTER_BLOCK_RATIO can be used to set a fraction of the block cache to set aside for "high priority" (aka index and filter) blocks, preventing them from being evicted by data blocks. See the full signature of the LRUCache constructor. NOTE: the boolean parameter in the cache constructor lets you control whether the cache should enforce a strict memory limit by failing the read or iteration in the rare cases where it might go larger than its capacity. Due to a diff --git a/docs/streams/developer-guide/processor-api.html b/docs/streams/developer-guide/processor-api.html index 9cabac030b90c..589a3ff0aa40e 100644 --- a/docs/streams/developer-guide/processor-api.html +++ b/docs/streams/developer-guide/processor-api.html @@ -119,47 +119,46 @@

      Overviewprocess() method, upon each received record, split the value string into words, and update their counts into the state store (we will talk about this later in this section).
    1. In the punctuate() method, iterate the local state store and send the aggregated counts to the downstream processor (we will talk about downstream processors later in this section), and commit the current stream state.
    2. -
      public class WordCountProcessor implements Processor<String, String> {
      +            
      public class WordCountProcessor implements Processor<String, String> {
       
      -  private ProcessorContext context;
      -  private KeyValueStore<String, Long> kvStore;
      +  private ProcessorContext context;
      +  private KeyValueStore<String, Long> kvStore;
       
      -  @Override
      -  @SuppressWarnings("unchecked")
      -  public void init(ProcessorContext context) {
      -      // keep the processor context locally because we need it in punctuate() and commit()
      -      this.context = context;
      +  @Override
      +  @SuppressWarnings("unchecked")
      +  public void init(ProcessorContext context) {
      +      // keep the processor context locally because we need it in punctuate() and commit()
      +      this.context = context;
       
      -      // retrieve the key-value store named "Counts"
      -      kvStore = (KeyValueStore) context.getStateStore("Counts");
      +      // retrieve the key-value store named "Counts"
      +      kvStore = (KeyValueStore) context.getStateStore("Counts");
       
      -      // schedule a punctuate() method every second based on stream-time
      -      this.context.schedule(Duration.ofSeconds(1000), PunctuationType.STREAM_TIME, (timestamp) -> {
      -          KeyValueIterator<String, Long> iter = this.kvStore.all();
      -          while (iter.hasNext()) {
      -              KeyValue<String, Long> entry = iter.next();
      -              context.forward(entry.key, entry.value.toString());
      -          }
      -          iter.close();
      +      // schedule a punctuate() method every second based on stream-time
      +      this.context.schedule(Duration.ofSeconds(1000), PunctuationType.STREAM_TIME, (timestamp) -> {
      +          KeyValueIterator<String, Long> iter = this.kvStore.all();
      +          while (iter.hasNext()) {
      +              KeyValue<String, Long> entry = iter.next();
      +              context.forward(entry.key, entry.value.toString());
      +          }
      +          iter.close();
       
      -          // commit the current processing progress
      -          context.commit();
      -      });
      -  }
      +          // commit the current processing progress
      +          context.commit();
      +      });
      +  }
       
      -  @Override
      -  public void punctuate(long timestamp) {
      -      // this method is deprecated and should not be used anymore
      -  }
      +  @Override
      +  public void punctuate(long timestamp) {
      +      // this method is deprecated and should not be used anymore
      +  }
       
      -  @Override
      -  public void close() {
      -      // close any resources managed by this processor
      -      // Note: Do not close any StateStores as these are managed by the library
      -  }
      +  @Override
      +  public void close() {
      +      // close any resources managed by this processor
      +      // Note: Do not close any StateStores as these are managed by the library
      +  }
       
      -}
      -
      +}

      Note

      Stateful processing with state stores: @@ -234,19 +233,18 @@

    3. Use persistentTimestampedWindowStore when you need a persistent windowedKey-(value/timestamp) store.
    4. -
      // Creating a persistent key-value store:
      -// here, we create a `KeyValueStore<String, Long>` named "persistent-counts".
      -import org.apache.kafka.streams.state.StoreBuilder;
      -import org.apache.kafka.streams.state.Stores;
      +                            
      // Creating a persistent key-value store:
      +// here, we create a `KeyValueStore<String, Long>` named "persistent-counts".
      +import org.apache.kafka.streams.state.StoreBuilder;
      +import org.apache.kafka.streams.state.Stores;
       
      -// Using a `KeyValueStoreBuilder` to build a `KeyValueStore`.
      -StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier =
      -  Stores.keyValueStoreBuilder(
      -    Stores.persistentKeyValueStore("persistent-counts"),
      -    Serdes.String(),
      -    Serdes.Long());
      -KeyValueStore<String, Long> countStore = countStoreSupplier.build();
      -
      +// Using a `KeyValueStoreBuilder` to build a `KeyValueStore`. +StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = + Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("persistent-counts"), + Serdes.String(), + Serdes.Long()); +KeyValueStore<String, Long> countStore = countStoreSupplier.build();

      @@ -317,15 +314,14 @@

      of the store through enableLogging() and disableLogging(). You can also fine-tune the associated topic’s configuration if needed.

      Example for disabling fault-tolerance:

      -
      import org.apache.kafka.streams.state.StoreBuilder;
      -import org.apache.kafka.streams.state.Stores;
      +                
      import org.apache.kafka.streams.state.StoreBuilder;
      +import org.apache.kafka.streams.state.Stores;
       
      -StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Stores.keyValueStoreBuilder(
      -  Stores.persistentKeyValueStore("Counts"),
      -    Serdes.String(),
      -    Serdes.Long())
      -  .withLoggingDisabled(); // disable backing up the store to a changelog topic
      -
      +StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("Counts"), + Serdes.String(), + Serdes.Long()) + .withLoggingDisabled(); // disable backing up the store to a changelog topic

      Attention

      If the changelog is disabled then the attached state store is no longer fault tolerant and it can’t have any standby replicas.

      @@ -333,19 +329,18 @@

      Here is an example for enabling fault tolerance, with additional changelog-topic configuration: You can add any log config from kafka.log.LogConfig. Unrecognized configs will be ignored.

      -
      import org.apache.kafka.streams.state.StoreBuilder;
      -import org.apache.kafka.streams.state.Stores;
      +                
      import org.apache.kafka.streams.state.StoreBuilder;
      +import org.apache.kafka.streams.state.Stores;
       
      -Map<String, String> changelogConfig = new HashMap();
      -// override min.insync.replicas
      -changelogConfig.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "1")
      +Map<String, String> changelogConfig = new HashMap();
      +// override min.insync.replicas
      +changelogConfig.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "1")
       
      -StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Stores.keyValueStoreBuilder(
      -  Stores.persistentKeyValueStore("Counts"),
      -    Serdes.String(),
      -    Serdes.Long())
      -  .withLoggingEnabled(changlogConfig); // enable changelogging, with custom changelog settings
      -
      +StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("Counts"), + Serdes.String(), + Serdes.Long()) + .withLoggingEnabled(changlogConfig); // enable changelogging, with custom changelog settings

      Timestamped State Stores

      @@ -389,12 +384,11 @@

      Accessing Processor Contextpartition, offset, timestamp and headers.

      Here is an example implementation of how to add a new header to the record:

      -
      public void process(String key, String value) {
      +            
      public void process(String key, String value) {
       
      -    // add a header to the elements
      -    context().headers().add.("key", "key"
      -}
      -
      + // add a header to the elements + context().headers().add.("key", "value"); +}

      Connecting Processors and State Stores

      Now that a processor (WordCountProcessor) and the @@ -403,16 +397,16 @@

      Connecting Processors and State Stores

      Here is an example implementation:

      -
                      Topology builder = new Topology();
      -                // add the source processor node that takes Kafka topic "source-topic" as input
      -                builder.addSource("Source", "source-topic")
      -                    // add the WordCountProcessor node which takes the source processor as its upstream processor
      -                    .addProcessor("Process", () -> new WordCountProcessor(), "Source")
      -                    // add the count store associated with the WordCountProcessor processor
      -                    .addStateStore(countStoreBuilder, "Process")
      -                    // add the sink processor node that takes Kafka topic "sink-topic" as output
      -                    // and the WordCountProcessor node as its upstream processor
      -                    .addSink("Sink", "sink-topic", "Process");
      +
      Topology builder = new Topology();
      +// add the source processor node that takes Kafka topic "source-topic" as input
      +builder.addSource("Source", "source-topic")
      +    // add the WordCountProcessor node which takes the source processor as its upstream processor
      +    .addProcessor("Process", () -> new WordCountProcessor(), "Source")
      +    // add the count store associated with the WordCountProcessor processor
      +    .addStateStore(countStoreBuilder, "Process")
      +    // add the sink processor node that takes Kafka topic "sink-topic" as output
      +    // and the WordCountProcessor node as its upstream processor
      +    .addSink("Sink", "sink-topic", "Process");

      Here is a quick explanation of this example:

      • A source processor node named "Source" is added to the topology using the addSource method, with one Kafka topic @@ -429,22 +423,22 @@

        Connecting Processors and State StoresConnectedStoreProvider#stores() on the ProcessorSupplier instead of calling Topology#addStateStore(), like this:

        -
                        Topology builder = new Topology();
        -                // add the source processor node that takes Kafka "source-topic" as input
        -                builder.addSource("Source", "source-topic")
        -                    // add the WordCountProcessor node which takes the source processor as its upstream processor.
        -                    // the ProcessorSupplier provides the count store associated with the WordCountProcessor
        -                    .addProcessor("Process", new ProcessorSupplier<String, String>() {
        -                        public Processor<String, String> get() {
        -                            return new WordCountProcessor();
        -                        }
        -                        public Set<StoreBuilder<?>> stores() {
        -                            return countStoreBuilder;
        -                        }
        -                    }, "Source")
        -                    // add the sink processor node that takes Kafka topic "sink-topic" as output
        -                    // and the WordCountProcessor node as its upstream processor
        -                    .addSink("Sink", "sink-topic", "Process");
        +
        Topology builder = new Topology();
        +// add the source processor node that takes Kafka "source-topic" as input
        +builder.addSource("Source", "source-topic")
        +    // add the WordCountProcessor node which takes the source processor as its upstream processor.
        +    // the ProcessorSupplier provides the count store associated with the WordCountProcessor
        +    .addProcessor("Process", new ProcessorSupplier<String, String>() {
        +        public Processor<String, String> get() {
        +            return new WordCountProcessor();
        +        }
        +        public Set<StoreBuilder<?>> stores() {
        +            return countStoreBuilder;
        +        }
        +    }, "Source")
        +    // add the sink processor node that takes Kafka topic "sink-topic" as output
        +    // and the WordCountProcessor node as its upstream processor
        +    .addSink("Sink", "sink-topic", "Process");

        This allows for a processor to "own" state stores, effectively encapsulating their usage from the user wiring the topology. Multiple processors that share a state store may provide the same store with this technique, as long as the StoreBuilder is the same instance.

        In these topologies, the "Process" stream processor node is considered a downstream processor of the "Source" node, and an diff --git a/docs/streams/developer-guide/running-app.html b/docs/streams/developer-guide/running-app.html index 87ee8f0a9da30..ff3ed75d29010 100644 --- a/docs/streams/developer-guide/running-app.html +++ b/docs/streams/developer-guide/running-app.html @@ -51,10 +51,9 @@

        Starting a Kafka Streams application

        You can package your Java application as a fat JAR file and then start the application like this:

        -
        # Start the application in class `com.example.MyStreamsApp`
        -# from the fat JAR named `path-to-app-fatjar.jar`.
        -$ java -cp path-to-app-fatjar.jar com.example.MyStreamsApp
        -
        +
        # Start the application in class `com.example.MyStreamsApp`
        +# from the fat JAR named `path-to-app-fatjar.jar`.
        +$ java -cp path-to-app-fatjar.jar com.example.MyStreamsApp

        When you start your application you are launching a Kafka Streams instance of your application. You can run multiple instances of your application. A common scenario is that there are multiple instances of your application running in parallel. For more information, see Parallelism Model.

        diff --git a/docs/streams/developer-guide/security.html b/docs/streams/developer-guide/security.html index 05de0794d63a9..63bc942f7f5a0 100644 --- a/docs/streams/developer-guide/security.html +++ b/docs/streams/developer-guide/security.html @@ -98,47 +98,44 @@ then you must also include these SSL certificates in the correct locations within the Docker image.

        The snippet below shows the settings to enable client authentication and SSL encryption for data-in-transit between your Kafka Streams application and the Kafka cluster it is reading and writing from:

        -
        # Essential security settings to enable client authentication and SSL encryption
        -bootstrap.servers=kafka.example.com:9093
        -security.protocol=SSL
        -ssl.truststore.location=/etc/security/tls/kafka.client.truststore.jks
        -ssl.truststore.password=test1234
        -ssl.keystore.location=/etc/security/tls/kafka.client.keystore.jks
        -ssl.keystore.password=test1234
        -ssl.key.password=test1234
        -
        +
        # Essential security settings to enable client authentication and SSL encryption
        +bootstrap.servers=kafka.example.com:9093
        +security.protocol=SSL
        +ssl.truststore.location=/etc/security/tls/kafka.client.truststore.jks
        +ssl.truststore.password=test1234
        +ssl.keystore.location=/etc/security/tls/kafka.client.keystore.jks
        +ssl.keystore.password=test1234
        +ssl.key.password=test1234

        Configure these settings in the application for your Properties instance. These settings will encrypt any data-in-transit that is being read from or written to Kafka, and your application will authenticate itself against the Kafka brokers that it is communicating with. Note that this example does not cover client authorization.

        -
        // Code of your Java application that uses the Kafka Streams library
        -Properties settings = new Properties();
        -settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "secure-kafka-streams-app");
        -// Where to find secure Kafka brokers.  Here, it's on port 9093.
        -settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka.example.com:9093");
        -//
        -// ...further non-security related settings may follow here...
        -//
        -// Security settings.
        -// 1. These settings must match the security settings of the secure Kafka cluster.
        -// 2. The SSL trust store and key store files must be locally accessible to the application.
        -settings.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL");
        -settings.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "/etc/security/tls/kafka.client.truststore.jks");
        -settings.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "test1234");
        -settings.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "/etc/security/tls/kafka.client.keystore.jks");
        -settings.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "test1234");
        -settings.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, "test1234");
        -
        +
        // Code of your Java application that uses the Kafka Streams library
        +Properties settings = new Properties();
        +settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "secure-kafka-streams-app");
        +// Where to find secure Kafka brokers.  Here, it's on port 9093.
        +settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka.example.com:9093");
        +//
        +// ...further non-security related settings may follow here...
        +//
        +// Security settings.
        +// 1. These settings must match the security settings of the secure Kafka cluster.
        +// 2. The SSL trust store and key store files must be locally accessible to the application.
        +settings.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL");
        +settings.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "/etc/security/tls/kafka.client.truststore.jks");
        +settings.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "test1234");
        +settings.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "/etc/security/tls/kafka.client.keystore.jks");
        +settings.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "test1234");
        +settings.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, "test1234");

        If you incorrectly configure a security setting in your application, it will fail at runtime, typically right after you start it. For example, if you enter an incorrect password for the ssl.keystore.password setting, an error message similar to this would be logged and then the application would terminate:

        -
        # Misconfigured ssl.keystore.password
        -Exception in thread "main" org.apache.kafka.common.KafkaException: Failed to construct kafka producer
        -[...snip...]
        +            
        # Misconfigured ssl.keystore.password
        +Exception in thread "main" org.apache.kafka.common.KafkaException: Failed to construct kafka producer
        +[...snip...]
         Caused by: org.apache.kafka.common.KafkaException: org.apache.kafka.common.KafkaException:
            java.io.IOException: Keystore was tampered with, or password was incorrect
        -[...snip...]
        -Caused by: java.security.UnrecoverableKeyException: Password verification failed
        -
        +[...snip...] +Caused by: java.security.UnrecoverableKeyException: Password verification failed

        Monitor your Kafka Streams application log files for such error messages to spot any misconfigured applications quickly.

      diff --git a/docs/streams/developer-guide/testing.html b/docs/streams/developer-guide/testing.html index ceef648d9c4d8..b5fadb12b3d3e 100644 --- a/docs/streams/developer-guide/testing.html +++ b/docs/streams/developer-guide/testing.html @@ -71,15 +71,15 @@

      // Processor API +
      // Processor API
       Topology topology = new Topology();
      -topology.addSource("sourceProcessor", "input-topic");
      -topology.addProcessor("processor", ..., "sourceProcessor");
      -topology.addSink("sinkProcessor", "output-topic", "processor");
      +topology.addSource("sourceProcessor", "input-topic");
      +topology.addProcessor("processor", ..., "sourceProcessor");
      +topology.addSink("sinkProcessor", "output-topic", "processor");
       // or
       // using DSL
       StreamsBuilder builder = new StreamsBuilder();
      -builder.stream("input-topic").filter(...).to("output-topic");
      +builder.stream("input-topic").filter(...).to("output-topic");
       Topology topology = builder.build();
       
       // create test driver
      @@ -88,7 +88,7 @@ 

      TestInputTopic<String, Long> inputTopic = testDriver.createInputTopic("input-topic", stringSerde.serializer(), longSerde.serializer()); +
      TestInputTopic<String, Long> inputTopic = testDriver.createInputTopic("input-topic", stringSerde.serializer(), longSerde.serializer());
       inputTopic.pipeInput("key", 42L);

      To verify the output, you can use TestOutputTopic @@ -97,7 +97,7 @@

      TestOutputTopic<String, Long> outputTopic = testDriver.createOutputTopic("output-topic", stringSerde.deserializer(), longSerde.deserializer()); +
      TestOutputTopic<String, Long> outputTopic = testDriver.createOutputTopic("output-topic", stringSerde.deserializer(), longSerde.deserializer());
       assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("key", 42L)));

      TopologyTestDriver supports punctuations, too. @@ -105,18 +105,18 @@

      testDriver.advanceWallClockTime(Duration.ofSeconds(20));

      +
      testDriver.advanceWallClockTime(Duration.ofSeconds(20));

      Additionally, you can access state stores via the test driver before or after a test. Accessing stores before a test is useful to pre-populate a store with some initial values. After data was processed, expected updates to the store can be verified.

      -
      KeyValueStore store = testDriver.getKeyValueStore("store-name");
      +
      KeyValueStore store = testDriver.getKeyValueStore("store-name");

      Note, that you should always close the test driver at the end to make sure all resources are release properly.

      -
      testDriver.close();
      +
      testDriver.close();

      Example

      @@ -125,7 +125,7 @@

      Example

      While processing, no output is generated, but only the store is updated. Output is only sent downstream based on event-time and wall-clock punctuations.

      -
      private TopologyTestDriver testDriver;
      +            
      private TopologyTestDriver testDriver;
       private TestInputTopic<String, Long> inputTopic;
       private TestOutputTopic<String, Long> outputTopic;
       private KeyValueStore<String, Long> store;
      @@ -275,21 +275,21 @@ 

      Construction

      To begin with, instantiate your processor and initialize it with the mock context: -

      final Processor processorUnderTest = ...;
      +            
      final Processor processorUnderTest = ...;
       final MockProcessorContext context = new MockProcessorContext();
       processorUnderTest.init(context);
      If you need to pass configuration to your processor or set the default serdes, you can create the mock with config: -
      final Properties props = new Properties();
      +            
      final Properties props = new Properties();
       props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
       props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
      -props.put("some.other.config", "some config value");
      +props.put("some.other.config", "some config value");
       final MockProcessorContext context = new MockProcessorContext(props);

      Captured data

      The mock will capture any values that your processor forwards. You can make assertions on them: -

      processorUnderTest.process("key", "value");
      +            
      processorUnderTest.process("key", "value");
       
       final Iterator<CapturedForward> forwarded = context.forwarded().iterator();
       assertEquals(forwarded.next().keyValue(), new KeyValue<>(..., ...));
      @@ -301,9 +301,9 @@ 

      assertEquals(context.forwarded().size(), 0);

      If your processor forwards to specific child processors, you can query the context for captured data by child name: -
      final List<CapturedForward> captures = context.forwarded("childProcessorName");
      +
      final List<CapturedForward> captures = context.forwarded("childProcessorName");
      The mock also captures whether your processor has called commit() on the context: -
      assertTrue(context.committed());
      +            
      assertTrue(context.committed());
       
       // commit captures can also be reset.
       context.resetCommit();
      @@ -314,8 +314,8 @@ 

      In case your processor logic depends on the record metadata (topic, partition, offset, or timestamp), you can set them on the context, either all together or individually: -

      context.setRecordMetadata("topicName", /*partition*/ 0, /*offset*/ 0L, /*timestamp*/ 0L);
      -context.setTopic("topicName");
      +            
      context.setRecordMetadata("topicName", /*partition*/ 0, /*offset*/ 0L, /*timestamp*/ 0L);
      +context.setTopic("topicName");
       context.setPartition(0);
       context.setOffset(0L);
       context.setTimestamp(0L);
      @@ -327,7 +327,7 @@

      You're encouraged to use a simple in-memory store of the appropriate type (KeyValue, Windowed, or Session), since the mock context does not manage changelogs, state directories, etc.

      -
      final KeyValueStore<String, Integer> store =
      +            
      final KeyValueStore<String, Integer> store =
           Stores.keyValueStoreBuilder(
                   Stores.inMemoryKeyValueStore("myStore"),
                   Serdes.String(),
      @@ -342,7 +342,7 @@ 

      Processors can schedule punctuators to handle periodic tasks. The mock context does not automatically execute punctuators, but it does capture them to allow you to unit test them as well: -
      final MockProcessorContext.CapturedPunctuator capturedPunctuator = context.scheduledPunctuators().get(0);
      +            
      final MockProcessorContext.CapturedPunctuator capturedPunctuator = context.scheduledPunctuators().get(0);
       final long interval = capturedPunctuator.getIntervalMs();
       final PunctuationType type = capturedPunctuator.getType();
       final boolean cancelled = capturedPunctuator.cancelled();
      diff --git a/docs/streams/developer-guide/write-streams.html b/docs/streams/developer-guide/write-streams.html
      index 720b0c376756c..03bd16328efbe 100644
      --- a/docs/streams/developer-guide/write-streams.html
      +++ b/docs/streams/developer-guide/write-streams.html
      @@ -90,22 +90,22 @@
                     

      See the section Data Types and Serialization for more information about Serializers/Deserializers.

      Example pom.xml snippet when using Maven:

      -
      
      -    org.apache.kafka
      -    kafka-streams
      -    {{fullDotVersion}}
      -
      -
      -    org.apache.kafka
      -    kafka-clients
      -    {{fullDotVersion}}
      -
      -
      -
      -    org.apache.kafka
      -    kafka-streams-scala_{{scalaVersion}}
      -    {{fullDotVersion}}
      -
      +
      <dependency>
      +    <groupId>org.apache.kafka</groupId>
      +    <artifactId>kafka-streams</artifactId>
      +    <version>{{fullDotVersion}}</version>
      +</dependency>
      +<dependency>
      +    <groupId>org.apache.kafka</groupId>
      +    <artifactId>kafka-clients</artifactId>
      +    <version>{{fullDotVersion}}</version>
      +</dependency>
      +<!-- Optionally include Kafka Streams DSL for Scala for Scala {{scalaVersion}} -->
      +<dependency>
      +    <groupId>org.apache.kafka</groupId>
      +    <artifactId>kafka-streams-scala_{{scalaVersion}}</artifactId>
      +    <version>{{fullDotVersion}}</version>
      +</dependency>

      Using Kafka Streams within your application code

      @@ -120,79 +120,69 @@

      Using Kafka Streams within your application codejava.util.Properties, which defines the configuration for this specific topology.

      Code example:

      -
      import org.apache.kafka.streams.KafkaStreams;
      -import org.apache.kafka.streams.kstream.StreamsBuilder;
      -import org.apache.kafka.streams.processor.Topology;
      -
      -// Use the builders to define the actual processing topology, e.g. to specify
      -// from which input topics to read, which stream operations (filter, map, etc.)
      -// should be called, and so on.  We will cover this in detail in the subsequent
      -// sections of this Developer Guide.
      -
      -StreamsBuilder builder = ...;  // when using the DSL
      -Topology topology = builder.build();
      -//
      -// OR
      -//
      -Topology topology = ...; // when using the Processor API
      -
      -// Use the configuration to tell your application where the Kafka cluster is,
      -// which Serializers/Deserializers to use by default, to specify security settings,
      -// and so on.
      -Properties props = ...;
      -
      -KafkaStreams streams = new KafkaStreams(topology, props);
      -
      +
      import org.apache.kafka.streams.KafkaStreams;
      +import org.apache.kafka.streams.kstream.StreamsBuilder;
      +import org.apache.kafka.streams.processor.Topology;
      +
      +// Use the builders to define the actual processing topology, e.g. to specify
      +// from which input topics to read, which stream operations (filter, map, etc.)
      +// should be called, and so on.  We will cover this in detail in the subsequent
      +// sections of this Developer Guide.
      +
      +StreamsBuilder builder = ...;  // when using the DSL
      +Topology topology = builder.build();
      +//
      +// OR
      +//
      +Topology topology = ...; // when using the Processor API
      +
      +// Use the configuration to tell your application where the Kafka cluster is,
      +// which Serializers/Deserializers to use by default, to specify security settings,
      +// and so on.
      +Properties props = ...;
      +
      +KafkaStreams streams = new KafkaStreams(topology, props);

      At this point, internal structures are initialized, but the processing is not started yet. You have to explicitly start the Kafka Streams thread by calling the KafkaStreams#start() method:

      -
      // Start the Kafka Streams threads
      -streams.start();
      -
      +
      // Start the Kafka Streams threads
      +streams.start();

      If there are other instances of this stream processing application running elsewhere (e.g., on another machine), Kafka Streams transparently re-assigns tasks from the existing instances to the new instance that you just started. For more information, see Stream Partitions and Tasks and Threading Model.

      To catch any unexpected exceptions, you can set an java.lang.Thread.UncaughtExceptionHandler before you start the application. This handler is called whenever a stream thread is terminated by an unexpected exception:

      -
      // Java 8+, using lambda expressions
      -streams.setUncaughtExceptionHandler((Thread thread, Throwable throwable) -> {
      -  // here you should examine the throwable/exception and perform an appropriate action!
      -});
      -
      -
      -// Java 7
      -streams.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
      -  public void uncaughtException(Thread thread, Throwable throwable) {
      -    // here you should examine the throwable/exception and perform an appropriate action!
      -  }
      -});
      -
      +
      // Java 8+, using lambda expressions
      +streams.setUncaughtExceptionHandler((Thread thread, Throwable throwable) -> {
      +  // here you should examine the throwable/exception and perform an appropriate action!
      +});
      +
      +
      +// Java 7
      +streams.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
      +  public void uncaughtException(Thread thread, Throwable throwable) {
      +    // here you should examine the throwable/exception and perform an appropriate action!
      +  }
      +});

      To stop the application instance, call the KafkaStreams#close() method:

      -
      // Stop the Kafka Streams threads
      -streams.close();
      -
      +
      // Stop the Kafka Streams threads
      +streams.close();

      To allow your application to gracefully shutdown in response to SIGTERM, it is recommended that you add a shutdown hook and call KafkaStreams#close.

      • Here is a shutdown hook example in Java 8+:

        -
        -
        // Add shutdown hook to stop the Kafka Streams threads.
        -// You can optionally provide a timeout to `close`.
        -Runtime.getRuntime().addShutdownHook(new Thread(streams::close));
        -
        -
        +
        // Add shutdown hook to stop the Kafka Streams threads.
        +// You can optionally provide a timeout to `close`.
        +Runtime.getRuntime().addShutdownHook(new Thread(streams::close));
      • Here is a shutdown hook example in Java 7:

        -
        -
        // Add shutdown hook to stop the Kafka Streams threads.
        -// You can optionally provide a timeout to `close`.
        -Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
        -  @Override
        -  public void run() {
        -      streams.close();
        -  }
        -}));
        -
        -
        +
        // Add shutdown hook to stop the Kafka Streams threads.
        +// You can optionally provide a timeout to `close`.
        +Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
        +  @Override
        +  public void run() {
        +      streams.close();
        +  }
        +}));

      After an application is stopped, Kafka Streams will migrate any tasks that had been running in this instance to available remaining diff --git a/docs/streams/index.html b/docs/streams/index.html index 3d84bbfee7802..e38b3890af9ce 100644 --- a/docs/streams/index.html +++ b/docs/streams/index.html @@ -154,95 +154,95 @@

      Hello Kafka Streams

      -
                         import org.apache.kafka.common.serialization.Serdes;
      -                   import org.apache.kafka.common.utils.Bytes;
      -                   import org.apache.kafka.streams.KafkaStreams;
      -                   import org.apache.kafka.streams.StreamsBuilder;
      -                   import org.apache.kafka.streams.StreamsConfig;
      -                   import org.apache.kafka.streams.kstream.KStream;
      -                   import org.apache.kafka.streams.kstream.KTable;
      -                   import org.apache.kafka.streams.kstream.Materialized;
      -                   import org.apache.kafka.streams.kstream.Produced;
      -                   import org.apache.kafka.streams.state.KeyValueStore;
      -
      -                   import java.util.Arrays;
      -                   import java.util.Properties;
      -
      -                   public class WordCountApplication {
      -
      -                       public static void main(final String[] args) throws Exception {
      -                           Properties props = new Properties();
      -                           props.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application");
      -                           props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
      -                           props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      -                           props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      -
      -                           StreamsBuilder builder = new StreamsBuilder();
      -                           KStream<String, String> textLines = builder.stream("TextLinesTopic");
      -                           KTable<String, Long> wordCounts = textLines
      -                               .flatMapValues(textLine -> Arrays.asList(textLine.toLowerCase().split("\\W+")))
      -                               .groupBy((key, word) -> word)
      -                               .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"));
      -                           wordCounts.toStream().to("WordsWithCountsTopic", Produced.with(Serdes.String(), Serdes.Long()));
      -
      -                           KafkaStreams streams = new KafkaStreams(builder.build(), props);
      -                           streams.start();
      -                       }
      -
      -                   }
      +
      import org.apache.kafka.common.serialization.Serdes;
      +import org.apache.kafka.common.utils.Bytes;
      +import org.apache.kafka.streams.KafkaStreams;
      +import org.apache.kafka.streams.StreamsBuilder;
      +import org.apache.kafka.streams.StreamsConfig;
      +import org.apache.kafka.streams.kstream.KStream;
      +import org.apache.kafka.streams.kstream.KTable;
      +import org.apache.kafka.streams.kstream.Materialized;
      +import org.apache.kafka.streams.kstream.Produced;
      +import org.apache.kafka.streams.state.KeyValueStore;
      +
      +import java.util.Arrays;
      +import java.util.Properties;
      +
      +public class WordCountApplication {
      +
      +   public static void main(final String[] args) throws Exception {
      +       Properties props = new Properties();
      +       props.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application");
      +       props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
      +       props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +       props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +
      +       StreamsBuilder builder = new StreamsBuilder();
      +       KStream<String, String> textLines = builder.stream("TextLinesTopic");
      +       KTable<String, Long> wordCounts = textLines
      +           .flatMapValues(textLine -> Arrays.asList(textLine.toLowerCase().split("\\W+")))
      +           .groupBy((key, word) -> word)
      +           .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"));
      +       wordCounts.toStream().to("WordsWithCountsTopic", Produced.with(Serdes.String(), Serdes.Long()));
      +
      +       KafkaStreams streams = new KafkaStreams(builder.build(), props);
      +       streams.start();
      +   }
      +
      +}
      -
                         import org.apache.kafka.common.serialization.Serdes;
      -                   import org.apache.kafka.common.utils.Bytes;
      -                   import org.apache.kafka.streams.KafkaStreams;
      -                   import org.apache.kafka.streams.StreamsBuilder;
      -                   import org.apache.kafka.streams.StreamsConfig;
      -                   import org.apache.kafka.streams.kstream.KStream;
      -                   import org.apache.kafka.streams.kstream.KTable;
      -                   import org.apache.kafka.streams.kstream.ValueMapper;
      -                   import org.apache.kafka.streams.kstream.KeyValueMapper;
      -                   import org.apache.kafka.streams.kstream.Materialized;
      -                   import org.apache.kafka.streams.kstream.Produced;
      -                   import org.apache.kafka.streams.state.KeyValueStore;
      -
      -                   import java.util.Arrays;
      -                   import java.util.Properties;
      -
      -                   public class WordCountApplication {
      -
      -                       public static void main(final String[] args) throws Exception {
      -                           Properties props = new Properties();
      -                           props.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application");
      -                           props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
      -                           props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      -                           props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      -
      -                           StreamsBuilder builder = new StreamsBuilder();
      -                           KStream<String, String> textLines = builder.stream("TextLinesTopic");
      -                           KTable<String, Long> wordCounts = textLines
      -                               .flatMapValues(new ValueMapper<String, Iterable<String>>() {
      -                                   @Override
      -                                   public Iterable<String> apply(String textLine) {
      -                                       return Arrays.asList(textLine.toLowerCase().split("\\W+"));
      -                                   }
      -                               })
      -                               .groupBy(new KeyValueMapper<String, String, String>() {
      -                                   @Override
      -                                   public String apply(String key, String word) {
      -                                       return word;
      -                                   }
      -                               })
      -                               .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"));
      -
      -
      -                           wordCounts.toStream().to("WordsWithCountsTopic", Produced.with(Serdes.String(), Serdes.Long()));
      -
      -                           KafkaStreams streams = new KafkaStreams(builder.build(), props);
      -                           streams.start();
      -                       }
      -
      -                   }
      +
      import org.apache.kafka.common.serialization.Serdes;
      +import org.apache.kafka.common.utils.Bytes;
      +import org.apache.kafka.streams.KafkaStreams;
      +import org.apache.kafka.streams.StreamsBuilder;
      +import org.apache.kafka.streams.StreamsConfig;
      +import org.apache.kafka.streams.kstream.KStream;
      +import org.apache.kafka.streams.kstream.KTable;
      +import org.apache.kafka.streams.kstream.ValueMapper;
      +import org.apache.kafka.streams.kstream.KeyValueMapper;
      +import org.apache.kafka.streams.kstream.Materialized;
      +import org.apache.kafka.streams.kstream.Produced;
      +import org.apache.kafka.streams.state.KeyValueStore;
      +
      +import java.util.Arrays;
      +import java.util.Properties;
      +
      +public class WordCountApplication {
      +
      +   public static void main(final String[] args) throws Exception {
      +       Properties props = new Properties();
      +       props.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application");
      +       props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
      +       props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +       props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +
      +       StreamsBuilder builder = new StreamsBuilder();
      +       KStream<String, String> textLines = builder.stream("TextLinesTopic");
      +       KTable<String, Long> wordCounts = textLines
      +           .flatMapValues(new ValueMapper<String, Iterable<String>>() {
      +               @Override
      +               public Iterable<String> apply(String textLine) {
      +                   return Arrays.asList(textLine.toLowerCase().split("\\W+"));
      +               }
      +           })
      +           .groupBy(new KeyValueMapper<String, String, String>() {
      +               @Override
      +               public String apply(String key, String word) {
      +                   return word;
      +               }
      +           })
      +           .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"));
      +
      +
      +       wordCounts.toStream().to("WordsWithCountsTopic", Produced.with(Serdes.String(), Serdes.Long()));
      +
      +       KafkaStreams streams = new KafkaStreams(builder.build(), props);
      +       streams.start();
      +   }
      +
      +}
      diff --git a/docs/streams/tutorial.html b/docs/streams/tutorial.html index 2fa4dc3ec5a0c..a526de568abb1 100644 --- a/docs/streams/tutorial.html +++ b/docs/streams/tutorial.html @@ -42,32 +42,31 @@

      We are going to use a Kafka Streams Maven Archetype for creating a Streams project structure with the following commands:

      -
              mvn archetype:generate \
      -            -DarchetypeGroupId=org.apache.kafka \
      -            -DarchetypeArtifactId=streams-quickstart-java \
      -            -DarchetypeVersion={{fullDotVersion}} \
      -            -DgroupId=streams.examples \
      -            -DartifactId=streams.examples \
      -            -Dversion=0.1 \
      -            -Dpackage=myapps
      - +
      mvn archetype:generate \
      +    -DarchetypeGroupId=org.apache.kafka \
      +    -DarchetypeArtifactId=streams-quickstart-java \
      +    -DarchetypeVersion={{fullDotVersion}} \
      +    -DgroupId=streams.examples \
      +    -DartifactId=streams.examples \
      +    -Dversion=0.1 \
      +    -Dpackage=myapps

      You can use a different value for groupId, artifactId and package parameters if you like. Assuming the above parameter values are used, this command will create a project structure that looks like this:

      -
              > tree streams.examples
      -        streams-quickstart
      -        |-- pom.xml
      -        |-- src
      -            |-- main
      -                |-- java
      -                |   |-- myapps
      -                |       |-- LineSplit.java
      -                |       |-- Pipe.java
      -                |       |-- WordCount.java
      -                |-- resources
      -                    |-- log4j.properties
      +
      > tree streams.examples
      +    streams-quickstart
      +    |-- pom.xml
      +    |-- src
      +        |-- main
      +            |-- java
      +            |   |-- myapps
      +            |       |-- LineSplit.java
      +            |       |-- Pipe.java
      +            |       |-- WordCount.java
      +            |-- resources
      +                |-- log4j.properties

      The pom.xml file included in the project already has the Streams dependency defined. @@ -79,22 +78,22 @@

      Since we are going to start writing such programs from scratch, we can now delete these examples:

      -
              > cd streams-quickstart
      -        > rm src/main/java/myapps/*.java
      +
      > cd streams-quickstart
      +> rm src/main/java/myapps/*.java

      Writing a first Streams application: Pipe

      It's coding time now! Feel free to open your favorite IDE and import this Maven project, or simply open a text editor and create a java file under src/main/java/myapps. Let's name it Pipe.java: -
              package myapps;
      +    
      package myapps;
       
      -        public class Pipe {
      +public class Pipe {
       
      -            public static void main(String[] args) throws Exception {
      +    public static void main(String[] args) throws Exception {
       
      -            }
      -        }
      + } +}

      We are going to fill in the main function to write this pipe program. Note that we will not list the import statements as we go since IDEs can usually add them automatically. @@ -107,16 +106,16 @@

      Writing a first Stream and StreamsConfig.APPLICATION_ID_CONFIG, which gives the unique identifier of your Streams application to distinguish itself with other applications talking to the same Kafka cluster:

      -
              Properties props = new Properties();
      -        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe");
      -        props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");    // assuming that the Kafka broker this application is talking to runs on local machine with port 9092
      +
      Properties props = new Properties();
      +props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe");
      +props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");    // assuming that the Kafka broker this application is talking to runs on local machine with port 9092

      In addition, you can customize other configurations in the same map, for example, default serialization and deserialization libraries for the record key-value pairs:

      -
              props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      -        props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +
      props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());

      For a full list of configurations of Kafka Streams please refer to this table. @@ -128,13 +127,13 @@

      Writing a first Stream We can use a topology builder to construct such a topology,

      -
              final StreamsBuilder builder = new StreamsBuilder();
      +
      final StreamsBuilder builder = new StreamsBuilder();

      And then create a source stream from a Kafka topic named streams-plaintext-input using this topology builder:

      -
              KStream<String, String> source = builder.stream("streams-plaintext-input");
      +
      KStream<String, String> source = builder.stream("streams-plaintext-input");

      Now we get a KStream that is continuously generating records from its source Kafka topic streams-plaintext-input. @@ -142,38 +141,38 @@

      Writing a first Stream The simplest thing we can do with this stream is to write it into another Kafka topic, say it's named streams-pipe-output:

      -
              source.to("streams-pipe-output");
      +
      source.to("streams-pipe-output");

      Note that we can also concatenate the above two lines into a single line as:

      -
              builder.stream("streams-plaintext-input").to("streams-pipe-output");
      +
      builder.stream("streams-plaintext-input").to("streams-pipe-output");

      We can inspect what kind of topology is created from this builder by doing the following:

      -
              final Topology topology = builder.build();
      +
      final Topology topology = builder.build();

      And print its description to standard output as:

      -
              System.out.println(topology.describe());
      +
      System.out.println(topology.describe());

      If we just stop here, compile and run the program, it will output the following information:

      -
              > mvn clean package
      -        > mvn exec:java -Dexec.mainClass=myapps.Pipe
      -        Sub-topologies:
      -          Sub-topology: 0
      -            Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-SINK-0000000001
      -            Sink: KSTREAM-SINK-0000000001(topic: streams-pipe-output) <-- KSTREAM-SOURCE-0000000000
      -        Global Stores:
      -          none
      +
      > mvn clean package
      +> mvn exec:java -Dexec.mainClass=myapps.Pipe
      +Sub-topologies:
      +  Sub-topology: 0
      +    Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-SINK-0000000001
      +    Sink: KSTREAM-SINK-0000000001(topic: streams-pipe-output) <-- KSTREAM-SOURCE-0000000000
      +Global Stores:
      +  none

      As shown above, it illustrates that the constructed topology has two processor nodes, a source node KSTREAM-SOURCE-0000000000 and a sink node KSTREAM-SINK-0000000001. @@ -189,7 +188,7 @@

      Writing a first Stream we can now construct the Streams client with the two components we have just constructed above: the configuration map specified in a java.util.Properties instance and the Topology object.

      -
              final KafkaStreams streams = new KafkaStreams(topology, props);
      +
      final KafkaStreams streams = new KafkaStreams(topology, props);

      By calling its start() function we can trigger the execution of this client. @@ -197,76 +196,76 @@

      Writing a first Stream We can, for example, add a shutdown hook with a countdown latch to capture a user interrupt and close the client upon terminating this program:

      -
              final CountDownLatch latch = new CountDownLatch(1);
      +    
      final CountDownLatch latch = new CountDownLatch(1);
       
      -        // attach shutdown handler to catch control-c
      -        Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") {
      -            @Override
      -            public void run() {
      -                streams.close();
      -                latch.countDown();
      -            }
      -        });
      +// attach shutdown handler to catch control-c
      +Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") {
      +    @Override
      +    public void run() {
      +        streams.close();
      +        latch.countDown();
      +    }
      +});
       
      -        try {
      -            streams.start();
      -            latch.await();
      -        } catch (Throwable e) {
      -            System.exit(1);
      -        }
      -        System.exit(0);
      +try { + streams.start(); + latch.await(); +} catch (Throwable e) { + System.exit(1); +} +System.exit(0);

      The complete code so far looks like this:

      -
              package myapps;
      -
      -        import org.apache.kafka.common.serialization.Serdes;
      -        import org.apache.kafka.streams.KafkaStreams;
      -        import org.apache.kafka.streams.StreamsBuilder;
      -        import org.apache.kafka.streams.StreamsConfig;
      -        import org.apache.kafka.streams.Topology;
      +    
      package myapps;
       
      -        import java.util.Properties;
      -        import java.util.concurrent.CountDownLatch;
      +import org.apache.kafka.common.serialization.Serdes;
      +import org.apache.kafka.streams.KafkaStreams;
      +import org.apache.kafka.streams.StreamsBuilder;
      +import org.apache.kafka.streams.StreamsConfig;
      +import org.apache.kafka.streams.Topology;
       
      -        public class Pipe {
      +import java.util.Properties;
      +import java.util.concurrent.CountDownLatch;
       
      -            public static void main(String[] args) throws Exception {
      -                Properties props = new Properties();
      -                props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe");
      -                props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
      -                props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      -                props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +public class Pipe {
       
      -                final StreamsBuilder builder = new StreamsBuilder();
      +    public static void main(String[] args) throws Exception {
      +        Properties props = new Properties();
      +        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe");
      +        props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
      +        props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +        props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
       
      -                builder.stream("streams-plaintext-input").to("streams-pipe-output");
      +        final StreamsBuilder builder = new StreamsBuilder();
       
      -                final Topology topology = builder.build();
      +        builder.stream("streams-plaintext-input").to("streams-pipe-output");
       
      -                final KafkaStreams streams = new KafkaStreams(topology, props);
      -                final CountDownLatch latch = new CountDownLatch(1);
      +        final Topology topology = builder.build();
       
      -                // attach shutdown handler to catch control-c
      -                Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") {
      -                    @Override
      -                    public void run() {
      -                        streams.close();
      -                        latch.countDown();
      -                    }
      -                });
      +        final KafkaStreams streams = new KafkaStreams(topology, props);
      +        final CountDownLatch latch = new CountDownLatch(1);
       
      -                try {
      -                    streams.start();
      -                    latch.await();
      -                } catch (Throwable e) {
      -                    System.exit(1);
      -                }
      -                System.exit(0);
      +        // attach shutdown handler to catch control-c
      +        Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") {
      +            @Override
      +            public void run() {
      +                streams.close();
      +                latch.countDown();
                   }
      -        }
      + }); + + try { + streams.start(); + latch.await(); + } catch (Throwable e) { + System.exit(1); + } + System.exit(0); + } +}

      If you already have the Kafka broker up and running at localhost:9092, @@ -274,8 +273,8 @@

      Writing a first Stream you can run this code in your IDE or on the command line, using Maven:

      -
              > mvn clean package
      -        > mvn exec:java -Dexec.mainClass=myapps.Pipe
      +
      > mvn clean package
      +> mvn exec:java -Dexec.mainClass=myapps.Pipe

      For detailed instructions on how to run a Streams application and observe its computing results, @@ -291,33 +290,33 @@

      Writing a se We can first create another program by first copy the existing Pipe.java class:

      -
              > cp src/main/java/myapps/Pipe.java src/main/java/myapps/LineSplit.java
      +
      > cp src/main/java/myapps/Pipe.java src/main/java/myapps/LineSplit.java

      And change its class name as well as the application id config to distinguish with the original program:

      -
              public class LineSplit {
      +    
      public class LineSplit {
       
      -            public static void main(String[] args) throws Exception {
      -                Properties props = new Properties();
      -                props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-linesplit");
      -                // ...
      -            }
      -        }
      + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-linesplit"); + // ... + } +}

      Since each of the source stream's record is a String typed key-value pair, let's treat the value string as a text line and split it into words with a FlatMapValues operator:

      -
              KStream<String, String> source = builder.stream("streams-plaintext-input");
      -        KStream<String, String> words = source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
      -                    @Override
      -                    public Iterable<String> apply(String value) {
      -                        return Arrays.asList(value.split("\\W+"));
      -                    }
      -                });
      +
      KStream<String, String> source = builder.stream("streams-plaintext-input");
      +KStream<String, String> words = source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
      +            @Override
      +            public Iterable<String> apply(String value) {
      +                return Arrays.asList(value.split("\\W+"));
      +            }
      +        });

      The operator will take the source stream as its input, and generate a new stream named words @@ -327,31 +326,31 @@

      Writing a se Note if you are using JDK 8 you can use lambda expression and simplify the above code as:

      -
              KStream<String, String> source = builder.stream("streams-plaintext-input");
      -        KStream<String, String> words = source.flatMapValues(value -> Arrays.asList(value.split("\\W+")));
      +
      KStream<String, String> source = builder.stream("streams-plaintext-input");
      +KStream<String, String> words = source.flatMapValues(value -> Arrays.asList(value.split("\\W+")));

      And finally we can write the word stream back into another Kafka topic, say streams-linesplit-output. Again, these two steps can be concatenated as the following (assuming lambda expression is used):

      -
              KStream<String, String> source = builder.stream("streams-plaintext-input");
      -        source.flatMapValues(value -> Arrays.asList(value.split("\\W+")))
      -              .to("streams-linesplit-output");
      +
      KStream<String, String> source = builder.stream("streams-plaintext-input");
      +source.flatMapValues(value -> Arrays.asList(value.split("\\W+")))
      +      .to("streams-linesplit-output");

      If we now describe this augmented topology as System.out.println(topology.describe()), we will get the following:

      -
              > mvn clean package
      -        > mvn exec:java -Dexec.mainClass=myapps.LineSplit
      -        Sub-topologies:
      -          Sub-topology: 0
      -            Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-FLATMAPVALUES-0000000001
      -            Processor: KSTREAM-FLATMAPVALUES-0000000001(stores: []) --> KSTREAM-SINK-0000000002 <-- KSTREAM-SOURCE-0000000000
      -            Sink: KSTREAM-SINK-0000000002(topic: streams-linesplit-output) <-- KSTREAM-FLATMAPVALUES-0000000001
      -          Global Stores:
      -            none
      +
      > mvn clean package
      +> mvn exec:java -Dexec.mainClass=myapps.LineSplit
      +Sub-topologies:
      +  Sub-topology: 0
      +    Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-FLATMAPVALUES-0000000001
      +    Processor: KSTREAM-FLATMAPVALUES-0000000001(stores: []) --> KSTREAM-SINK-0000000002 <-- KSTREAM-SOURCE-0000000000
      +    Sink: KSTREAM-SINK-0000000002(topic: streams-linesplit-output) <-- KSTREAM-FLATMAPVALUES-0000000001
      +  Global Stores:
      +    none

      As we can see above, a new processor node KSTREAM-FLATMAPVALUES-0000000001 is injected into the topology between the original source and sink nodes. @@ -365,41 +364,41 @@

      Writing a se The complete code looks like this (assuming lambda expression is used):

      -
              package myapps;
      +    
      package myapps;
       
      -        import org.apache.kafka.common.serialization.Serdes;
      -        import org.apache.kafka.streams.KafkaStreams;
      -        import org.apache.kafka.streams.StreamsBuilder;
      -        import org.apache.kafka.streams.StreamsConfig;
      -        import org.apache.kafka.streams.Topology;
      -        import org.apache.kafka.streams.kstream.KStream;
      +import org.apache.kafka.common.serialization.Serdes;
      +import org.apache.kafka.streams.KafkaStreams;
      +import org.apache.kafka.streams.StreamsBuilder;
      +import org.apache.kafka.streams.StreamsConfig;
      +import org.apache.kafka.streams.Topology;
      +import org.apache.kafka.streams.kstream.KStream;
       
      -        import java.util.Arrays;
      -        import java.util.Properties;
      -        import java.util.concurrent.CountDownLatch;
      +import java.util.Arrays;
      +import java.util.Properties;
      +import java.util.concurrent.CountDownLatch;
       
      -        public class LineSplit {
      +public class LineSplit {
       
      -            public static void main(String[] args) throws Exception {
      -                Properties props = new Properties();
      -                props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-linesplit");
      -                props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
      -                props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      -                props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +    public static void main(String[] args) throws Exception {
      +        Properties props = new Properties();
      +        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-linesplit");
      +        props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
      +        props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +        props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
       
      -                final StreamsBuilder builder = new StreamsBuilder();
      +        final StreamsBuilder builder = new StreamsBuilder();
       
      -                KStream<String, String> source = builder.stream("streams-plaintext-input");
      -                source.flatMapValues(value -> Arrays.asList(value.split("\\W+")))
      -                      .to("streams-linesplit-output");
      +        KStream<String, String> source = builder.stream("streams-plaintext-input");
      +        source.flatMapValues(value -> Arrays.asList(value.split("\\W+")))
      +              .to("streams-linesplit-output");
       
      -                final Topology topology = builder.build();
      -                final KafkaStreams streams = new KafkaStreams(topology, props);
      -                final CountDownLatch latch = new CountDownLatch(1);
      +        final Topology topology = builder.build();
      +        final KafkaStreams streams = new KafkaStreams(topology, props);
      +        final CountDownLatch latch = new CountDownLatch(1);
       
      -                // ... same as Pipe.java above
      -            }
      -        }
      + // ... same as Pipe.java above + } +}

      Writing a third Streams application: Wordcount

      @@ -408,47 +407,47 @@

      Writing a th Following similar steps let's create another program based on the LineSplit.java class:

      -
              public class WordCount {
      +    
      public class WordCount {
       
      -            public static void main(String[] args) throws Exception {
      -                Properties props = new Properties();
      -                props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount");
      -                // ...
      -            }
      -        }
      + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); + // ... + } +}

      In order to count the words we can first modify the flatMapValues operator to treat all of them as lower case (assuming lambda expression is used):

      -
              source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
      -                    @Override
      -                    public Iterable<String> apply(String value) {
      -                        return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"));
      -                    }
      -                });
      +
      source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
      +    @Override
      +    public Iterable<String> apply(String value) {
      +        return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"));
      +    }
      +});

      In order to do the counting aggregation we have to first specify that we want to key the stream on the value string, i.e. the lower cased word, with a groupBy operator. This operator generate a new grouped stream, which can then be aggregated by a count operator, which generates a running count on each of the grouped keys:

      -
              KTable<String, Long> counts =
      -        source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
      -                    @Override
      -                    public Iterable<String> apply(String value) {
      -                        return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"));
      -                    }
      -                })
      -              .groupBy(new KeyValueMapper<String, String, String>() {
      -                   @Override
      -                   public String apply(String key, String value) {
      -                       return value;
      -                   }
      -                })
      -              // Materialize the result into a KeyValueStore named "counts-store".
      -              // The Materialized store is always of type <Bytes, byte[]> as this is the format of the inner most store.
      -              .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>> as("counts-store"));
      +
      KTable<String, Long> counts =
      +source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
      +            @Override
      +            public Iterable<String> apply(String value) {
      +                return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"));
      +            }
      +        })
      +      .groupBy(new KeyValueMapper<String, String, String>() {
      +           @Override
      +           public String apply(String key, String value) {
      +               return value;
      +           }
      +        })
      +      // Materialize the result into a KeyValueStore named "counts-store".
      +      // The Materialized store is always of type <Bytes, byte[]> as this is the format of the inner most store.
      +      .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>> as("counts-store"));

      Note that the count operator has a Materialized parameter that specifies that the @@ -463,7 +462,7 @@

      Writing a th We need to provide overridden serialization methods for Long types, otherwise a runtime exception will be thrown:

      -
              counts.toStream().to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));
      +
      counts.toStream().to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));

      Note that in order to read the changelog stream from topic streams-wordcount-output, @@ -472,33 +471,33 @@

      Writing a th Assuming lambda expression from JDK 8 can be used, the above code can be simplified as:

      -
              KStream<String, String> source = builder.stream("streams-plaintext-input");
      -        source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")))
      -              .groupBy((key, value) -> value)
      -              .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"))
      -              .toStream()
      -              .to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));
      +
      KStream<String, String> source = builder.stream("streams-plaintext-input");
      +source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")))
      +      .groupBy((key, value) -> value)
      +      .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"))
      +      .toStream()
      +      .to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));

      If we again describe this augmented topology as System.out.println(topology.describe()), we will get the following:

      -
              > mvn clean package
      -        > mvn exec:java -Dexec.mainClass=myapps.WordCount
      -        Sub-topologies:
      -          Sub-topology: 0
      -            Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-FLATMAPVALUES-0000000001
      -            Processor: KSTREAM-FLATMAPVALUES-0000000001(stores: []) --> KSTREAM-KEY-SELECT-0000000002 <-- KSTREAM-SOURCE-0000000000
      -            Processor: KSTREAM-KEY-SELECT-0000000002(stores: []) --> KSTREAM-FILTER-0000000005 <-- KSTREAM-FLATMAPVALUES-0000000001
      -            Processor: KSTREAM-FILTER-0000000005(stores: []) --> KSTREAM-SINK-0000000004 <-- KSTREAM-KEY-SELECT-0000000002
      -            Sink: KSTREAM-SINK-0000000004(topic: Counts-repartition) <-- KSTREAM-FILTER-0000000005
      -          Sub-topology: 1
      -            Source: KSTREAM-SOURCE-0000000006(topics: Counts-repartition) --> KSTREAM-AGGREGATE-0000000003
      -            Processor: KSTREAM-AGGREGATE-0000000003(stores: [Counts]) --> KTABLE-TOSTREAM-0000000007 <-- KSTREAM-SOURCE-0000000006
      -            Processor: KTABLE-TOSTREAM-0000000007(stores: []) --> KSTREAM-SINK-0000000008 <-- KSTREAM-AGGREGATE-0000000003
      -            Sink: KSTREAM-SINK-0000000008(topic: streams-wordcount-output) <-- KTABLE-TOSTREAM-0000000007
      -        Global Stores:
      -          none
      +
      > mvn clean package
      +> mvn exec:java -Dexec.mainClass=myapps.WordCount
      +Sub-topologies:
      +  Sub-topology: 0
      +    Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-FLATMAPVALUES-0000000001
      +    Processor: KSTREAM-FLATMAPVALUES-0000000001(stores: []) --> KSTREAM-KEY-SELECT-0000000002 <-- KSTREAM-SOURCE-0000000000
      +    Processor: KSTREAM-KEY-SELECT-0000000002(stores: []) --> KSTREAM-FILTER-0000000005 <-- KSTREAM-FLATMAPVALUES-0000000001
      +    Processor: KSTREAM-FILTER-0000000005(stores: []) --> KSTREAM-SINK-0000000004 <-- KSTREAM-KEY-SELECT-0000000002
      +    Sink: KSTREAM-SINK-0000000004(topic: Counts-repartition) <-- KSTREAM-FILTER-0000000005
      +  Sub-topology: 1
      +    Source: KSTREAM-SOURCE-0000000006(topics: Counts-repartition) --> KSTREAM-AGGREGATE-0000000003
      +    Processor: KSTREAM-AGGREGATE-0000000003(stores: [Counts]) --> KTABLE-TOSTREAM-0000000007 <-- KSTREAM-SOURCE-0000000006
      +    Processor: KTABLE-TOSTREAM-0000000007(stores: []) --> KSTREAM-SINK-0000000008 <-- KSTREAM-AGGREGATE-0000000003
      +    Sink: KSTREAM-SINK-0000000008(topic: streams-wordcount-output) <-- KTABLE-TOSTREAM-0000000007
      +Global Stores:
      +  none

      As we can see above, the topology now contains two disconnected sub-topologies. @@ -517,49 +516,49 @@

      Writing a th The complete code looks like this (assuming lambda expression is used):

      -
              package myapps;
      +    
      package myapps;
       
      -        import org.apache.kafka.common.serialization.Serdes;
      -        import org.apache.kafka.common.utils.Bytes;
      -        import org.apache.kafka.streams.KafkaStreams;
      -        import org.apache.kafka.streams.StreamsBuilder;
      -        import org.apache.kafka.streams.StreamsConfig;
      -        import org.apache.kafka.streams.Topology;
      -        import org.apache.kafka.streams.kstream.KStream;
      -        import org.apache.kafka.streams.kstream.Materialized;
      -        import org.apache.kafka.streams.kstream.Produced;
      -        import org.apache.kafka.streams.state.KeyValueStore;
      +import org.apache.kafka.common.serialization.Serdes;
      +import org.apache.kafka.common.utils.Bytes;
      +import org.apache.kafka.streams.KafkaStreams;
      +import org.apache.kafka.streams.StreamsBuilder;
      +import org.apache.kafka.streams.StreamsConfig;
      +import org.apache.kafka.streams.Topology;
      +import org.apache.kafka.streams.kstream.KStream;
      +import org.apache.kafka.streams.kstream.Materialized;
      +import org.apache.kafka.streams.kstream.Produced;
      +import org.apache.kafka.streams.state.KeyValueStore;
       
      -        import java.util.Arrays;
      -        import java.util.Locale;
      -        import java.util.Properties;
      -        import java.util.concurrent.CountDownLatch;
      +import java.util.Arrays;
      +import java.util.Locale;
      +import java.util.Properties;
      +import java.util.concurrent.CountDownLatch;
       
      -        public class WordCount {
      +public class WordCount {
       
      -            public static void main(String[] args) throws Exception {
      -                Properties props = new Properties();
      -                props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount");
      -                props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
      -                props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      -                props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +    public static void main(String[] args) throws Exception {
      +        Properties props = new Properties();
      +        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount");
      +        props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
      +        props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
      +        props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
       
      -                final StreamsBuilder builder = new StreamsBuilder();
      +        final StreamsBuilder builder = new StreamsBuilder();
       
      -                KStream<String, String> source = builder.stream("streams-plaintext-input");
      -                source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")))
      -                      .groupBy((key, value) -> value)
      -                      .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"))
      -                      .toStream()
      -                      .to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));
      +        KStream<String, String> source = builder.stream("streams-plaintext-input");
      +        source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")))
      +              .groupBy((key, value) -> value)
      +              .count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"))
      +              .toStream()
      +              .to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));
       
      -                final Topology topology = builder.build();
      -                final KafkaStreams streams = new KafkaStreams(topology, props);
      -                final CountDownLatch latch = new CountDownLatch(1);
      +        final Topology topology = builder.build();
      +        final KafkaStreams streams = new KafkaStreams(topology, props);
      +        final CountDownLatch latch = new CountDownLatch(1);
       
      -                // ... same as Pipe.java above
      -            }
      -        }
      + // ... same as Pipe.java above + } +}

      In-memory @@ -268,19 +266,18 @@

    5. Use TimestampedWindowStore when you need to store windowedKey-(value/timestamp) pairs.
    6. -
      // Creating an in-memory key-value store:
      -// here, we create a `KeyValueStore<String, Long>` named "inmemory-counts".
      -import org.apache.kafka.streams.state.StoreBuilder;
      -import org.apache.kafka.streams.state.Stores;
      +                            
      // Creating an in-memory key-value store:
      +// here, we create a `KeyValueStore<String, Long>` named "inmemory-counts".
      +import org.apache.kafka.streams.state.StoreBuilder;
      +import org.apache.kafka.streams.state.Stores;
       
      -// Using a `KeyValueStoreBuilder` to build a `KeyValueStore`.
      -StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier =
      -  Stores.keyValueStoreBuilder(
      -    Stores.inMemoryKeyValueStore("inmemory-counts"),
      -    Serdes.String(),
      -    Serdes.Long());
      -KeyValueStore<String, Long> countStore = countStoreSupplier.build();
      -
      +// Using a `KeyValueStoreBuilder` to build a `KeyValueStore`. +StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = + Stores.keyValueStoreBuilder( + Stores.inMemoryKeyValueStore("inmemory-counts"), + Serdes.String(), + Serdes.Long()); +KeyValueStore<String, Long> countStore = countStoreSupplier.build();