diff --git a/.gitignore b/.gitignore index 23da234d8bb5..6a08f2cc39c6 100644 --- a/.gitignore +++ b/.gitignore @@ -53,6 +53,7 @@ kafkatest.egg-info/ systest/ *.swp clients/src/generated +clients/src/generated-test support-metrics-client/src/main/generated support-metrics-client/src/test/generated support-metrics-common/src/main/generated diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 884ef54a0459..a47b1daed868 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.12.9 + SCALA_VERSION=2.12.10 fi if [ -z "$SCALA_BINARY_VERSION" ]; then diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index 782985a2ae14..52e1bab87005 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.12.9 + set SCALA_VERSION=2.12.10 ) IF ["%SCALA_BINARY_VERSION%"] EQU [""] ( diff --git a/build.gradle b/build.gradle index d34baae6b70e..299cf49a5891 100644 --- a/build.gradle +++ b/build.gradle @@ -1038,11 +1038,23 @@ project(':clients') { task processMessages(type:JavaExec) { main = "org.apache.kafka.message.MessageGenerator" classpath = project(':generator').sourceSets.main.runtimeClasspath - args = [ "src/generated/java/org/apache/kafka/common/message", "src/main/resources/common/message" ] + args = [ "org.apache.kafka.common.message", + "src/generated/java/org/apache/kafka/common/message", + "src/main/resources/common/message" ] inputs.dir("src/main/resources/common/message") outputs.dir("src/generated/java/org/apache/kafka/common/message") } + task processTestMessages(type:JavaExec) { + main = "org.apache.kafka.message.MessageGenerator" + classpath = project(':generator').sourceSets.main.runtimeClasspath + args = [ "org.apache.kafka.common.message", + "src/generated-test/java/org/apache/kafka/common/message", + "src/test/resources/common/message" ] + inputs.dir("src/test/resources/common/message") + outputs.dir("src/generated-test/java/org/apache/kafka/common/message") + } + sourceSets { main { java { @@ -1051,13 +1063,15 @@ project(':clients') { } test { java { - srcDirs = ["src/generated/java", "src/test/java"] + srcDirs = ["src/generated/java", "src/generated-test/java", "src/test/java"] } } } compileJava.dependsOn 'processMessages' + compileTestJava.dependsOn 'processTestMessages' + javadoc { include "**/org/apache/kafka/clients/admin/*" include "**/org/apache/kafka/clients/consumer/*" @@ -1135,6 +1149,7 @@ project(':tools') { project(':streams') { archivesBaseName = "kafka-streams" + ext.buildStreamsVersionFileName = "kafka-streams-version.properties" dependencies { compile project(':clients') @@ -1182,7 +1197,46 @@ project(':streams') { duplicatesStrategy 'exclude' } + task determineCommitId { + def takeFromHash = 16 + if (commitId) { + commitId = commitId.take(takeFromHash) + } else if (file("$rootDir/.git/HEAD").exists()) { + def headRef = file("$rootDir/.git/HEAD").text + if (headRef.contains('ref: ')) { + headRef = headRef.replaceAll('ref: ', '').trim() + if (file("$rootDir/.git/$headRef").exists()) { + commitId = file("$rootDir/.git/$headRef").text.trim().take(takeFromHash) + } + } else { + commitId = headRef.trim().take(takeFromHash) + } + } else { + commitId = "unknown" + } + } + + task createStreamsVersionFile(dependsOn: determineCommitId) { + ext.receiptFile = file("$buildDir/kafka/$buildStreamsVersionFileName") + outputs.file receiptFile + outputs.upToDateWhen { false } + doLast { + def data = [ + commitId: commitId, + version: version, + ] + + receiptFile.parentFile.mkdirs() + def content = data.entrySet().collect { "$it.key=$it.value" }.sort().join("\n") + receiptFile.setText(content, "ISO-8859-1") + } + } + jar { + dependsOn 'createStreamsVersionFile' + from("$buildDir") { + include "kafka/$buildStreamsVersionFileName" + } dependsOn 'copyDependantLibs' } @@ -1596,9 +1650,11 @@ project(':jmh-benchmarks') { } dependencies { + compile project(':core') compile project(':clients') compile project(':streams') compile libs.jmhCore + compile libs.mockitoCore annotationProcessor libs.jmhGeneratorAnnProcess compile libs.jmhCoreBenchmarks } @@ -1609,6 +1665,9 @@ project(':jmh-benchmarks') { } } + checkstyle { + configProperties = checkstyleConfigProperties("import-control-jmh-benchmarks.xml") + } task jmh(type: JavaExec, dependsOn: [':jmh-benchmarks:clean', ':jmh-benchmarks:shadowJar']) { diff --git a/checkstyle/import-control-jmh-benchmarks.xml b/checkstyle/import-control-jmh-benchmarks.xml new file mode 100644 index 000000000000..49bd8c786920 --- /dev/null +++ b/checkstyle/import-control-jmh-benchmarks.xml @@ -0,0 +1,45 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 44b06a7a381c..066fc33239a9 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -118,6 +118,7 @@ + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index fb2301f6ace7..a5b996d9cc27 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -8,6 +8,14 @@ + + + + + @@ -49,7 +57,7 @@ files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData|JoinGroupRequest).java"/> + files="(ConsumerCoordinator|Fetcher|Sender|KafkaProducer|BufferPool|ConfigDef|RecordAccumulator|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator).java"/> @@ -60,6 +68,12 @@ + + + + @@ -127,6 +141,9 @@ + + @@ -155,6 +172,9 @@ + + @@ -193,6 +213,10 @@ + + + @@ -226,13 +250,6 @@ - - - - - diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index 7b44ca3ad3c4..a160102db1dd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients; +import org.apache.kafka.common.message.RequestHeaderData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.RequestHeader; @@ -82,7 +83,14 @@ public ApiKeys apiKey() { } public RequestHeader makeHeader(short version) { - return new RequestHeader(apiKey(), version, clientId, correlationId); + short requestApiKey = requestBuilder.apiKey().id; + return new RequestHeader( + new RequestHeaderData(). + setRequestApiKey(requestApiKey). + setRequestApiVersion(version). + setClientId(clientId). + setCorrelationId(correlationId), + ApiKeys.forId(requestApiKey).headerVersion(version)); } public AbstractRequest.Builder requestBuilder() { diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 54c3891ec26b..af47e55d75a6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -42,9 +42,9 @@ public class CommonClientConfigs { + "servers (you may want more than one, though, in case a server is down)."; public static final String CLIENT_DNS_LOOKUP_CONFIG = "client.dns.lookup"; - public static final String CLIENT_DNS_LOOKUP_DOC = "

Controls how the client uses DNS lookups.

If set to use_all_dns_ips then, when the lookup returns multiple IP addresses for a hostname," - + " they will all be attempted to connect to before failing the connection. Applies to both bootstrap and advertised servers.

" - + "

If the value is resolve_canonical_bootstrap_servers_only each entry will be resolved and expanded into a list of canonical names.

"; + public static final String CLIENT_DNS_LOOKUP_DOC = "Controls how the client uses DNS lookups. If set to use_all_dns_ips then, when the lookup returns multiple IP addresses for a hostname," + + " they will all be attempted to connect to before failing the connection. Applies to both bootstrap and advertised servers." + + " If the value is resolve_canonical_bootstrap_servers_only each entry will be resolved and expanded into a list of canonical names."; public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions."; diff --git a/clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java b/clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java index 494bbbe4769e..0dc8943fdcbe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java @@ -397,14 +397,15 @@ public boolean handleResponse(FetchResponse response) { nextMetadata = FetchMetadata.INITIAL; return false; } else if (response.sessionId() == INVALID_SESSION_ID) { - log.debug("Node {} sent a full fetch response{}", - node, responseDataToLogString(response)); + if (log.isDebugEnabled()) + log.debug("Node {} sent a full fetch response{}", node, responseDataToLogString(response)); nextMetadata = FetchMetadata.INITIAL; return true; } else { // The server created a new incremental fetch session. - log.debug("Node {} sent a full fetch response that created a new incremental " + - "fetch session {}{}", node, response.sessionId(), responseDataToLogString(response)); + if (log.isDebugEnabled()) + log.debug("Node {} sent a full fetch response that created a new incremental " + + "fetch session {}{}", node, response.sessionId(), responseDataToLogString(response)); nextMetadata = FetchMetadata.newIncremental(response.sessionId()); return true; } @@ -416,14 +417,16 @@ public boolean handleResponse(FetchResponse response) { return false; } else if (response.sessionId() == INVALID_SESSION_ID) { // The incremental fetch session was closed by the server. - log.debug("Node {} sent an incremental fetch response closing session {}{}", - node, nextMetadata.sessionId(), responseDataToLogString(response)); + if (log.isDebugEnabled()) + log.debug("Node {} sent an incremental fetch response closing session {}{}", + node, nextMetadata.sessionId(), responseDataToLogString(response)); nextMetadata = FetchMetadata.INITIAL; return true; } else { // The incremental fetch session was continued by the server. - log.debug("Node {} sent an incremental fetch response for session {}{}", - node, response.sessionId(), responseDataToLogString(response)); + if (log.isDebugEnabled()) + log.debug("Node {} sent an incremental fetch response for session {}{}", + node, response.sessionId(), responseDataToLogString(response)); nextMetadata = nextMetadata.nextIncremental(); return true; } 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 047b03585c01..44c9fd3c0724 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -705,7 +705,7 @@ public static AbstractResponse parseResponse(ByteBuffer responseBuffer, RequestH private static Struct parseStructMaybeUpdateThrottleTimeMetrics(ByteBuffer responseBuffer, RequestHeader requestHeader, Sensor throttleTimeSensor, long now) { - ResponseHeader responseHeader = ResponseHeader.parse(responseBuffer); + ResponseHeader responseHeader = ResponseHeader.parse(responseBuffer, requestHeader.headerVersion()); // Always expect the response version id to be the same as the request version id Struct responseBody = requestHeader.apiKey().parseResponse(requestHeader.apiVersion(), responseBuffer); correlate(requestHeader, responseHeader); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 48630d46bb16..915bd728e18c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -837,6 +837,29 @@ default DeleteConsumerGroupsResult deleteConsumerGroups(Collection group return deleteConsumerGroups(groupIds, new DeleteConsumerGroupsOptions()); } + /** + * Delete committed offsets for a set of partitions in a consumer group. This will + * succeed at the partition level only if the group is not actively subscribed + * to the corresponding topic. + * + * @param options The options to use when deleting offsets in a consumer group. + * @return The DeleteConsumerGroupOffsetsResult. + */ + DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsets(String groupId, + Set partitions, + DeleteConsumerGroupOffsetsOptions options); + + /** + * Delete committed offsets for a set of partitions in a consumer group with the default + * options. This will succeed at the partition level only if the group is not actively + * subscribed to the corresponding topic. + * + * @return The DeleteConsumerGroupOffsetsResult. + */ + default DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsets(String groupId, Set partitions) { + return deleteConsumerGroupOffsets(groupId, partitions, new DeleteConsumerGroupOffsetsOptions()); + } + /** * Elect the preferred replica as leader for topic partitions. *

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java index 35a781e1deb5..107eb56d63ae 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java @@ -217,7 +217,7 @@ public static ConfigDef configDef() { } public static void main(String[] args) { - System.out.println(CONFIG.toHtmlTable()); + System.out.println(CONFIG.toHtml()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/CreateTopicsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/CreateTopicsResult.java index 4fada35c790c..85c0a905240b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/CreateTopicsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/CreateTopicsResult.java @@ -18,9 +18,11 @@ import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.errors.ApiException; import java.util.Collection; import java.util.Map; +import java.util.stream.Collectors; /** * The result of {@link Admin#createTopics(Collection)}. @@ -29,9 +31,11 @@ */ @InterfaceStability.Evolving public class CreateTopicsResult { - private final Map> futures; + final static int UNKNOWN = -1; - CreateTopicsResult(Map> futures) { + private final Map> futures; + + CreateTopicsResult(Map> futures) { this.futures = futures; } @@ -40,7 +44,8 @@ public class CreateTopicsResult { * topic creations. */ public Map> values() { - return futures; + return futures.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().thenApply(v -> (Void) null))); } /** @@ -49,4 +54,84 @@ public Map> values() { public KafkaFuture all() { return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])); } + + /** + * Returns a future that provides topic configs for the topic when the request completes. + *

+ * If broker version doesn't support replication factor in the response, throw + * {@link org.apache.kafka.common.errors.UnsupportedVersionException}. + * If broker returned an error for topic configs, throw appropriate exception. For example, + * {@link org.apache.kafka.common.errors.TopicAuthorizationException} is thrown if user does not + * have permission to describe topic configs. + */ + public KafkaFuture config(String topic) { + return futures.get(topic).thenApply(TopicMetadataAndConfig::config); + } + + /** + * Returns a future that provides number of partitions in the topic when the request completes. + *

+ * If broker version doesn't support replication factor in the response, throw + * {@link org.apache.kafka.common.errors.UnsupportedVersionException}. + * If broker returned an error for topic configs, throw appropriate exception. For example, + * {@link org.apache.kafka.common.errors.TopicAuthorizationException} is thrown if user does not + * have permission to describe topic configs. + */ + public KafkaFuture numPartitions(String topic) { + return futures.get(topic).thenApply(TopicMetadataAndConfig::numPartitions); + } + + /** + * Returns a future that provides replication factor for the topic when the request completes. + *

+ * If broker version doesn't support replication factor in the response, throw + * {@link org.apache.kafka.common.errors.UnsupportedVersionException}. + * If broker returned an error for topic configs, throw appropriate exception. For example, + * {@link org.apache.kafka.common.errors.TopicAuthorizationException} is thrown if user does not + * have permission to describe topic configs. + */ + public KafkaFuture replicationFactor(String topic) { + return futures.get(topic).thenApply(TopicMetadataAndConfig::replicationFactor); + } + + static class TopicMetadataAndConfig { + private final ApiException exception; + private final int numPartitions; + private final int replicationFactor; + private final Config config; + + TopicMetadataAndConfig(int numPartitions, int replicationFactor, Config config) { + this.exception = null; + this.numPartitions = numPartitions; + this.replicationFactor = replicationFactor; + this.config = config; + } + + TopicMetadataAndConfig(ApiException exception) { + this.exception = exception; + this.numPartitions = UNKNOWN; + this.replicationFactor = UNKNOWN; + this.config = null; + } + + public int numPartitions() { + ensureSuccess(); + return numPartitions; + } + + public int replicationFactor() { + ensureSuccess(); + return replicationFactor; + } + + public Config config() { + ensureSuccess(); + return config; + } + + private void ensureSuccess() { + if (exception != null) + throw exception; + } + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsOptions.java new file mode 100644 index 000000000000..63e6b4be84ba --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsOptions.java @@ -0,0 +1,30 @@ +/* + * 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.clients.admin; + +import java.util.Set; +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * Options for the {@link Admin#deleteConsumerGroupOffsets(String, Set)} call. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class DeleteConsumerGroupOffsetsOptions extends AbstractOptions { + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsResult.java new file mode 100644 index 000000000000..433f478c3862 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsResult.java @@ -0,0 +1,84 @@ +/* + * 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.clients.admin; + +import java.util.Set; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.KafkaFuture.BaseFunction; +import org.apache.kafka.common.KafkaFuture.BiConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Map; +import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.protocol.Errors; + +/** + * The result of the {@link Admin#deleteConsumerGroupOffsets(String, Set)} call. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class DeleteConsumerGroupOffsetsResult { + private final KafkaFuture> future; + + DeleteConsumerGroupOffsetsResult(KafkaFuture> future) { + this.future = future; + } + + /** + * Return a future which can be used to check the result for a given partition. + */ + public KafkaFuture partitionResult(final TopicPartition partition) { + final KafkaFutureImpl result = new KafkaFutureImpl<>(); + + this.future.whenComplete(new BiConsumer, Throwable>() { + @Override + public void accept(final Map topicPartitions, final Throwable throwable) { + if (throwable != null) { + result.completeExceptionally(throwable); + } else if (!topicPartitions.containsKey(partition)) { + result.completeExceptionally(new IllegalArgumentException( + "Group offset deletion for partition \"" + partition + + "\" was not attempted")); + } else { + final Errors error = topicPartitions.get(partition); + if (error == Errors.NONE) { + result.complete(null); + } else { + result.completeExceptionally(error.exception()); + } + } + + } + }); + + return result; + } + + /** + * Return a future which succeeds only if all the deletions succeed. + */ + public KafkaFuture all() { + return this.future.thenApply(new BaseFunction, Void>() { + @Override + public Void apply(final Map topicPartitionErrorsMap) { + return null; + } + }); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index c7ea97005a51..a7663da3789d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -25,6 +25,7 @@ import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.StaleMetadataException; +import org.apache.kafka.clients.admin.CreateTopicsResult.TopicMetadataAndConfig; import org.apache.kafka.clients.admin.DeleteAclsResult.FilterResult; import org.apache.kafka.clients.admin.DeleteAclsResult.FilterResults; import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo; @@ -68,6 +69,7 @@ import org.apache.kafka.common.message.CreateDelegationTokenResponseData; import org.apache.kafka.common.message.CreateTopicsRequestData; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection; +import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicConfigs; import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; import org.apache.kafka.common.message.DeleteGroupsRequestData; import org.apache.kafka.common.message.DeleteTopicsRequestData; @@ -85,6 +87,10 @@ import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.MetadataRequestData; +import org.apache.kafka.common.message.OffsetDeleteRequestData; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestPartition; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopic; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; @@ -151,6 +157,9 @@ import org.apache.kafka.common.requests.ListPartitionReassignmentsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.OffsetDeleteRequest; +import org.apache.kafka.common.requests.OffsetDeleteRequest.Builder; +import org.apache.kafka.common.requests.OffsetDeleteResponse; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.RenewDelegationTokenRequest; @@ -1302,11 +1311,11 @@ int numPendingCalls() { @Override public CreateTopicsResult createTopics(final Collection newTopics, final CreateTopicsOptions options) { - final Map> topicFutures = new HashMap<>(newTopics.size()); + final Map> topicFutures = new HashMap<>(newTopics.size()); final CreatableTopicCollection topics = new CreatableTopicCollection(); for (NewTopic newTopic : newTopics) { if (topicNameIsUnrepresentable(newTopic.name())) { - KafkaFutureImpl future = new KafkaFutureImpl<>(); + KafkaFutureImpl future = new KafkaFutureImpl<>(); future.completeExceptionally(new InvalidTopicException("The given topic name '" + newTopic.name() + "' cannot be represented in a request.")); topicFutures.put(newTopic.name(), future); @@ -1341,7 +1350,7 @@ public void handleResponse(AbstractResponse abstractResponse) { } // Handle server responses for particular topics. for (CreatableTopicResult result : response.data().topics()) { - KafkaFutureImpl future = topicFutures.get(result.name()); + KafkaFutureImpl future = topicFutures.get(result.name()); if (future == null) { log.warn("Server response mentioned unknown topic {}", result.name()); } else { @@ -1351,13 +1360,33 @@ public void handleResponse(AbstractResponse abstractResponse) { if (exception != null) { future.completeExceptionally(exception); } else { - future.complete(null); + TopicMetadataAndConfig topicMetadataAndConfig; + if (result.topicConfigErrorCode() != Errors.NONE.code()) { + topicMetadataAndConfig = new TopicMetadataAndConfig(Errors.forCode(result.topicConfigErrorCode()).exception()); + } else if (result.numPartitions() == CreateTopicsResult.UNKNOWN) { + topicMetadataAndConfig = new TopicMetadataAndConfig(new UnsupportedVersionException( + "Topic metadata and configs in CreateTopics response not supported")); + } else { + List configs = result.configs(); + Config topicConfig = new Config(configs.stream() + .map(config -> new ConfigEntry(config.name(), + config.value(), + configSource(DescribeConfigsResponse.ConfigSource.forId(config.configSource())), + config.isSensitive(), + config.readOnly(), + Collections.emptyList())) + .collect(Collectors.toSet())); + topicMetadataAndConfig = new TopicMetadataAndConfig(result.numPartitions(), + result.replicationFactor(), + topicConfig); + } + future.complete(topicMetadataAndConfig); } } } // The server should send back a response for every topic. But do a sanity check anyway. - for (Map.Entry> entry : topicFutures.entrySet()) { - KafkaFutureImpl future = entry.getValue(); + for (Map.Entry> entry : topicFutures.entrySet()) { + KafkaFutureImpl future = entry.getValue(); if (!future.isDone()) { future.completeExceptionally(new ApiException("The server response did not " + "contain a reference to node " + entry.getKey())); @@ -3055,6 +3084,92 @@ void handleFailure(Throwable throwable) { }; } + @Override + public DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsets( + String groupId, + Set partitions, + DeleteConsumerGroupOffsetsOptions options) { + final KafkaFutureImpl> future = new KafkaFutureImpl<>(); + + if (groupIdIsUnrepresentable(groupId)) { + future.completeExceptionally(new InvalidGroupIdException("The given group id '" + + groupId + "' cannot be represented in a request.")); + return new DeleteConsumerGroupOffsetsResult(future); + } + + final long startFindCoordinatorMs = time.milliseconds(); + final long deadline = calcDeadlineMs(startFindCoordinatorMs, options.timeoutMs()); + ConsumerGroupOperationContext, DeleteConsumerGroupOffsetsOptions> context = + new ConsumerGroupOperationContext<>(groupId, options, deadline, future); + + Call findCoordinatorCall = getFindCoordinatorCall(context, + () -> KafkaAdminClient.this.getDeleteConsumerGroupOffsetsCall(context, partitions)); + runnable.call(findCoordinatorCall, startFindCoordinatorMs); + + return new DeleteConsumerGroupOffsetsResult(future); + } + + private Call getDeleteConsumerGroupOffsetsCall( + ConsumerGroupOperationContext, DeleteConsumerGroupOffsetsOptions> context, + Set partitions) { + return new Call("deleteConsumerGroupOffsets", context.getDeadline(), new ConstantNodeIdProvider(context.getNode().get().id())) { + + @Override + AbstractRequest.Builder createRequest(int timeoutMs) { + final OffsetDeleteRequestTopicCollection topics = new OffsetDeleteRequestTopicCollection(); + + partitions.stream().collect(Collectors.groupingBy(TopicPartition::topic)).forEach((topic, topicPartitions) -> { + topics.add( + new OffsetDeleteRequestTopic() + .setName(topic) + .setPartitions(topicPartitions.stream() + .map(tp -> new OffsetDeleteRequestPartition().setPartitionIndex(tp.partition())) + .collect(Collectors.toList()) + ) + ); + }); + + return new OffsetDeleteRequest.Builder( + new OffsetDeleteRequestData() + .setGroupId(context.groupId) + .setTopics(topics) + ); + } + + @Override + void handleResponse(AbstractResponse abstractResponse) { + final OffsetDeleteResponse response = (OffsetDeleteResponse) abstractResponse; + + // If coordinator changed since we fetched it, retry + if (context.hasCoordinatorMoved(response)) { + rescheduleTask(context, () -> getDeleteConsumerGroupOffsetsCall(context, partitions)); + return; + } + + // If the error is an error at the group level, the future is failed with it + final Errors groupError = Errors.forCode(response.data.errorCode()); + if (handleGroupRequestError(groupError, context.getFuture())) + return; + + final Map partitions = new HashMap<>(); + response.data.topics().forEach(topic -> { + topic.partitions().forEach(partition -> { + partitions.put( + new TopicPartition(topic.name(), partition.partitionIndex()), + Errors.forCode(partition.errorCode())); + }); + }); + + context.getFuture().complete(partitions); + } + + @Override + void handleFailure(Throwable throwable) { + context.getFuture().completeExceptionally(throwable); + } + }; + } + @Override public Map metrics() { return Collections.unmodifiableMap(this.metrics.metrics()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 7fdf4398b9ad..5136d6886dd2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -154,13 +154,25 @@ public interface Consumer extends Closeable { /** * @see KafkaConsumer#committed(TopicPartition) */ + @Deprecated OffsetAndMetadata committed(TopicPartition partition); /** * @see KafkaConsumer#committed(TopicPartition, Duration) */ + @Deprecated OffsetAndMetadata committed(TopicPartition partition, final Duration timeout); + /** + * @see KafkaConsumer#committed(Set) + */ + Map committed(Set partitions); + + /** + * @see KafkaConsumer#committed(Set, Duration) + */ + Map committed(Set partitions, final Duration timeout); + /** * @see KafkaConsumer#metrics() */ diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index a4f3003cc500..e0f7c4d160d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -251,9 +251,9 @@ public class ConsumerConfig extends AbstractConfig { /** isolation.level */ public static final String ISOLATION_LEVEL_CONFIG = "isolation.level"; - public static final String ISOLATION_LEVEL_DOC = "

Controls how to read messages written transactionally. If set to read_committed, consumer.poll() will only return" + + public static final String ISOLATION_LEVEL_DOC = "Controls how to read messages written transactionally. If set to read_committed, consumer.poll() will only return" + " transactional messages which have been committed. If set to read_uncommitted' (the default), consumer.poll() will return all messages, even transactional messages" + - " which have been aborted. Non-transactional messages will be returned unconditionally in either mode.

Messages will always be returned in offset order. Hence, in " + + " which have been aborted. Non-transactional messages will be returned unconditionally in either mode.

Messages will always be returned in offset order. Hence, in " + " read_committed mode, consumer.poll() will only return messages up to the last stable offset (LSO), which is the one less than the offset of the first open transaction." + " In particular any messages appearing after messages belonging to ongoing transactions will be withheld until the relevant transaction has been completed. As a result, read_committed" + " consumers will not be able to read up to the high watermark when there are in flight transactions.

Further, when in read_committed the seekToEnd method will" + @@ -556,7 +556,7 @@ public static ConfigDef configDef() { } public static void main(String[] args) { - System.out.println(CONFIG.toHtmlTable()); + System.out.println(CONFIG.toHtml()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java index 6046ef954682..ec2ce39a2d9b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java @@ -67,7 +67,9 @@ * During a rebalance event, the {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} function will always be triggered exactly once when * the rebalance completes. That is, even if there is no newly assigned partitions for a consumer member, its {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} * will still be triggered with an empty collection of partitions. As a result this function can be used also to notify when a rebalance event has happened. - * On the other hand, {@link #onPartitionsRevoked(Collection)} and {@link #onPartitionsLost(Collection)} + * With eager rebalancing, {@link #onPartitionsRevoked(Collection)} will always be called at the start of a rebalance. On the other hand, {@link #onPartitionsLost(Collection)} + * will only be called when there were non-empty partitions that were lost. + * With cooperative rebalancing, {@link #onPartitionsRevoked(Collection)} and {@link #onPartitionsLost(Collection)} * will only be triggered when there are non-empty partitions revoked or lost from this consumer member during a rebalance event. *

* It is possible @@ -117,16 +119,16 @@ public interface ConsumerRebalanceListener { /** - * A callback method the user can implement to provide handling of offset commits to a customized store on the start - * of a rebalance operation. This method will be called before a rebalance operation starts and after the consumer - * stops fetching data. It can also be called when consumer is being closed ({@link KafkaConsumer#close(Duration)}) + * A callback method the user can implement to provide handling of offset commits to a customized store. + * This method will be called during a rebalance operation when the consumer has to give up some partitions. + * It can also be called when consumer is being closed ({@link KafkaConsumer#close(Duration)}) * or is unsubscribing ({@link KafkaConsumer#unsubscribe()}). * It is recommended that offsets should be committed in this callback to either Kafka or a * custom offset store to prevent duplicate data. *

- * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} - *

- * NOTE: This method is only called before rebalances. It is not called prior to {@link KafkaConsumer#close()}. + * In eager rebalancing, it will always be called at the start of a rebalance and after the consumer stops fetching data. + * In cooperative rebalancing, it will be called at the end of a rebalance on the set of partitions being revoked iff the set is non-empty. + * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}. *

* It is common for the revocation callback to use the consumer instance in order to commit offsets. It is possible * for a {@link org.apache.kafka.common.errors.WakeupException} or {@link org.apache.kafka.common.errors.InterruptException} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index be3e176c2eeb..7e023415dc3d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1735,7 +1735,10 @@ public long position(TopicPartition partition, final Duration timeout) { * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors * @throws org.apache.kafka.common.errors.TimeoutException if the committed offset cannot be found before * the timeout specified by {@code default.api.timeout.ms} expires. + * + * @deprecated since 2.4 Use {@link #committed(Set)} instead */ + @Deprecated @Override public OffsetAndMetadata committed(TopicPartition partition) { return committed(partition, Duration.ofMillis(defaultApiTimeoutMs)); @@ -1745,7 +1748,8 @@ public OffsetAndMetadata committed(TopicPartition partition) { * Get the last committed offset for the given partition (whether the commit happened by this process or * another). This offset will be used as the position for the consumer in the event of a failure. *

- * This call will block to do a remote call to get the latest committed offsets from the server. + * This call will block until the position can be determined, an unrecoverable error is + * encountered (in which case it is thrown to the caller), or the timeout expires. * * @param partition The partition to check * @param timeout The maximum amount of time to await the current committed offset @@ -1760,21 +1764,85 @@ public OffsetAndMetadata committed(TopicPartition partition) { * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors * @throws org.apache.kafka.common.errors.TimeoutException if the committed offset cannot be found before * expiration of the timeout + * + * @deprecated since 2.4 Use {@link #committed(Set, Duration)} instead */ + @Deprecated @Override public OffsetAndMetadata committed(TopicPartition partition, final Duration timeout) { + return committed(Collections.singleton(partition), timeout).get(partition); + } + + /** + * Get the last committed offsets for the given partitions (whether the commit happened by this process or + * another). The returned offsets will be used as the position for the consumer in the event of a failure. + *

+ * Partitions that do not have a committed offset would not be included in the returned map. + *

+ * If any of the partitions requested do not exist, an exception would be thrown. + *

+ * This call will do a remote call to get the latest committed offsets from the server, and will block until the + * committed offsets are gotten successfully, an unrecoverable error is encountered (in which case it is thrown to + * the caller), or the timeout specified by {@code default.api.timeout.ms} expires (in which case a + * {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller). + * + * @param partitions The partitions to check + * @return The latest committed offsets for the given partitions; partitions that do not have any committed offsets + * would not be included in the returned result + * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this + * function is called + * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while + * this function is called + * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details + * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the + * configured groupId. See the exception for more details + * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors + * @throws org.apache.kafka.common.errors.TimeoutException if the committed offset cannot be found before + * the timeout specified by {@code default.api.timeout.ms} expires. + */ + @Override + public Map committed(final Set partitions) { + return committed(partitions, Duration.ofMillis(defaultApiTimeoutMs)); + } + + /** + * Get the last committed offsets for the given partitions (whether the commit happened by this process or + * another). The returned offsets will be used as the position for the consumer in the event of a failure. + *

+ * Partitions that do not have a committed offset would not be included in the returned map. + *

+ * If any of the partitions requested do not exist, an exception would be thrown. + *

+ * This call will block to do a remote call to get the latest committed offsets from the server. + * + * @param partitions The partitions to check + * @param timeout The maximum amount of time to await the latest committed offsets + * @return The latest committed offsets for the given partitions; partitions that do not have any committed offsets + * would not be included in the returned result + * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this + * function is called + * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while + * this function is called + * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details + * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the + * configured groupId. See the exception for more details + * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors + * @throws org.apache.kafka.common.errors.TimeoutException if the committed offset cannot be found before + * expiration of the timeout + */ + @Override + public Map committed(final Set partitions, final Duration timeout) { acquireAndEnsureOpen(); try { maybeThrowInvalidGroupIdException(); - Map offsets = coordinator.fetchCommittedOffsets( - Collections.singleton(partition), time.timer(timeout)); + Map offsets = coordinator.fetchCommittedOffsets(partitions, time.timer(timeout)); if (offsets == null) { throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the last " + - "committed offset for partition " + partition + " could be determined. Try tuning default.api.timeout.ms " + - "larger to relax the threshold."); + "committed offset for partitions " + partitions + " could be determined. Try tuning default.api.timeout.ms " + + "larger to relax the threshold."); } else { offsets.forEach(this::updateLastSeenEpochIfNewer); - return offsets.get(partition); + return offsets; } } finally { release(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 947746f3ce1a..0db883ed1b41 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -42,6 +42,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; +import java.util.stream.Collectors; /** @@ -290,20 +291,33 @@ public void seek(TopicPartition partition, OffsetAndMetadata offsetAndMetadata) subscriptions.seek(partition, offsetAndMetadata.offset()); } + @Deprecated @Override - public synchronized OffsetAndMetadata committed(TopicPartition partition) { - ensureNotClosed(); - if (subscriptions.isAssigned(partition)) { - return committed.get(partition); - } - return new OffsetAndMetadata(0); + public synchronized OffsetAndMetadata committed(final TopicPartition partition) { + return committed(Collections.singleton(partition)).get(partition); } + @Deprecated @Override - public OffsetAndMetadata committed(TopicPartition partition, final Duration timeout) { + public OffsetAndMetadata committed(final TopicPartition partition, final Duration timeout) { return committed(partition); } + @Override + public synchronized Map committed(final Set partitions) { + ensureNotClosed(); + + return partitions.stream() + .filter(committed::containsKey) + .collect(Collectors.toMap(tp -> tp, tp -> subscriptions.isAssigned(tp) ? + committed.get(tp) : new OffsetAndMetadata(0))); + } + + @Override + public synchronized Map committed(final Set partitions, final Duration timeout) { + return committed(partitions); + } + @Override public synchronized long position(TopicPartition partition) { ensureNotClosed(); @@ -446,7 +460,6 @@ public synchronized void close() { @SuppressWarnings("deprecation") @Override public synchronized void close(long timeout, TimeUnit unit) { - ensureNotClosed(); this.closed = true; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java index 78956abb2182..ea642885c062 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java @@ -40,6 +40,29 @@ *

  • C0: [t0p0, t0p1, t1p0, t1p1]
  • *
  • C1: [t0p2, t1p2]
  • * + * + * Since the introduction of static membership, we could leverage group.instance.id to make the assignment behavior more sticky. + * For the above example, after one rolling bounce, group coordinator will attempt to assign new member.id towards consumers, + * for example C0 -> C3 C1 -> C2. + * + *

    The assignment could be completely shuffled to: + *

      + *
    • C3 (was C0): [t0p2, t1p2] (before was [t0p0, t0p1, t1p0, t1p1]) + *
    • C2 (was C1): [t0p0, t0p1, t1p0, t1p1] (before was [t0p2, t1p2]) + *
    + * + * The assignment change was caused by the change of member.id relative order, and + * can be avoided by setting the group.instance.id. + * Consumers will have individual instance ids I1, I2. As long as + * 1. Number of members remain the same across generation + * 2. Static members' identities persist across generation + * 3. Subscription pattern doesn't change for any member + * + *

    The assignment will always be: + *

      + *
    • I0: [t0p0, t0p1, t1p0, t1p1] + *
    • I1: [t0p2, t1p2] + *
    */ public class RangeAssignor extends AbstractPartitionAssignor { @@ -48,27 +71,30 @@ public String name() { return "range"; } - private Map> consumersPerTopic(Map consumerMetadata) { - Map> res = new HashMap<>(); + private Map> consumersPerTopic(Map consumerMetadata) { + Map> topicToConsumers = new HashMap<>(); for (Map.Entry subscriptionEntry : consumerMetadata.entrySet()) { String consumerId = subscriptionEntry.getKey(); - for (String topic : subscriptionEntry.getValue().topics()) - put(res, topic, consumerId); + MemberInfo memberInfo = new MemberInfo(consumerId, subscriptionEntry.getValue().groupInstanceId()); + for (String topic : subscriptionEntry.getValue().topics()) { + put(topicToConsumers, topic, memberInfo); + } } - return res; + return topicToConsumers; } @Override public Map> assign(Map partitionsPerTopic, Map subscriptions) { - Map> consumersPerTopic = consumersPerTopic(subscriptions); + Map> consumersPerTopic = consumersPerTopic(subscriptions); + Map> assignment = new HashMap<>(); for (String memberId : subscriptions.keySet()) assignment.put(memberId, new ArrayList<>()); - for (Map.Entry> topicEntry : consumersPerTopic.entrySet()) { + for (Map.Entry> topicEntry : consumersPerTopic.entrySet()) { String topic = topicEntry.getKey(); - List consumersForTopic = topicEntry.getValue(); + List consumersForTopic = topicEntry.getValue(); Integer numPartitionsForTopic = partitionsPerTopic.get(topic); if (numPartitionsForTopic == null) @@ -83,10 +109,9 @@ public Map> assign(Map partitionsP for (int i = 0, n = consumersForTopic.size(); i < n; i++) { int start = numPartitionsPerConsumer * i + Math.min(i, consumersWithExtraPartition); int length = numPartitionsPerConsumer + (i + 1 > consumersWithExtraPartition ? 0 : 1); - assignment.get(consumersForTopic.get(i)).addAll(partitions.subList(start, start + length)); + assignment.get(consumersForTopic.get(i).memberId).addAll(partitions.subList(start, start + length)); } } return assignment; } - } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java index 0981400a6e5a..0ac90b386834 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java @@ -77,7 +77,7 @@ * After one rolling bounce, group coordinator will attempt to assign new member.id towards consumers, * for example C0 -> C5 C1 -> C3, C2 -> C4. * - *

    the assignment could be completely shuffled to: + *

    The assignment could be completely shuffled to: *

      *
    • C3 (was C1): [t0p0, t1p0] (before was [t0p1, t1p1]) *
    • C4 (was C2): [t0p1, t1p1] (before was [t0p2, t1p2]) @@ -88,6 +88,7 @@ * I1, I2, I3. As long as * 1. Number of members remain the same across generation * 2. Static members' identities persist across generation + * 3. Subscription pattern doesn't change for any member * *

      The assignment will always be: *

        diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 4c71a89f3bdc..fbd15d8e905f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -39,12 +39,13 @@ import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.message.SyncGroupRequestData; import org.apache.kafka.common.metrics.Measurable; -import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.CumulativeCount; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Meter; +import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.metrics.stats.WindowedCount; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -122,13 +123,15 @@ private enum MemberState { protected final ConsumerNetworkClient client; protected final Time time; - private HeartbeatThread heartbeatThread = null; + private Node coordinator = null; private boolean rejoinNeeded = true; private boolean needsJoinPrepare = true; private MemberState state = MemberState.UNJOINED; + private HeartbeatThread heartbeatThread = null; private RequestFuture joinFuture = null; - private Node coordinator = null; private Generation generation = Generation.NO_GENERATION; + private long lastRebalanceStartMs = -1L; + private long lastRebalanceEndMs = -1L; private RequestFuture findCoordinatorFuture = null; @@ -440,6 +443,10 @@ private synchronized RequestFuture initiateJoinGroup() { disableHeartbeatThread(); state = MemberState.REBALANCING; + // a rebalance can be triggered consecutively if the previous one failed, + // in this case we would not update the start time. + if (lastRebalanceStartMs == -1L) + lastRebalanceStartMs = time.milliseconds(); joinFuture = sendJoinGroupRequest(); joinFuture.addListener(new RequestFutureListener() { @Override @@ -450,6 +457,10 @@ public void onSuccess(ByteBuffer value) { log.info("Successfully joined group with generation {}", generation.generationId); state = MemberState.STABLE; rejoinNeeded = false; + // record rebalance latency + lastRebalanceEndMs = time.milliseconds(); + sensors.successfulRebalanceSensor.record(lastRebalanceEndMs - lastRebalanceStartMs); + lastRebalanceStartMs = -1L; if (heartbeatThread != null) heartbeatThread.enable(); @@ -462,6 +473,7 @@ public void onFailure(RuntimeException e) { // after having been woken up, the exception is ignored and we will rejoin synchronized (AbstractCoordinator.this) { state = MemberState.UNJOINED; + sensors.failedRebalanceSensor.record(); } } }); @@ -511,7 +523,7 @@ public void handle(JoinGroupResponse joinResponse, RequestFuture fut Errors error = joinResponse.error(); if (error == Errors.NONE) { log.debug("Received successful JoinGroup response: {}", joinResponse); - sensors.joinLatency.record(response.requestLatencyMs()); + sensors.joinSensor.record(response.requestLatencyMs()); synchronized (AbstractCoordinator.this) { if (state != MemberState.REBALANCING) { @@ -641,7 +653,7 @@ public void handle(SyncGroupResponse syncResponse, RequestFuture future) { Errors error = syncResponse.error(); if (error == Errors.NONE) { - sensors.syncLatency.record(response.requestLatencyMs()); + sensors.syncSensor.record(response.requestLatencyMs()); future.complete(ByteBuffer.wrap(syncResponse.data.assignment())); } else { requestRejoin(); @@ -931,7 +943,7 @@ synchronized RequestFuture sendHeartbeatRequest() { private class HeartbeatResponseHandler extends CoordinatorResponseHandler { @Override public void handle(HeartbeatResponse heartbeatResponse, RequestFuture future) { - sensors.heartbeatLatency.record(response.requestLatencyMs()); + sensors.heartbeatSensor.record(response.requestLatencyMs()); Errors error = heartbeatResponse.error(); if (error == Errors.NONE) { log.debug("Received successful Heartbeat response"); @@ -1005,44 +1017,98 @@ protected Meter createMeter(Metrics metrics, String groupName, String baseName, private class GroupCoordinatorMetrics { public final String metricGrpName; - public final Sensor heartbeatLatency; - public final Sensor joinLatency; - public final Sensor syncLatency; + public final Sensor heartbeatSensor; + public final Sensor joinSensor; + public final Sensor syncSensor; + public final Sensor successfulRebalanceSensor; + public final Sensor failedRebalanceSensor; public GroupCoordinatorMetrics(Metrics metrics, String metricGrpPrefix) { this.metricGrpName = metricGrpPrefix + "-coordinator-metrics"; - this.heartbeatLatency = metrics.sensor("heartbeat-latency"); - this.heartbeatLatency.add(metrics.metricName("heartbeat-response-time-max", - this.metricGrpName, - "The max time taken to receive a response to a heartbeat request"), new Max()); - this.heartbeatLatency.add(createMeter(metrics, metricGrpName, "heartbeat", "heartbeats")); + this.heartbeatSensor = metrics.sensor("heartbeat-latency"); + this.heartbeatSensor.add(metrics.metricName("heartbeat-response-time-max", + this.metricGrpName, + "The max time taken to receive a response to a heartbeat request"), new Max()); + this.heartbeatSensor.add(createMeter(metrics, metricGrpName, "heartbeat", "heartbeats")); - this.joinLatency = metrics.sensor("join-latency"); - this.joinLatency.add(metrics.metricName("join-time-avg", + this.joinSensor = metrics.sensor("join-latency"); + this.joinSensor.add(metrics.metricName("join-time-avg", + this.metricGrpName, + "The average time taken for a group rejoin"), new Avg()); + this.joinSensor.add(metrics.metricName("join-time-max", + this.metricGrpName, + "The max time taken for a group rejoin"), new Max()); + this.joinSensor.add(createMeter(metrics, metricGrpName, "join", "group joins")); + + this.syncSensor = metrics.sensor("sync-latency"); + this.syncSensor.add(metrics.metricName("sync-time-avg", + this.metricGrpName, + "The average time taken for a group sync"), new Avg()); + this.syncSensor.add(metrics.metricName("sync-time-max", + this.metricGrpName, + "The max time taken for a group sync"), new Max()); + this.syncSensor.add(createMeter(metrics, metricGrpName, "sync", "group syncs")); + + this.successfulRebalanceSensor = metrics.sensor("rebalance-latency"); + this.successfulRebalanceSensor.add(metrics.metricName("rebalance-latency-avg", + this.metricGrpName, + "The average time taken for a group to complete a successful rebalance, which may be composed of " + + "several failed re-trials until it succeeded"), new Avg()); + this.successfulRebalanceSensor.add(metrics.metricName("rebalance-latency-max", + this.metricGrpName, + "The max time taken for a group to complete a successful rebalance, which may be composed of " + + "several failed re-trials until it succeeded"), new Max()); + this.successfulRebalanceSensor.add( + metrics.metricName("rebalance-total", this.metricGrpName, - "The average time taken for a group rejoin"), new Avg()); - this.joinLatency.add(metrics.metricName("join-time-max", + "The total number of successful rebalance events, each event is composed of " + + "several failed re-trials until it succeeded"), + new CumulativeCount() + ); + this.successfulRebalanceSensor.add( + metrics.metricName( + "rebalance-rate-per-hour", this.metricGrpName, - "The max time taken for a group rejoin"), new Max()); - this.joinLatency.add(createMeter(metrics, metricGrpName, "join", "group joins")); - + "The number of successful rebalance events per hour, each event is composed of " + + "several failed re-trials until it succeeded"), + new Rate(TimeUnit.HOURS, new WindowedCount()) + ); - this.syncLatency = metrics.sensor("sync-latency"); - this.syncLatency.add(metrics.metricName("sync-time-avg", + this.failedRebalanceSensor = metrics.sensor("failed-rebalance"); + this.failedRebalanceSensor.add( + metrics.metricName("failed-rebalance-total", this.metricGrpName, - "The average time taken for a group sync"), new Avg()); - this.syncLatency.add(metrics.metricName("sync-time-max", + "The total number of failed rebalance events"), + new CumulativeCount() + ); + this.failedRebalanceSensor.add( + metrics.metricName( + "failed-rebalance-rate-per-hour", this.metricGrpName, - "The max time taken for a group sync"), new Max()); - this.syncLatency.add(createMeter(metrics, metricGrpName, "sync", "group syncs")); + "The number of failed rebalance events per hour"), + new Rate(TimeUnit.HOURS, new WindowedCount()) + ); - Measurable lastHeartbeat = - new Measurable() { - public double measure(MetricConfig config, long now) { - return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS); - } - }; + Measurable lastRebalance = (config, now) -> { + if (lastRebalanceEndMs == -1L) + // if no rebalance is ever triggered, we just return -1. + return -1d; + else + return TimeUnit.SECONDS.convert(now - lastRebalanceEndMs, TimeUnit.MILLISECONDS); + }; + metrics.addMetric(metrics.metricName("last-rebalance-seconds-ago", + this.metricGrpName, + "The number of seconds since the last successful rebalance event"), + lastRebalance); + + Measurable lastHeartbeat = (config, now) -> { + if (heartbeat.lastHeartbeatSend() == 0L) + // if no heartbeat is ever triggered, just return -1. + return -1d; + else + return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS); + }; metrics.addMetric(metrics.metricName("last-heartbeat-seconds-ago", this.metricGrpName, "The number of seconds since the last coordinator heartbeat was sent"), @@ -1251,4 +1317,8 @@ private static class UnjoinedGroupException extends RetriableException { public Heartbeat heartbeat() { return heartbeat; } + + public void setLastRebalanceTime(final long timestamp) { + lastRebalanceEndMs = timestamp; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index a3aaface7822..b5b5ce291292 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -268,7 +268,9 @@ private Exception invokePartitionsAssigned(final Set assignedPar ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); try { + final long startMs = time.milliseconds(); listener.onPartitionsAssigned(assignedPartitions); + sensors.assignCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { @@ -285,7 +287,9 @@ private Exception invokePartitionsRevoked(final Set revokedParti ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); try { + final long startMs = time.milliseconds(); listener.onPartitionsRevoked(revokedPartitions); + sensors.revokeCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { @@ -302,7 +306,9 @@ private Exception invokePartitionsLost(final Set lostPartitions) ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); try { + final long startMs = time.milliseconds(); listener.onPartitionsLost(lostPartitions); + sensors.loseCallbackSensor.record(time.milliseconds() - startMs); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { @@ -345,62 +351,50 @@ protected void onJoinComplete(int generation, return; } - // The leader may have assigned partitions which match our subscription pattern, but which - // were not explicitly requested, so we update the joined subscription here. - maybeUpdateJoinedSubscription(assignedPartitions); - - // give the assignor a chance to update internal state based on the received assignment - ConsumerGroupMetadata metadata = new ConsumerGroupMetadata(rebalanceConfig.groupId, generation, memberId, rebalanceConfig.groupInstanceId); - assignor.onAssignment(assignment, metadata); - - // reschedule the auto commit starting from now - if (autoCommitEnabled) - this.nextAutoCommitTimer.updateAndReset(autoCommitIntervalMs); - - // execute the user's callback after rebalance final AtomicReference firstException = new AtomicReference<>(null); Set addedPartitions = new HashSet<>(assignedPartitions); addedPartitions.removeAll(ownedPartitions); - switch (protocol) { - case EAGER: - // assign partitions that are not yet owned - subscriptions.assignFromSubscribed(assignedPartitions); - - firstException.compareAndSet(null, invokePartitionsAssigned(addedPartitions)); - - break; + // Invoke user's revocation callback before changing assignment or updating state + if (protocol == RebalanceProtocol.COOPERATIVE) { + Set revokedPartitions = new HashSet<>(ownedPartitions); + revokedPartitions.removeAll(assignedPartitions); - case COOPERATIVE: - Set revokedPartitions = new HashSet<>(ownedPartitions); - revokedPartitions.removeAll(assignedPartitions); + log.info("Updating with newly assigned partitions: {}, compare with already owned partitions: {}, " + + "newly added partitions: {}, revoking partitions: {}", + Utils.join(assignedPartitions, ", "), + Utils.join(ownedPartitions, ", "), + Utils.join(addedPartitions, ", "), + Utils.join(revokedPartitions, ", ")); - log.info("Updating with newly assigned partitions: {}, compare with already owned partitions: {}, " + - "newly added partitions: {}, revoking partitions: {}", - Utils.join(assignedPartitions, ", "), - Utils.join(ownedPartitions, ", "), - Utils.join(addedPartitions, ", "), - Utils.join(revokedPartitions, ", ")); + if (!revokedPartitions.isEmpty()) { // revoke partitions that was previously owned but no longer assigned; // note that we should only change the assignment AFTER we've triggered // the revoke callback - if (!revokedPartitions.isEmpty()) { - firstException.compareAndSet(null, invokePartitionsRevoked(revokedPartitions)); - } + firstException.compareAndSet(null, invokePartitionsRevoked(revokedPartitions)); - subscriptions.assignFromSubscribed(assignedPartitions); + // if revoked any partitions, need to re-join the group afterwards + requestRejoin(); + } + } - // add partitions that were not previously owned but are now assigned - firstException.compareAndSet(null, invokePartitionsAssigned(addedPartitions)); + // The leader may have assigned partitions which match our subscription pattern, but which + // were not explicitly requested, so we update the joined subscription here. + maybeUpdateJoinedSubscription(assignedPartitions); - // if revoked any partitions, need to re-join the group afterwards - if (!revokedPartitions.isEmpty()) { - requestRejoin(); - } + // give the assignor a chance to update internal state based on the received assignment + ConsumerGroupMetadata metadata = new ConsumerGroupMetadata(rebalanceConfig.groupId, generation, memberId, rebalanceConfig.groupInstanceId); + assignor.onAssignment(assignment, metadata); - break; - } + // reschedule the auto commit starting from now + if (autoCommitEnabled) + this.nextAutoCommitTimer.updateAndReset(autoCommitIntervalMs); + + subscriptions.assignFromSubscribed(assignedPartitions); + + // add partitions that were not previously owned but are now assigned + firstException.compareAndSet(null, invokePartitionsAssigned(addedPartitions)); if (firstException.get() != null) throw new KafkaException("User rebalance callback throws an error", firstException.get()); @@ -560,7 +554,7 @@ protected Map performAssignment(String leaderId, // when these topics gets updated from metadata refresh. // // TODO: this is a hack and not something we want to support long-term unless we push regex into the protocol - // we may need to modify the PartitionAssignor API to better support this case. + // we may need to modify the ConsumerPartitionAssignor API to better support this case. Set assignedTopics = new HashSet<>(); for (Assignment assigned : assignments.values()) { for (TopicPartition tp : assigned.partitions()) @@ -1078,7 +1072,7 @@ private OffsetCommitResponseHandler(Map offse @Override public void handle(OffsetCommitResponse commitResponse, RequestFuture future) { - sensors.commitLatency.record(response.requestLatencyMs()); + sensors.commitSensor.record(response.requestLatencyMs()); Set unauthorizedTopics = new HashSet<>(); for (OffsetCommitResponseData.OffsetCommitResponseTopic topic : commitResponse.data().topics()) { @@ -1241,19 +1235,46 @@ public void handle(OffsetFetchResponse response, RequestFuture subscriptions.numAssignedPartitions(); metrics.addMetric(metrics.metricName("assigned-partitions", diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java index e852e62cd46b..8b29835d6296 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java @@ -90,7 +90,7 @@ public class ConsumerProtocol { public static final short CONSUMER_PROTOCOL_V0 = 0; public static final short CONSUMER_PROTOCOL_V1 = 1; - public static final short CONSUMER_PROTOCL_LATEST_VERSION = CONSUMER_PROTOCOL_V1; + public static final short CONSUMER_PROTOCOL_LATEST_VERSION = CONSUMER_PROTOCOL_V1; public static final Schema CONSUMER_PROTOCOL_HEADER_SCHEMA = new Schema( new Field(VERSION_KEY_NAME, Type.INT16)); @@ -120,6 +120,11 @@ public class ConsumerProtocol { new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT_V0)), new Field(USER_DATA_KEY_NAME, Type.NULLABLE_BYTES)); + public static Short deserializeVersion(ByteBuffer buffer) { + Struct header = CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer); + return header.getShort(VERSION_KEY_NAME); + } + public static ByteBuffer serializeSubscriptionV0(Subscription subscription) { Struct struct = new Struct(SUBSCRIPTION_V0); struct.set(USER_DATA_KEY_NAME, subscription.userData()); @@ -154,7 +159,7 @@ public static ByteBuffer serializeSubscriptionV1(Subscription subscription) { } public static ByteBuffer serializeSubscription(Subscription subscription) { - return serializeSubscription(subscription, CONSUMER_PROTOCL_LATEST_VERSION); + return serializeSubscription(subscription, CONSUMER_PROTOCOL_LATEST_VERSION); } public static ByteBuffer serializeSubscription(Subscription subscription, short version) { @@ -201,8 +206,7 @@ public static Subscription deserializeSubscriptionV1(ByteBuffer buffer) { } public static Subscription deserializeSubscription(ByteBuffer buffer) { - Struct header = CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer); - Short version = header.getShort(VERSION_KEY_NAME); + Short version = deserializeVersion(buffer); if (version < CONSUMER_PROTOCOL_V0) throw new SchemaException("Unsupported subscription version: " + version); @@ -261,7 +265,7 @@ public static ByteBuffer serializeAssignmentV1(Assignment assignment) { } public static ByteBuffer serializeAssignment(Assignment assignment) { - return serializeAssignment(assignment, CONSUMER_PROTOCL_LATEST_VERSION); + return serializeAssignment(assignment, CONSUMER_PROTOCOL_LATEST_VERSION); } public static ByteBuffer serializeAssignment(Assignment assignment, short version) { @@ -297,8 +301,7 @@ public static Assignment deserializeAssignmentV1(ByteBuffer buffer) { } public static Assignment deserializeAssignment(ByteBuffer buffer) { - Struct header = CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer); - Short version = header.getShort(VERSION_KEY_NAME); + Short version = deserializeVersion(buffer); if (version < CONSUMER_PROTOCOL_V0) throw new SchemaException("Unsupported assignment version: " + version); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java index 3bf8c92c2edb..4d19ef4a0141 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -31,7 +31,7 @@ public final class Heartbeat { private final Timer sessionTimer; private final Timer pollTimer; - private volatile long lastHeartbeatSend; + private volatile long lastHeartbeatSend = 0L; public Heartbeat(GroupRebalanceConfig config, Time time) { 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 00767eb29d97..f81480433df1 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 @@ -83,13 +83,14 @@ public class ProducerConfig extends AbstractConfig { + " server at all. The record will be immediately added to the socket buffer and considered sent. No guarantee can be" + " made that the server has received the record in this case, and the retries configuration will not" + " take effect (as the client won't generally know of any failures). The offset given back for each record will" - + " always be set to -1." + + " always be set to -1." + "
      • acks=1 This will mean the leader will write the record to its local log but will respond" + " without awaiting full acknowledgement from all followers. In this case should the leader fail immediately after" + " acknowledging the record but before the followers have replicated it then the record will be lost." + "
      • acks=all This means the leader will wait for the full set of in-sync replicas to" + " acknowledge the record. This guarantees that the record will not be lost as long as at least one in-sync replica" - + " remains alive. This is the strongest available guarantee. This is equivalent to the acks=-1 setting."; + + " remains alive. This is the strongest available guarantee. This is equivalent to the acks=-1 setting." + + "
      "; /** linger.ms */ public static final String LINGER_MS_CONFIG = "linger.ms"; @@ -421,7 +422,7 @@ public static ConfigDef configDef() { } public static void main(String[] args) { - System.out.println(CONFIG.toHtmlTable()); + System.out.println(CONFIG.toHtml()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/Endpoint.java b/clients/src/main/java/org/apache/kafka/common/Endpoint.java index 963d3aea987b..2353de26ec4d 100644 --- a/clients/src/main/java/org/apache/kafka/common/Endpoint.java +++ b/clients/src/main/java/org/apache/kafka/common/Endpoint.java @@ -17,6 +17,8 @@ package org.apache.kafka.common; import java.util.Objects; +import java.util.Optional; + import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.security.auth.SecurityProtocol; @@ -27,23 +29,24 @@ @InterfaceStability.Evolving public class Endpoint { - private final String listener; + private final String listenerName; private final SecurityProtocol securityProtocol; private final String host; private final int port; - public Endpoint(String listener, SecurityProtocol securityProtocol, String host, int port) { - this.listener = listener; + public Endpoint(String listenerName, SecurityProtocol securityProtocol, String host, int port) { + this.listenerName = listenerName; this.securityProtocol = securityProtocol; this.host = host; this.port = port; } /** - * Returns the listener name of this endpoint. + * Returns the listener name of this endpoint. This is non-empty for endpoints provided + * to broker plugins, but may be empty when used in clients. */ - public String listener() { - return listener; + public Optional listenerName() { + return Optional.ofNullable(listenerName); } /** @@ -77,7 +80,7 @@ public boolean equals(Object o) { } Endpoint that = (Endpoint) o; - return Objects.equals(this.listener, that.listener) && + return Objects.equals(this.listenerName, that.listenerName) && Objects.equals(this.securityProtocol, that.securityProtocol) && Objects.equals(this.host, that.host) && this.port == that.port; @@ -86,13 +89,13 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(listener, securityProtocol, host, port); + return Objects.hash(listenerName, securityProtocol, host, port); } @Override public String toString() { return "Endpoint(" + - "listener='" + listener + '\'' + + "listenerName='" + listenerName + '\'' + ", securityProtocol=" + securityProtocol + ", host='" + host + '\'' + ", port=" + port + 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 fa99931783e0..c22b9c77dac9 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 @@ -1409,4 +1409,54 @@ public boolean visible(String name, Map parsedConfig) { }; } + public String toHtml() { + return toHtml(Collections.emptyMap()); + } + + /** + * Converts this config into an HTML list that can be embedded into docs. + * If dynamicUpdateModes is non-empty, a "Dynamic Update Mode" label + * will be included in the config details with the value of the update mode. Default + * mode is "read-only". + * @param dynamicUpdateModes Config name -> update mode mapping + */ + public String toHtml(Map dynamicUpdateModes) { + boolean hasUpdateModes = !dynamicUpdateModes.isEmpty(); + List configs = sortedConfigs(); + StringBuilder b = new StringBuilder(); + b.append("
        \n"); + + for (ConfigKey key : configs) { + if (key.internalConfig) { + continue; + } + b.append("
      • "); + b.append(""); + b.append(key.name); + b.append(": "); + b.append(key.documentation); + b.append("
        "); + // details + b.append("
          "); + for (String detail : headers()) { + if (detail.equals("Name") || detail.equals("Description")) continue; + addConfigDetail(b, detail, getConfigValue(key, detail)); + } + if (hasUpdateModes) { + String updateMode = dynamicUpdateModes.get(key.name); + if (updateMode == null) + updateMode = "read-only"; + addConfigDetail(b, "Update Mode", updateMode); + } + b.append("
        "); + b.append("
      • \n"); + } + b.append("
      \n"); + return b.toString(); + } + + private static void addConfigDetail(StringBuilder builder, String name, String value) { + builder.append("
    • " + name + ": " + value + "
    • "); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 24f8a5acbf46..9b36736524f3 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -76,12 +76,12 @@ public class TopicConfig { "their data. If set to -1, no time limit is applied."; public static final String MAX_MESSAGE_BYTES_CONFIG = "max.message.bytes"; - public static final String MAX_MESSAGE_BYTES_DOC = "

      The largest record batch size allowed by Kafka. If this " + + public static final String MAX_MESSAGE_BYTES_DOC = "The largest record batch size allowed by Kafka. If this " + "is increased and there are consumers older than 0.10.2, the consumers' fetch size must also be increased so that " + - "the they can fetch record batches this large.

      " + - "

      In the latest message format version, records are always grouped into batches for efficiency. In previous " + + "the they can fetch record batches this large. " + + "In the latest message format version, records are always grouped into batches for efficiency. In previous " + "message format versions, uncompressed records are not grouped into batches and this limit only applies to a " + - "single record in that case.

      "; + "single record in that case."; public static final String INDEX_INTERVAL_BYTES_CONFIG = "index.interval.bytes"; public static final String INDEX_INTERVAL_BYTES_DOCS = "This setting controls how frequently " + diff --git a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java index de54f164be59..8a4a51738ded 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java @@ -73,7 +73,8 @@ public class BrokerSecurityConfigs { + " client authentication is required." + "
    • ssl.client.auth=requested This means client authentication is optional." + " unlike requested , if this option is set client can choose not to provide authentication information about itself" - + "
    • ssl.client.auth=none This means client authentication is not needed."; + + "
    • ssl.client.auth=none This means client authentication is not needed." + + "
    "; public static final String SASL_ENABLED_MECHANISMS_DOC = "The list of SASL mechanisms enabled in the Kafka server. " + "The list may contain any mechanism for which a security provider is available. " diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupSubscribedToTopicException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupSubscribedToTopicException.java new file mode 100644 index 000000000000..a62fe325d813 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/GroupSubscribedToTopicException.java @@ -0,0 +1,23 @@ +/* + * 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.errors; + +public class GroupSubscribedToTopicException extends ApiException { + public GroupSubscribedToTopicException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java index 8d7bdbe85a03..3f852cf1a538 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java @@ -78,6 +78,7 @@ public void init(List metrics) { public boolean containsMbean(String mbeanName) { return mbeans.containsKey(mbeanName); } + @Override public void metricChange(KafkaMetric metric) { synchronized (LOCK) { 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 2393da202369..ba59940175ea 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 @@ -44,6 +44,8 @@ import org.apache.kafka.common.message.InitProducerIdResponseData; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupResponseData; +import org.apache.kafka.common.message.LeaderAndIsrRequestData; +import org.apache.kafka.common.message.LeaderAndIsrResponseData; import org.apache.kafka.common.message.LeaveGroupRequestData; import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.ListGroupsRequestData; @@ -56,6 +58,8 @@ import org.apache.kafka.common.message.MetadataResponseData; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; +import org.apache.kafka.common.message.OffsetDeleteRequestData; +import org.apache.kafka.common.message.OffsetDeleteResponseData; import org.apache.kafka.common.message.OffsetFetchRequestData; import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; @@ -64,8 +68,12 @@ import org.apache.kafka.common.message.SaslAuthenticateResponseData; import org.apache.kafka.common.message.SaslHandshakeRequestData; import org.apache.kafka.common.message.SaslHandshakeResponseData; +import org.apache.kafka.common.message.StopReplicaRequestData; +import org.apache.kafka.common.message.StopReplicaResponseData; import org.apache.kafka.common.message.SyncGroupRequestData; import org.apache.kafka.common.message.SyncGroupResponseData; +import org.apache.kafka.common.message.UpdateMetadataRequestData; +import org.apache.kafka.common.message.UpdateMetadataResponseData; import org.apache.kafka.common.message.TxnOffsetCommitRequestData; import org.apache.kafka.common.message.TxnOffsetCommitResponseData; import org.apache.kafka.common.protocol.types.Schema; @@ -101,18 +109,12 @@ import org.apache.kafka.common.requests.EndTxnResponse; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; -import org.apache.kafka.common.requests.LeaderAndIsrRequest; -import org.apache.kafka.common.requests.LeaderAndIsrResponse; import org.apache.kafka.common.requests.ListOffsetRequest; import org.apache.kafka.common.requests.ListOffsetResponse; import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest; import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse; import org.apache.kafka.common.requests.ProduceRequest; import org.apache.kafka.common.requests.ProduceResponse; -import org.apache.kafka.common.requests.StopReplicaRequest; -import org.apache.kafka.common.requests.StopReplicaResponse; -import org.apache.kafka.common.requests.UpdateMetadataRequest; -import org.apache.kafka.common.requests.UpdateMetadataResponse; import org.apache.kafka.common.requests.WriteTxnMarkersRequest; import org.apache.kafka.common.requests.WriteTxnMarkersResponse; @@ -131,10 +133,9 @@ public enum ApiKeys { FETCH(1, "Fetch", FetchRequest.schemaVersions(), FetchResponse.schemaVersions()), LIST_OFFSETS(2, "ListOffsets", ListOffsetRequest.schemaVersions(), ListOffsetResponse.schemaVersions()), METADATA(3, "Metadata", MetadataRequestData.SCHEMAS, MetadataResponseData.SCHEMAS), - LEADER_AND_ISR(4, "LeaderAndIsr", true, LeaderAndIsrRequest.schemaVersions(), LeaderAndIsrResponse.schemaVersions()), - STOP_REPLICA(5, "StopReplica", true, StopReplicaRequest.schemaVersions(), StopReplicaResponse.schemaVersions()), - UPDATE_METADATA(6, "UpdateMetadata", true, UpdateMetadataRequest.schemaVersions(), - UpdateMetadataResponse.schemaVersions()), + LEADER_AND_ISR(4, "LeaderAndIsr", true, LeaderAndIsrRequestData.SCHEMAS, LeaderAndIsrResponseData.SCHEMAS), + STOP_REPLICA(5, "StopReplica", true, StopReplicaRequestData.SCHEMAS, StopReplicaResponseData.SCHEMAS), + UPDATE_METADATA(6, "UpdateMetadata", true, UpdateMetadataRequestData.SCHEMAS, UpdateMetadataResponseData.SCHEMAS), CONTROLLED_SHUTDOWN(7, "ControlledShutdown", true, ControlledShutdownRequestData.SCHEMAS, ControlledShutdownResponseData.SCHEMAS), OFFSET_COMMIT(8, "OffsetCommit", OffsetCommitRequestData.SCHEMAS, OffsetCommitResponseData.SCHEMAS), @@ -201,7 +202,8 @@ public Struct parseResponse(short version, ByteBuffer buffer) { ALTER_PARTITION_REASSIGNMENTS(45, "AlterPartitionReassignments", AlterPartitionReassignmentsRequestData.SCHEMAS, AlterPartitionReassignmentsResponseData.SCHEMAS), LIST_PARTITION_REASSIGNMENTS(46, "ListPartitionReassignments", ListPartitionReassignmentsRequestData.SCHEMAS, - ListPartitionReassignmentsResponseData.SCHEMAS); + ListPartitionReassignmentsResponseData.SCHEMAS), + OFFSET_DELETE(47, "OffsetDelete", OffsetDeleteRequestData.SCHEMAS, OffsetDeleteResponseData.SCHEMAS); private static final ApiKeys[] ID_TO_TYPE; private static final int MIN_API_KEY = 0; @@ -332,6 +334,14 @@ public boolean isVersionSupported(short apiVersion) { return apiVersion >= oldestVersion() && apiVersion <= latestVersion(); } + public short headerVersion(short apiVersion) { + if ((this == CONTROLLED_SHUTDOWN) && (apiVersion == 0)) { + return 0; + } else { + return 1; + } + } + private static String toHtml() { final StringBuilder b = new StringBuilder(); b.append("\n"); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 9f11fc89adf0..0425ef1a4d31 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.errors.BrokerNotAvailableException; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.ConcurrentTransactionsException; +import org.apache.kafka.common.errors.GroupSubscribedToTopicException; import org.apache.kafka.common.errors.ControllerMovedException; import org.apache.kafka.common.errors.CoordinatorLoadInProgressException; import org.apache.kafka.common.errors.CoordinatorNotAvailableException; @@ -312,7 +313,9 @@ public enum Errors { EligibleLeadersNotAvailableException::new), ELECTION_NOT_NEEDED(84, "Leader election not needed for topic partition", ElectionNotNeededException::new), NO_REASSIGNMENT_IN_PROGRESS(85, "No partition reassignment is in progress.", - NoReassignmentInProgressException::new); + NoReassignmentInProgressException::new), + GROUP_SUBSCRIBED_TO_TOPIC(86, "Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it.", + GroupSubscribedToTopicException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java b/clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java index e3fdea7feae3..6e31e50c47c2 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java @@ -19,9 +19,13 @@ import org.apache.kafka.common.utils.Utils; +import java.nio.ByteBuffer; import java.util.Iterator; +import java.util.UUID; public final class MessageUtil { + public static final UUID ZERO_UUID = new UUID(0L, 0L); + /** * Get the length of the UTF8 representation of a string, without allocating * a byte buffer for the string. @@ -34,6 +38,21 @@ public static short serializedUtf8Length(CharSequence input) { return (short) count; } + /** + * Copy a byte buffer into an array. This will not affect the buffer's + * position or mark. + */ + public static byte[] byteBufferToArray(ByteBuffer buf) { + byte[] arr = new byte[buf.remaining()]; + int prevPosition = buf.position(); + try { + buf.get(arr); + } finally { + buf.position(prevPosition); + } + return arr; + } + public static String deepToString(Iterator iter) { StringBuilder bld = new StringBuilder("["); String prefix = ""; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index b5042c3f92a8..0e0975ccb831 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -16,12 +16,11 @@ */ package org.apache.kafka.common.protocol; -import org.apache.kafka.common.protocol.types.ArrayOf; +import org.apache.kafka.common.message.RequestHeaderData; +import org.apache.kafka.common.message.ResponseHeaderData; import org.apache.kafka.common.protocol.types.BoundField; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Type; -import org.apache.kafka.common.requests.RequestHeader; -import org.apache.kafka.common.requests.ResponseHeader; import java.util.LinkedHashMap; import java.util.LinkedHashSet; @@ -43,18 +42,19 @@ private static void schemaToBnfHtml(Schema schema, StringBuilder b, int indentSi // Top level fields for (BoundField field: schema.fields()) { - if (field.def.type instanceof ArrayOf) { + Type type = field.def.type; + if (type.isArray()) { b.append("["); b.append(field.def.name); b.append("] "); - Type innerType = ((ArrayOf) field.def.type).type(); - if (!subTypes.containsKey(field.def.name)) - subTypes.put(field.def.name, innerType); + if (!subTypes.containsKey(field.def.name)) { + subTypes.put(field.def.name, type.arrayElementType().get()); + } } else { b.append(field.def.name); b.append(" "); if (!subTypes.containsKey(field.def.name)) - subTypes.put(field.def.name, field.def.type); + subTypes.put(field.def.name, type); } } b.append("\n"); @@ -81,8 +81,8 @@ private static void schemaToBnfHtml(Schema schema, StringBuilder b, int indentSi private static void populateSchemaFields(Schema schema, Set fields) { for (BoundField field: schema.fields()) { fields.add(field); - if (field.def.type instanceof ArrayOf) { - Type innerType = ((ArrayOf) field.def.type).type(); + if (field.def.type.isArray()) { + Type innerType = field.def.type.arrayElementType().get(); if (innerType instanceof Schema) populateSchemaFields((Schema) innerType, fields); } else if (field.def.type instanceof Schema) @@ -116,18 +116,20 @@ public static String toHtml() { final StringBuilder b = new StringBuilder(); b.append("
    Headers:
    \n"); - b.append("
    ");
    -        b.append("Request Header => ");
    -        schemaToBnfHtml(RequestHeader.SCHEMA, b, 2);
    -        b.append("
    \n"); - schemaToFieldTableHtml(RequestHeader.SCHEMA, b); - - b.append("
    ");
    -        b.append("Response Header => ");
    -        schemaToBnfHtml(ResponseHeader.SCHEMA, b, 2);
    -        b.append("
    \n"); - schemaToFieldTableHtml(ResponseHeader.SCHEMA, b); - + for (int i = 0; i < RequestHeaderData.SCHEMAS.length; i++) { + b.append("
    ");
    +            b.append("Request Header v").append(i).append(" => ");
    +            schemaToBnfHtml(RequestHeaderData.SCHEMAS[i], b, 2);
    +            b.append("
    \n"); + schemaToFieldTableHtml(RequestHeaderData.SCHEMAS[i], b); + } + for (int i = 0; i < ResponseHeaderData.SCHEMAS.length; i++) { + b.append("
    ");
    +            b.append("Response Header v").append(i).append(" => ");
    +            schemaToBnfHtml(ResponseHeaderData.SCHEMAS[i], b, 2);
    +            b.append("
    \n"); + schemaToFieldTableHtml(ResponseHeaderData.SCHEMAS[i], b); + } for (ApiKeys key : ApiKeys.values()) { // Key b.append("
    "); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java b/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java index a527239ca486..e966bde1662a 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java @@ -18,6 +18,7 @@ package org.apache.kafka.common.protocol; import java.nio.charset.StandardCharsets; +import java.util.UUID; public interface Readable { byte readByte(); @@ -54,4 +55,11 @@ default byte[] readNullableBytes() { readArray(arr); return arr; } + + /** + * Read a UUID with the most significant digits first. + */ + default UUID readUUID() { + return new UUID(readLong(), readLong()); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Writable.java b/clients/src/main/java/org/apache/kafka/common/protocol/Writable.java index 9478ed3ec622..e929e53bc624 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Writable.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Writable.java @@ -18,6 +18,7 @@ package org.apache.kafka.common.protocol; import java.nio.charset.StandardCharsets; +import java.util.UUID; public interface Writable { void writeByte(byte val); @@ -68,4 +69,12 @@ default void writeString(String string) { writeShort((short) arr.length); writeArray(arr); } + + /** + * Write a UUID with the most significant digits first. + */ + default void writeUUID(UUID uuid) { + writeLong(uuid.getMostSignificantBits()); + writeLong(uuid.getLeastSignificantBits()); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java index 6609dfd51961..3333084ef663 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.protocol.types.Type.DocumentedType; import java.nio.ByteBuffer; +import java.util.Optional; /** * Represents a type for an array of a particular type @@ -91,8 +92,9 @@ public int sizeOf(Object o) { return size; } - public Type type() { - return type; + @Override + public Optional arrayElementType() { + return Optional.of(type); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java index 72e051c179e7..b79bc932b420 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java @@ -75,6 +75,16 @@ public Int64(String name, String docString, long defaultValue) { } } + public static class UUID extends Field { + public UUID(String name, String docString) { + super(name, Type.UUID, docString, false, null); + } + + public UUID(String name, String docString, UUID defaultValue) { + super(name, Type.UUID, docString, true, defaultValue); + } + } + public static class Int16 extends Field { public Int16(String name, String docString) { super(name, Type.INT16, docString, false, null); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java index 5b12eee6e7a8..721b8c63bd76 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java @@ -209,10 +209,9 @@ private static void handleNode(Type node, Visitor visitor) { visitor.visit(schema); for (BoundField f : schema.fields()) handleNode(f.def.type, visitor); - } else if (node instanceof ArrayOf) { - ArrayOf array = (ArrayOf) node; - visitor.visit(array); - handleNode(array.type(), visitor); + } else if (node.isArray()) { + visitor.visit(node); + handleNode(node.arrayElementType().get(), visitor); } else { visitor.visit(node); } @@ -223,7 +222,6 @@ private static void handleNode(Type node, Visitor visitor) { */ public static abstract class Visitor { public void visit(Schema schema) {} - public void visit(ArrayOf array) {} public void visit(Type field) {} } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index e47a2cdce2a6..0b69851f1644 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Objects; +import java.util.UUID; /** * A record that can be serialized and deserialized according to a pre-defined schema @@ -88,6 +89,10 @@ public Long get(Field.Int64 field) { return getLong(field.name); } + public UUID get(Field.UUID field) { + return getUUID(field.name); + } + public Short get(Field.Int16 field) { return getShort(field.name); } @@ -118,6 +123,12 @@ public Long getOrElse(Field.Int64 field, long alternative) { return alternative; } + public UUID getOrElse(Field.UUID field, UUID alternative) { + if (hasField(field.name)) + return getUUID(field.name); + return alternative; + } + public Short getOrElse(Field.Int16 field, short alternative) { if (hasField(field.name)) return getShort(field.name); @@ -245,6 +256,14 @@ public Long getLong(String name) { return (Long) get(name); } + public UUID getUUID(BoundField field) { + return (UUID) get(field); + } + + public UUID getUUID(String name) { + return (UUID) get(name); + } + public Object[] getArray(BoundField field) { return (Object[]) get(field); } @@ -342,6 +361,10 @@ public Struct set(Field.Int64 def, long value) { return set(def.name, value); } + public Struct set(Field.UUID def, UUID value) { + return set(def.name, value); + } + public Struct set(Field.Int16 def, short value) { return set(def.name, value); } @@ -395,9 +418,8 @@ public Struct instance(BoundField field) { validateField(field); if (field.def.type instanceof Schema) { return new Struct((Schema) field.def.type); - } else if (field.def.type instanceof ArrayOf) { - ArrayOf array = (ArrayOf) field.def.type; - return new Struct((Schema) array.type()); + } else if (field.def.type.isArray()) { + return new Struct((Schema) field.def.type.arrayElementType().get()); } else { throw new SchemaException("Field '" + field.def.name + "' is not a container type, it is of type " + field.def.type); } @@ -449,6 +471,7 @@ public void writeTo(ByteBuffer buffer) { * @throws SchemaException If validation fails */ private void validateField(BoundField field) { + Objects.requireNonNull(field, "`field` must be non-null"); if (this.schema != field.schema) throw new SchemaException("Attempt to access field '" + field.def.name + "' from a different schema instance."); if (field.index > values.length) @@ -472,7 +495,7 @@ public String toString() { BoundField f = this.schema.get(i); b.append(f.def.name); b.append('='); - if (f.def.type instanceof ArrayOf && this.values[i] != null) { + if (f.def.type.isArray() && this.values[i] != null) { Object[] arrayValue = (Object[]) this.values[i]; b.append('['); for (int j = 0; j < arrayValue.length; j++) { @@ -496,7 +519,7 @@ public int hashCode() { int result = 1; for (int i = 0; i < this.values.length; i++) { BoundField f = this.schema.get(i); - if (f.def.type instanceof ArrayOf) { + if (f.def.type.isArray()) { if (this.get(f) != null) { Object[] arrayObject = (Object[]) this.get(f); for (Object arrayItem: arrayObject) @@ -526,7 +549,7 @@ public boolean equals(Object obj) { for (int i = 0; i < this.values.length; i++) { BoundField f = this.schema.get(i); boolean result; - if (f.def.type instanceof ArrayOf) { + if (f.def.type.isArray()) { result = Arrays.equals((Object[]) this.get(f), (Object[]) other.get(f)); } else { Object thisField = this.get(f); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java index 4bd508b77989..aa0ee40e6809 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java @@ -23,6 +23,8 @@ import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; +import java.util.Optional; +import java.util.UUID; /** * A serializable type @@ -63,6 +65,20 @@ public boolean isNullable() { return false; } + /** + * If the type is an array, return the type of the array elements. Otherwise, return empty. + */ + public Optional arrayElementType() { + return Optional.empty(); + } + + /** + * Returns true if the type is an array. + */ + public final boolean isArray() { + return arrayElementType().isPresent(); + } + /** * A Type that can return its description for documentation purposes. */ @@ -313,6 +329,44 @@ public String documentation() { } }; + public static final DocumentedType UUID = new DocumentedType() { + @Override + public void write(ByteBuffer buffer, Object o) { + final java.util.UUID uuid = (java.util.UUID) o; + buffer.putLong(uuid.getMostSignificantBits()); + buffer.putLong(uuid.getLeastSignificantBits()); + } + + @Override + public Object read(ByteBuffer buffer) { + return new java.util.UUID(buffer.getLong(), buffer.getLong()); + } + + @Override + public int sizeOf(Object o) { + return 16; + } + + @Override + public String typeName() { + return "UUID"; + } + + @Override + public UUID validate(Object item) { + if (item instanceof UUID) + return (UUID) item; + else + throw new SchemaException(item + " is not a UUID."); + } + + @Override + public String documentation() { + return "Represents a java.util.UUID. " + + "The values are encoded using sixteen bytes in network byte order (big-endian)."; + } + }; + public static final DocumentedType STRING = new DocumentedType() { @Override public void write(ByteBuffer buffer, Object o) { @@ -665,10 +719,9 @@ public String documentation() { }; private static String toHtml() { - DocumentedType[] types = { BOOLEAN, INT8, INT16, INT32, INT64, - UNSIGNED_INT32, VARINT, VARLONG, + UNSIGNED_INT32, VARINT, VARLONG, UUID, STRING, NULLABLE_STRING, BYTES, NULLABLE_BYTES, RECORDS, new ArrayOf(STRING)}; final StringBuilder b = new StringBuilder(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractControlRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractControlRequest.java index d1fae0502c43..dc4a1e21e8dd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractControlRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractControlRequest.java @@ -17,20 +17,11 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Struct; // Abstract class for all control requests including UpdateMetadataRequest, LeaderAndIsrRequest and StopReplicaRequest public abstract class AbstractControlRequest extends AbstractRequest { - public static final long UNKNOWN_BROKER_EPOCH = -1L; - - protected static final Field.Int32 CONTROLLER_ID = new Field.Int32("controller_id", "The controller id"); - protected static final Field.Int32 CONTROLLER_EPOCH = new Field.Int32("controller_epoch", "The controller epoch"); - protected static final Field.Int64 BROKER_EPOCH = new Field.Int64("broker_epoch", "The broker epoch"); - protected final int controllerId; - protected final int controllerEpoch; - protected final long brokerEpoch; + public static final long UNKNOWN_BROKER_EPOCH = -1L; public static abstract class Builder extends AbstractRequest.Builder { protected final int controllerId; @@ -46,35 +37,14 @@ protected Builder(ApiKeys api, short version, int controllerId, int controllerEp } - public int controllerId() { - return controllerId; - } - - public int controllerEpoch() { - return controllerEpoch; - } - - public long brokerEpoch() { - return brokerEpoch; - } - - protected AbstractControlRequest(ApiKeys api, short version, int controllerId, int controllerEpoch, long brokerEpoch) { + protected AbstractControlRequest(ApiKeys api, short version) { super(api, version); - this.controllerId = controllerId; - this.controllerEpoch = controllerEpoch; - this.brokerEpoch = brokerEpoch; } - protected AbstractControlRequest(ApiKeys api, Struct struct, short version) { - super(api, version); - this.controllerId = struct.get(CONTROLLER_ID); - this.controllerEpoch = struct.get(CONTROLLER_EPOCH); - this.brokerEpoch = struct.getOrElse(BROKER_EPOCH, UNKNOWN_BROKER_EPOCH); - } + public abstract int controllerId(); - // Used for test - long size() { - return toStruct().sizeOf(); - } + public abstract int controllerEpoch(); + + public abstract long brokerEpoch(); } 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 58bf1283a77a..b737b49f4a5c 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 @@ -237,6 +237,8 @@ public static AbstractRequest parseRequest(ApiKeys apiKey, short apiVersion, Str return new AlterPartitionReassignmentsRequest(struct, apiVersion); case LIST_PARTITION_REASSIGNMENTS: return new ListPartitionReassignmentsRequest(struct, apiVersion); + case OFFSET_DELETE: + return new OffsetDeleteRequest(struct, 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 11466d724986..eb11a4244215 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 @@ -23,6 +23,7 @@ import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -31,14 +32,22 @@ public abstract class AbstractResponse extends AbstractRequestResponse { public static final int DEFAULT_THROTTLE_TIME = 0; protected Send toSend(String destination, ResponseHeader header, short apiVersion) { - return new NetworkSend(destination, serialize(apiVersion, header)); + return new NetworkSend(destination, serialize(header.toStruct(), toStruct(apiVersion))); } /** * Visible for testing, typically {@link #toSend(String, ResponseHeader, short)} should be used instead. */ - public ByteBuffer serialize(short version, ResponseHeader responseHeader) { - return serialize(responseHeader.toStruct(), toStruct(version)); + public ByteBuffer serialize(ApiKeys apiKey, int correlationId) { + return serialize(apiKey, apiKey.latestVersion(), correlationId); + } + + /** + * Visible for testing, typically {@link #toSend(String, ResponseHeader, short)} should be used instead. + */ + public ByteBuffer serialize(ApiKeys apiKey, short version, int correlationId) { + ResponseHeader header = new ResponseHeader(correlationId, apiKey.headerVersion(version)); + return serialize(header.toStruct(), toStruct(version)); } public abstract Map errorCounts(); @@ -47,9 +56,9 @@ protected Map errorCounts(Errors error) { return Collections.singletonMap(error, 1); } - protected Map errorCounts(Map errors) { + protected Map errorCounts(Collection errors) { Map errorCounts = new HashMap<>(); - for (Errors error : errors.values()) + for (Errors error : errors) updateErrorCounts(errorCounts, error); return errorCounts; } @@ -93,13 +102,13 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, Struct struct, shor case SYNC_GROUP: return new SyncGroupResponse(struct, version); case STOP_REPLICA: - return new StopReplicaResponse(struct); + return new StopReplicaResponse(struct, version); case CONTROLLED_SHUTDOWN: return new ControlledShutdownResponse(struct, version); case UPDATE_METADATA: - return new UpdateMetadataResponse(struct); + return new UpdateMetadataResponse(struct, version); case LEADER_AND_ISR: - return new LeaderAndIsrResponse(struct); + return new LeaderAndIsrResponse(struct, version); case DESCRIBE_GROUPS: return new DescribeGroupsResponse(struct, version); case LIST_GROUPS: @@ -164,6 +173,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, Struct struct, shor return new AlterPartitionReassignmentsResponse(struct, version); case LIST_PARTITION_REASSIGNMENTS: return new ListPartitionReassignmentsResponse(struct, version); + case OFFSET_DELETE: + return new OffsetDeleteResponse(struct, 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/AddPartitionsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java index ea8a073e0175..7f20f07b6a11 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java @@ -102,7 +102,7 @@ public Map errors() { @Override public Map errorCounts() { - return errorCounts(errors); + return errorCounts(errors.values()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java index 7a73275d7457..2bbdce2122bc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java @@ -132,7 +132,7 @@ public Map responses() { @Override public Map errorCounts() { - return errorCounts(responses); + return errorCounts(responses.values()); } public static AlterReplicaLogDirsResponse parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java index f89316aa3093..83e9444da0ad 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java @@ -64,18 +64,8 @@ public ControlledShutdownRequest(Struct struct, short version) { @Override public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { - ControlledShutdownResponseData response = new ControlledShutdownResponseData(); - response.setErrorCode(Errors.forException(e).code()); - short versionId = version(); - switch (versionId) { - case 0: - case 1: - case 2: - return new ControlledShutdownResponse(response); - default: - throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ApiKeys.CONTROLLED_SHUTDOWN.latestVersion())); - } + return new ControlledShutdownResponse(new ControlledShutdownResponseData(). + setErrorCode(Errors.forException(e).code())); } public static ControlledShutdownRequest parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java index d3061c484489..2ad947c5904b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java @@ -61,26 +61,18 @@ public DeleteGroupsRequest(Struct struct, short version) { @Override public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { Errors error = Errors.forException(e); - - switch (version()) { - case 0: - case 1: - DeletableGroupResultCollection groupResults = new DeletableGroupResultCollection(); - for (String groupId : data.groupsNames()) { - groupResults.add(new DeletableGroupResult() - .setGroupId(groupId) - .setErrorCode(error.code())); - } - - return new DeleteGroupsResponse( - new DeleteGroupsResponseData() - .setResults(groupResults) - .setThrottleTimeMs(throttleTimeMs) - ); - default: - throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - version(), ApiKeys.DELETE_GROUPS.name, ApiKeys.DELETE_GROUPS.latestVersion())); + DeletableGroupResultCollection groupResults = new DeletableGroupResultCollection(); + for (String groupId : data.groupsNames()) { + groupResults.add(new DeletableGroupResult() + .setGroupId(groupId) + .setErrorCode(error.code())); } + + return new DeleteGroupsResponse( + new DeleteGroupsResponseData() + .setResults(groupResults) + .setThrottleTimeMs(throttleTimeMs) + ); } public static DeleteGroupsRequest parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java index 5fe86eb8f164..a98a7c881d96 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java @@ -86,7 +86,7 @@ public Map errorCounts() { } public static DeleteGroupsResponse parse(ByteBuffer buffer, short version) { - return new DeleteGroupsResponse(ApiKeys.DELETE_GROUPS.parseResponse(version, buffer)); + return new DeleteGroupsResponse(ApiKeys.DELETE_GROUPS.parseResponse(version, buffer), version); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index ff034df34f13..e5cc6c8860a5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -65,21 +65,12 @@ public HeartbeatRequest(Struct struct, short version) { @Override public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { - HeartbeatResponseData response = new HeartbeatResponseData(); - response.setErrorCode(Errors.forException(e).code()); - short versionId = version(); - switch (versionId) { - case 0: - return new HeartbeatResponse(response); - case 1: - case 2: - case 3: - response.setThrottleTimeMs(throttleTimeMs); - return new HeartbeatResponse(response); - default: - throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ApiKeys.HEARTBEAT.latestVersion())); + HeartbeatResponseData responseData = new HeartbeatResponseData(). + setErrorCode(Errors.forException(e).code()); + if (version() >= 1) { + responseData.setThrottleTimeMs(throttleTimeMs); } + return new HeartbeatResponse(responseData); } public static HeartbeatRequest parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index 02c14e3274a9..b799e360bd4f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -17,152 +17,37 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.Node; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.LeaderAndIsrRequestData; +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrLiveLeader; +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrTopicState; +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState; +import org.apache.kafka.common.message.LeaderAndIsrResponseData; +import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.FlattenedIterator; import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.Collections; - -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; -import static org.apache.kafka.common.protocol.types.Type.INT32; +import java.util.stream.Collectors; public class LeaderAndIsrRequest extends AbstractControlRequest { - private static final Field.ComplexArray TOPIC_STATES = new Field.ComplexArray("topic_states", "Topic states"); - private static final Field.ComplexArray PARTITION_STATES = new Field.ComplexArray("partition_states", "Partition states"); - private static final Field.ComplexArray LIVE_LEADERS = new Field.ComplexArray("live_leaders", "Live leaders"); - - // PartitionState fields - private static final Field.Int32 LEADER = new Field.Int32("leader", "The broker id for the leader."); - private static final Field.Int32 LEADER_EPOCH = new Field.Int32("leader_epoch", "The leader epoch."); - private static final Field.Array ISR = new Field.Array("isr", INT32, "The in sync replica ids."); - private static final Field.Int32 ZK_VERSION = new Field.Int32("zk_version", "The ZK version."); - private static final Field.Array REPLICAS = new Field.Array("replicas", INT32, "The replica ids."); - private static final Field.Array ADDING_REPLICAS = new Field.Array("adding_replicas", INT32, - "The replica ids we are in the process of adding to the replica set during a reassignment."); - private static final Field.Array REMOVING_REPLICAS = new Field.Array("removing_replicas", INT32, - "The replica ids we are in the process of removing from the replica set during a reassignment."); - private static final Field.Bool IS_NEW = new Field.Bool("is_new", "Whether the replica should have existed on the broker or not"); - - // live_leaders fields - private static final Field.Int32 END_POINT_ID = new Field.Int32("id", "The broker id"); - private static final Field.Str HOST = new Field.Str("host", "The hostname of the broker."); - private static final Field.Int32 PORT = new Field.Int32("port", "The port on which the broker accepts requests."); - - private static final Field PARTITION_STATES_V0 = PARTITION_STATES.withFields( - TOPIC_NAME, - PARTITION_ID, - CONTROLLER_EPOCH, - LEADER, - LEADER_EPOCH, - ISR, - ZK_VERSION, - REPLICAS); - - // PARTITION_STATES_V1 added a per-partition is_new Field. - // This field specifies whether the replica should have existed on the broker or not. - private static final Field PARTITION_STATES_V1 = PARTITION_STATES.withFields( - TOPIC_NAME, - PARTITION_ID, - CONTROLLER_EPOCH, - LEADER, - LEADER_EPOCH, - ISR, - ZK_VERSION, - REPLICAS, - IS_NEW); - - private static final Field PARTITION_STATES_V2 = PARTITION_STATES.withFields( - PARTITION_ID, - CONTROLLER_EPOCH, - LEADER, - LEADER_EPOCH, - ISR, - ZK_VERSION, - REPLICAS, - IS_NEW); - - private static final Field PARTITION_STATES_V3 = PARTITION_STATES.withFields( - PARTITION_ID, - CONTROLLER_EPOCH, - LEADER, - LEADER_EPOCH, - ISR, - ZK_VERSION, - REPLICAS, - ADDING_REPLICAS, - REMOVING_REPLICAS, - IS_NEW); - - // TOPIC_STATES_V2 normalizes TOPIC_STATES_V1 to make it more memory efficient - private static final Field TOPIC_STATES_V2 = TOPIC_STATES.withFields( - TOPIC_NAME, - PARTITION_STATES_V2); - - // TOPIC_STATES_V3 adds two new fields - adding_replicas and removing_replicas - private static final Field TOPIC_STATES_V3 = TOPIC_STATES.withFields( - TOPIC_NAME, - PARTITION_STATES_V3); - - private static final Field LIVE_LEADERS_V0 = LIVE_LEADERS.withFields( - END_POINT_ID, - HOST, - PORT); - - private static final Schema LEADER_AND_ISR_REQUEST_V0 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - PARTITION_STATES_V0, - LIVE_LEADERS_V0); - - // LEADER_AND_ISR_REQUEST_V1 added a per-partition is_new Field. This field specifies whether the replica should - // have existed on the broker or not. - private static final Schema LEADER_AND_ISR_REQUEST_V1 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - PARTITION_STATES_V1, - LIVE_LEADERS_V0); - - // LEADER_AND_ISR_REQUEST_V2 added a broker_epoch Field. This field specifies the generation of the broker across - // bounces. It also normalizes partitions under each topic. - private static final Schema LEADER_AND_ISR_REQUEST_V2 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - BROKER_EPOCH, - TOPIC_STATES_V2, - LIVE_LEADERS_V0); - - // LEADER_AND_ISR_REQUEST_V3 added two new fields - adding_replicas and removing_replicas. - // These fields respectively specify the replica IDs we want to add or remove as part of a reassignment - private static final Schema LEADER_AND_ISR_REQUEST_V3 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - BROKER_EPOCH, - TOPIC_STATES_V3, - LIVE_LEADERS_V0); - - public static Schema[] schemaVersions() { - return new Schema[]{LEADER_AND_ISR_REQUEST_V0, LEADER_AND_ISR_REQUEST_V1, LEADER_AND_ISR_REQUEST_V2, LEADER_AND_ISR_REQUEST_V3}; - } public static class Builder extends AbstractControlRequest.Builder { - private final Map partitionStates; - private final Set liveLeaders; + + private final List partitionStates; + private final Collection liveLeaders; public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch, - Map partitionStates, Set liveLeaders) { + List partitionStates, Collection liveLeaders) { super(ApiKeys.LEADER_AND_ISR, version, controllerId, controllerEpoch, brokerEpoch); this.partitionStates = partitionStates; this.liveLeaders = liveLeaders; @@ -170,7 +55,38 @@ public Builder(short version, int controllerId, int controllerEpoch, long broker @Override public LeaderAndIsrRequest build(short version) { - return new LeaderAndIsrRequest(controllerId, controllerEpoch, brokerEpoch, partitionStates, liveLeaders, version); + List leaders = liveLeaders.stream().map(n -> new LeaderAndIsrLiveLeader() + .setBrokerId(n.id()) + .setHostName(n.host()) + .setPort(n.port()) + ).collect(Collectors.toList()); + + LeaderAndIsrRequestData data = new LeaderAndIsrRequestData() + .setControllerId(controllerId) + .setControllerEpoch(controllerEpoch) + .setBrokerEpoch(brokerEpoch) + .setLiveLeaders(leaders); + + if (version >= 2) { + Map topicStatesMap = groupByTopic(partitionStates); + data.setTopicStates(new ArrayList<>(topicStatesMap.values())); + } else { + data.setUngroupedPartitionStates(partitionStates); + } + + return new LeaderAndIsrRequest(data, version); + } + + private static Map groupByTopic(List partitionStates) { + Map topicStates = new HashMap<>(); + // We don't null out the topic name in LeaderAndIsrRequestPartition since it's ignored by + // the generated code if version >= 2 + for (LeaderAndIsrPartitionState partition : partitionStates) { + LeaderAndIsrTopicState topicState = topicStates.computeIfAbsent(partition.topicName(), + t -> new LeaderAndIsrTopicState().setTopicName(partition.topicName())); + topicState.partitionStates().add(partition); + } + return topicStates; } @Override @@ -184,116 +100,60 @@ public String toString() { .append(", liveLeaders=(").append(Utils.join(liveLeaders, ", ")).append(")") .append(")"); return bld.toString(); + } } - private final Map partitionStates; - private final Set liveLeaders; + private final LeaderAndIsrRequestData data; - private LeaderAndIsrRequest(int controllerId, int controllerEpoch, long brokerEpoch, Map partitionStates, - Set liveLeaders, short version) { - super(ApiKeys.LEADER_AND_ISR, version, controllerId, controllerEpoch, brokerEpoch); - this.partitionStates = partitionStates; - this.liveLeaders = liveLeaders; + LeaderAndIsrRequest(LeaderAndIsrRequestData data, short version) { + super(ApiKeys.LEADER_AND_ISR, version); + this.data = data; + // Do this from the constructor to make it thread-safe (even though it's only needed when some methods are called) + normalize(); } - public LeaderAndIsrRequest(Struct struct, short version) { - super(ApiKeys.LEADER_AND_ISR, struct, version); - - Map partitionStates = new HashMap<>(); - if (struct.hasField(TOPIC_STATES)) { - for (Object topicStatesDataObj : struct.get(TOPIC_STATES)) { - Struct topicStatesData = (Struct) topicStatesDataObj; - String topic = topicStatesData.get(TOPIC_NAME); - for (Object partitionStateDataObj : topicStatesData.get(PARTITION_STATES)) { - Struct partitionStateData = (Struct) partitionStateDataObj; - int partition = partitionStateData.get(PARTITION_ID); - PartitionState partitionState = new PartitionState(partitionStateData); - partitionStates.put(new TopicPartition(topic, partition), partitionState); + private void normalize() { + if (version() >= 2) { + for (LeaderAndIsrTopicState topicState : data.topicStates()) { + for (LeaderAndIsrPartitionState partitionState : topicState.partitionStates()) { + // Set the topic name so that we can always present the ungrouped view to callers + partitionState.setTopicName(topicState.topicName()); } } - } else { - for (Object partitionStateDataObj : struct.get(PARTITION_STATES)) { - Struct partitionStateData = (Struct) partitionStateDataObj; - String topic = partitionStateData.get(TOPIC_NAME); - int partition = partitionStateData.get(PARTITION_ID); - PartitionState partitionState = new PartitionState(partitionStateData); - partitionStates.put(new TopicPartition(topic, partition), partitionState); - } - } - - Set leaders = new HashSet<>(); - for (Object leadersDataObj : struct.get(LIVE_LEADERS)) { - Struct leadersData = (Struct) leadersDataObj; - int id = leadersData.get(END_POINT_ID); - String host = leadersData.get(HOST); - int port = leadersData.get(PORT); - leaders.add(new Node(id, host, port)); } + } - this.partitionStates = partitionStates; - this.liveLeaders = leaders; + public LeaderAndIsrRequest(Struct struct, short version) { + this(new LeaderAndIsrRequestData(struct, version), version); } @Override protected Struct toStruct() { - short version = version(); - Struct struct = new Struct(ApiKeys.LEADER_AND_ISR.requestSchema(version)); - struct.set(CONTROLLER_ID, controllerId); - struct.set(CONTROLLER_EPOCH, controllerEpoch); - struct.setIfExists(BROKER_EPOCH, brokerEpoch); - - if (struct.hasField(TOPIC_STATES)) { - Map> topicStates = CollectionUtils.groupPartitionDataByTopic(partitionStates); - List topicStatesData = new ArrayList<>(topicStates.size()); - for (Map.Entry> entry : topicStates.entrySet()) { - Struct topicStateData = struct.instance(TOPIC_STATES); - topicStateData.set(TOPIC_NAME, entry.getKey()); - Map partitionMap = entry.getValue(); - List partitionStatesData = new ArrayList<>(partitionMap.size()); - for (Map.Entry partitionEntry : partitionMap.entrySet()) { - Struct partitionStateData = topicStateData.instance(PARTITION_STATES); - partitionStateData.set(PARTITION_ID, partitionEntry.getKey()); - partitionEntry.getValue().setStruct(partitionStateData, version); - partitionStatesData.add(partitionStateData); - } - topicStateData.set(PARTITION_STATES, partitionStatesData.toArray()); - topicStatesData.add(topicStateData); - } - struct.set(TOPIC_STATES, topicStatesData.toArray()); - } else { - List partitionStatesData = new ArrayList<>(partitionStates.size()); - for (Map.Entry entry : partitionStates.entrySet()) { - Struct partitionStateData = struct.instance(PARTITION_STATES); - TopicPartition topicPartition = entry.getKey(); - partitionStateData.set(TOPIC_NAME, topicPartition.topic()); - partitionStateData.set(PARTITION_ID, topicPartition.partition()); - entry.getValue().setStruct(partitionStateData, version); - partitionStatesData.add(partitionStateData); - } - struct.set(PARTITION_STATES, partitionStatesData.toArray()); - } + return data.toStruct(version()); + } - List leadersData = new ArrayList<>(liveLeaders.size()); - for (Node leader : liveLeaders) { - Struct leaderData = struct.instance(LIVE_LEADERS); - leaderData.set(END_POINT_ID, leader.id()); - leaderData.set(HOST, leader.host()); - leaderData.set(PORT, leader.port()); - leadersData.add(leaderData); - } - struct.set(LIVE_LEADERS, leadersData.toArray()); - return struct; + protected ByteBuffer toBytes() { + ByteBuffer bytes = ByteBuffer.allocate(size()); + data.write(new ByteBufferAccessor(bytes), version()); + bytes.flip(); + return bytes; } @Override public LeaderAndIsrResponse getErrorResponse(int throttleTimeMs, Throwable e) { + LeaderAndIsrResponseData responseData = new LeaderAndIsrResponseData(); Errors error = Errors.forException(e); - - Map responses = new HashMap<>(partitionStates.size()); - for (TopicPartition partition : partitionStates.keySet()) { - responses.put(partition, error); + responseData.setErrorCode(error.code()); + + List partitions = new ArrayList<>(); + for (LeaderAndIsrPartitionState partition : partitionStates()) { + partitions.add(new LeaderAndIsrPartitionError() + .setTopicName(partition.topicName()) + .setPartitionIndex(partition.partitionIndex()) + .setErrorCode(error.code())); } + responseData.setPartitionErrors(partitions); short versionId = version(); switch (versionId) { @@ -301,133 +161,45 @@ public LeaderAndIsrResponse getErrorResponse(int throttleTimeMs, Throwable e) { case 1: case 2: case 3: - return new LeaderAndIsrResponse(error, responses); + return new LeaderAndIsrResponse(responseData); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ApiKeys.LEADER_AND_ISR.latestVersion())); } } + @Override public int controllerId() { - return controllerId; + return data.controllerId(); } + @Override public int controllerEpoch() { - return controllerEpoch; + return data.controllerEpoch(); } - public Map partitionStates() { - return partitionStates; + @Override + public long brokerEpoch() { + return data.brokerEpoch(); } - public Set liveLeaders() { - return liveLeaders; + public Iterable partitionStates() { + if (version() >= 2) + return () -> new FlattenedIterator<>(data.topicStates().iterator(), + topicState -> topicState.partitionStates().iterator()); + return data.ungroupedPartitionStates(); } - public static LeaderAndIsrRequest parse(ByteBuffer buffer, short version) { - return new LeaderAndIsrRequest(ApiKeys.LEADER_AND_ISR.parseRequest(version, buffer), version); + public List liveLeaders() { + return Collections.unmodifiableList(data.liveLeaders()); } - public static final class PartitionState { - public final BasePartitionState basePartitionState; - public final List addingReplicas; - public final List removingReplicas; - public final boolean isNew; - - public PartitionState(int controllerEpoch, - int leader, - int leaderEpoch, - List isr, - int zkVersion, - List replicas, - boolean isNew) { - this(controllerEpoch, - leader, - leaderEpoch, - isr, - zkVersion, - replicas, - Collections.emptyList(), - Collections.emptyList(), - isNew); - } - - public PartitionState(int controllerEpoch, - int leader, - int leaderEpoch, - List isr, - int zkVersion, - List replicas, - List addingReplicas, - List removingReplicas, - boolean isNew) { - this.basePartitionState = new BasePartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas); - this.addingReplicas = addingReplicas; - this.removingReplicas = removingReplicas; - this.isNew = isNew; - } - - private PartitionState(Struct struct) { - int controllerEpoch = struct.get(CONTROLLER_EPOCH); - int leader = struct.get(LEADER); - int leaderEpoch = struct.get(LEADER_EPOCH); - - Object[] isrArray = struct.get(ISR); - List isr = new ArrayList<>(isrArray.length); - for (Object r : isrArray) - isr.add((Integer) r); - - int zkVersion = struct.get(ZK_VERSION); - - Object[] replicasArray = struct.get(REPLICAS); - List replicas = new ArrayList<>(replicasArray.length); - for (Object r : replicasArray) - replicas.add((Integer) r); - - this.basePartitionState = new BasePartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas); - - List addingReplicas = new ArrayList<>(); - if (struct.hasField(ADDING_REPLICAS)) { - for (Object r : struct.get(ADDING_REPLICAS)) - addingReplicas.add((Integer) r); - } - this.addingReplicas = addingReplicas; - - List removingReplicas = new ArrayList<>(); - if (struct.hasField(REMOVING_REPLICAS)) { - for (Object r : struct.get(REMOVING_REPLICAS)) - removingReplicas.add((Integer) r); - } - this.removingReplicas = removingReplicas; - - this.isNew = struct.getOrElse(IS_NEW, false); - } - - @Override - public String toString() { - return "PartitionState(controllerEpoch=" + basePartitionState.controllerEpoch + - ", leader=" + basePartitionState.leader + - ", leaderEpoch=" + basePartitionState.leaderEpoch + - ", isr=" + Utils.join(basePartitionState.isr, ",") + - ", zkVersion=" + basePartitionState.zkVersion + - ", replicas=" + Utils.join(basePartitionState.replicas, ",") + - ", addingReplicas=" + Utils.join(addingReplicas, ",") + - ", removingReplicas=" + Utils.join(removingReplicas, ",") + - ", isNew=" + isNew + ")"; - } + protected int size() { + return data.size(version()); + } - private void setStruct(Struct struct, short version) { - struct.set(CONTROLLER_EPOCH, basePartitionState.controllerEpoch); - struct.set(LEADER, basePartitionState.leader); - struct.set(LEADER_EPOCH, basePartitionState.leaderEpoch); - struct.set(ISR, basePartitionState.isr.toArray()); - struct.set(ZK_VERSION, basePartitionState.zkVersion); - struct.set(REPLICAS, basePartitionState.replicas.toArray()); - if (version >= 3) { - struct.set(ADDING_REPLICAS, addingReplicas.toArray()); - struct.set(REMOVING_REPLICAS, removingReplicas.toArray()); - } - struct.setIfExists(IS_NEW, isNew); - } + public static LeaderAndIsrRequest parse(ByteBuffer buffer, short version) { + return new LeaderAndIsrRequest(ApiKeys.LEADER_AND_ISR.parseRequest(version, buffer), version); } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java index 3b80222eff0e..0329307e6669 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java @@ -16,45 +16,19 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.LeaderAndIsrResponseData; +import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; - -import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; +import java.util.stream.Collectors; public class LeaderAndIsrResponse extends AbstractResponse { - private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions", "Response for the requests partitions"); - - private static final Field PARTITIONS_V0 = PARTITIONS.withFields( - TOPIC_NAME, - PARTITION_ID, - ERROR_CODE); - private static final Schema LEADER_AND_ISR_RESPONSE_V0 = new Schema( - ERROR_CODE, - PARTITIONS_V0); - - // LeaderAndIsrResponse V1 may receive KAFKA_STORAGE_ERROR in the response - private static final Schema LEADER_AND_ISR_RESPONSE_V1 = LEADER_AND_ISR_RESPONSE_V0; - - private static final Schema LEADER_AND_ISR_RESPONSE_V2 = LEADER_AND_ISR_RESPONSE_V1; - - private static final Schema LEADER_AND_ISR_RESPONSE_V3 = LEADER_AND_ISR_RESPONSE_V2; - - public static Schema[] schemaVersions() { - return new Schema[]{LEADER_AND_ISR_RESPONSE_V0, LEADER_AND_ISR_RESPONSE_V1, LEADER_AND_ISR_RESPONSE_V2, LEADER_AND_ISR_RESPONSE_V3}; - } /** * Possible error code: @@ -62,74 +36,45 @@ public static Schema[] schemaVersions() { * STALE_CONTROLLER_EPOCH (11) * STALE_BROKER_EPOCH (77) */ - private final Errors error; - - private final Map responses; + private final LeaderAndIsrResponseData data; - public LeaderAndIsrResponse(Errors error, Map responses) { - this.responses = responses; - this.error = error; + public LeaderAndIsrResponse(LeaderAndIsrResponseData data) { + this.data = data; } - public LeaderAndIsrResponse(Struct struct) { - responses = new HashMap<>(); - for (Object responseDataObj : struct.get(PARTITIONS)) { - Struct responseData = (Struct) responseDataObj; - String topic = responseData.get(TOPIC_NAME); - int partition = responseData.get(PARTITION_ID); - Errors error = Errors.forCode(responseData.get(ERROR_CODE)); - responses.put(new TopicPartition(topic, partition), error); - } - - error = Errors.forCode(struct.get(ERROR_CODE)); + public LeaderAndIsrResponse(Struct struct, short version) { + this.data = new LeaderAndIsrResponseData(struct, version); } - public Map responses() { - return responses; + public List partitions() { + return data.partitionErrors(); } public Errors error() { - return error; + return Errors.forCode(data.errorCode()); } @Override public Map errorCounts() { + Errors error = error(); if (error != Errors.NONE) // Minor optimization since the top-level error applies to all partitions - return Collections.singletonMap(error, responses.size()); - return errorCounts(responses); + return Collections.singletonMap(error, data.partitionErrors().size()); + return errorCounts(data.partitionErrors().stream().map(l -> Errors.forCode(l.errorCode())).collect(Collectors.toList())); } public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) { - return new LeaderAndIsrResponse(ApiKeys.LEADER_AND_ISR.parseResponse(version, buffer)); + return new LeaderAndIsrResponse(ApiKeys.LEADER_AND_ISR.parseResponse(version, buffer), version); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ApiKeys.LEADER_AND_ISR.responseSchema(version)); - - List responseDatas = new ArrayList<>(responses.size()); - for (Map.Entry response : responses.entrySet()) { - Struct partitionData = struct.instance(PARTITIONS); - TopicPartition partition = response.getKey(); - partitionData.set(TOPIC_NAME, partition.topic()); - partitionData.set(PARTITION_ID, partition.partition()); - partitionData.set(ERROR_CODE, response.getValue().code()); - responseDatas.add(partitionData); - } - - struct.set(PARTITIONS, responseDatas.toArray()); - struct.set(ERROR_CODE, error.code()); - - return struct; + return data.toStruct(version); } @Override public String toString() { - return "LeaderAndIsrResponse(" + - "responses=" + responses + - ", error=" + error + - ")"; + return data.toString(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java index b0e8860f8c10..caa6f47906bf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java @@ -68,24 +68,13 @@ public ListGroupsRequest(Struct struct, short version) { @Override public ListGroupsResponse getErrorResponse(int throttleTimeMs, Throwable e) { - short versionId = version(); - switch (versionId) { - case 0: - return new ListGroupsResponse(new ListGroupsResponseData() - .setGroups(Collections.emptyList()) - .setErrorCode(Errors.forException(e).code()) - ); - case 1: - case 2: - return new ListGroupsResponse(new ListGroupsResponseData() - .setGroups(Collections.emptyList()) - .setErrorCode(Errors.forException(e).code()) - .setThrottleTimeMs(throttleTimeMs) - ); - default: - throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ApiKeys.LIST_GROUPS.latestVersion())); + ListGroupsResponseData listGroupsResponseData = new ListGroupsResponseData(). + setGroups(Collections.emptyList()). + setErrorCode(Errors.forException(e).code()); + if (version() >= 1) { + listGroupsResponseData.setThrottleTimeMs(throttleTimeMs); } + return new ListGroupsResponse(listGroupsResponseData); } public static ListGroupsRequest parse(ByteBuffer buffer, short version) { @@ -94,6 +83,6 @@ public static ListGroupsRequest parse(ByteBuffer buffer, short version) { @Override protected Struct toStruct() { - return new Struct(ApiKeys.LIST_GROUPS.requestSchema(version())); + return data.toStruct(version()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 9f30d3b911d0..3adeb157c188 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -27,6 +27,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; /** @@ -94,14 +95,13 @@ public OffsetCommitResponseData data() { @Override public Map errorCounts() { - Map errorMap = new HashMap<>(); + List errors = new ArrayList<>(); for (OffsetCommitResponseTopic topic : data.topics()) { for (OffsetCommitResponsePartition partition : topic.partitions()) { - errorMap.put(new TopicPartition(topic.name(), partition.partitionIndex()), - Errors.forCode(partition.errorCode())); + errors.add(Errors.forCode(partition.errorCode())); } } - return errorCounts(errorMap); + return errorCounts(errors); } public static OffsetCommitResponse parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteRequest.java new file mode 100644 index 000000000000..293efc2f6f2d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteRequest.java @@ -0,0 +1,83 @@ +/* + * 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.OffsetDeleteRequestData; +import org.apache.kafka.common.message.OffsetDeleteResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class OffsetDeleteRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final OffsetDeleteRequestData data; + + public Builder(OffsetDeleteRequestData data) { + super(ApiKeys.OFFSET_DELETE); + this.data = data; + } + + @Override + public OffsetDeleteRequest build(short version) { + return new OffsetDeleteRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + public final OffsetDeleteRequestData data; + + public OffsetDeleteRequest(OffsetDeleteRequestData data, short version) { + super(ApiKeys.OFFSET_DELETE, version); + this.data = data; + } + + public OffsetDeleteRequest(Struct struct, short version) { + super(ApiKeys.OFFSET_DELETE, version); + this.data = new OffsetDeleteRequestData(struct, version); + } + + public AbstractResponse getErrorResponse(int throttleTimeMs, Errors error) { + return new OffsetDeleteResponse( + new OffsetDeleteResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code()) + ); + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + return getErrorResponse(throttleTimeMs, Errors.forException(e)); + } + + public static OffsetDeleteRequest parse(ByteBuffer buffer, short version) { + return new OffsetDeleteRequest(ApiKeys.OFFSET_DELETE.parseRequest(version, buffer), + version); + } + + @Override + protected Struct toStruct() { + return data.toStruct(version()); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java new file mode 100644 index 000000000000..ff0c2967be89 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java @@ -0,0 +1,95 @@ +/* + * 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.OffsetDeleteResponseData; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartition; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +/** + * Possible error codes: + * + * - Partition errors: + * - {@link Errors#GROUP_SUBSCRIBED_TO_TOPIC} + * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} + * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} + * + * - Group or coordinator errors: + * - {@link Errors#COORDINATOR_LOAD_IN_PROGRESS} + * - {@link Errors#COORDINATOR_NOT_AVAILABLE} + * - {@link Errors#NOT_COORDINATOR} + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#INVALID_GROUP_ID} + * - {@link Errors#GROUP_ID_NOT_FOUND} + * - {@link Errors#NON_EMPTY_GROUP} + */ +public class OffsetDeleteResponse extends AbstractResponse { + + public final OffsetDeleteResponseData data; + + public OffsetDeleteResponse(OffsetDeleteResponseData data) { + this.data = data; + } + + public OffsetDeleteResponse(Struct struct) { + short latestVersion = (short) (OffsetDeleteResponseData.SCHEMAS.length - 1); + this.data = new OffsetDeleteResponseData(struct, latestVersion); + } + + public OffsetDeleteResponse(Struct struct, short version) { + this.data = new OffsetDeleteResponseData(struct, version); + } + + @Override + protected Struct toStruct(short version) { + return data.toStruct(version); + } + + @Override + public Map errorCounts() { + Map counts = new HashMap<>(); + counts.put(Errors.forCode(data.errorCode()), 1); + for (OffsetDeleteResponseTopic topic : data.topics()) { + for (OffsetDeleteResponsePartition partition : topic.partitions()) { + Errors error = Errors.forCode(partition.errorCode()); + counts.put(error, counts.getOrDefault(error, 0) + 1); + } + } + return counts; + } + + public static OffsetDeleteResponse parse(ByteBuffer buffer, short version) { + return new OffsetDeleteResponse(ApiKeys.OFFSET_DELETE.parseResponse(version, buffer)); + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public boolean shouldClientThrottle(short version) { + return version >= 0; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index f616ac1df810..2303f89f9198 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -118,10 +118,8 @@ public OffsetFetchResponse getErrorResponse(Errors error) { } public OffsetFetchResponse getErrorResponse(int throttleTimeMs, Errors error) { - short versionId = version(); - Map responsePartitions = new HashMap<>(); - if (versionId < 2) { + if (version() < 2) { OffsetFetchResponse.PartitionData partitionError = new OffsetFetchResponse.PartitionData( OffsetFetchResponse.INVALID_OFFSET, Optional.empty(), @@ -136,18 +134,10 @@ public OffsetFetchResponse getErrorResponse(int throttleTimeMs, Errors error) { } } - switch (versionId) { - case 0: - case 1: - case 2: - return new OffsetFetchResponse(error, responsePartitions); - case 3: - case 4: - case 5: - return new OffsetFetchResponse(throttleTimeMs, error, responsePartitions); - default: - throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ApiKeys.OFFSET_FETCH.latestVersion())); + if (version() >= 3) { + return new OffsetFetchResponse(throttleTimeMs, error, responsePartitions); + } else { + return new OffsetFetchResponse(error, responsePartitions); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java index 4002d2a5f44d..c067444478b0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java @@ -91,7 +91,7 @@ public short apiVersion() { } @Override - public String listener() { + public String listenerName() { return listenerName.value(); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java index e5ec43acebe8..65bc39d230a2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java @@ -17,123 +17,94 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.errors.InvalidRequestException; +import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.message.RequestHeaderData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; -import java.util.Objects; - -import static java.util.Objects.requireNonNull; -import static org.apache.kafka.common.protocol.types.Type.INT16; -import static org.apache.kafka.common.protocol.types.Type.INT32; -import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING; /** * The header for a request in the Kafka protocol */ public class RequestHeader extends AbstractRequestResponse { - private static final String API_KEY_FIELD_NAME = "api_key"; - private static final String API_VERSION_FIELD_NAME = "api_version"; - private static final String CLIENT_ID_FIELD_NAME = "client_id"; - private static final String CORRELATION_ID_FIELD_NAME = "correlation_id"; - - public static final Schema SCHEMA = new Schema( - new Field(API_KEY_FIELD_NAME, INT16, "The id of the request type."), - new Field(API_VERSION_FIELD_NAME, INT16, "The version of the API."), - new Field(CORRELATION_ID_FIELD_NAME, INT32, "A user-supplied integer value that will be passed back with the response"), - new Field(CLIENT_ID_FIELD_NAME, NULLABLE_STRING, "A user specified identifier for the client making the request.", "")); - - // Version 0 of the controlled shutdown API used a non-standard request header (the clientId is missing). - // This can be removed once we drop support for that version. - private static final Schema CONTROLLED_SHUTDOWN_V0_SCHEMA = new Schema( - new Field(API_KEY_FIELD_NAME, INT16, "The id of the request type."), - new Field(API_VERSION_FIELD_NAME, INT16, "The version of the API."), - new Field(CORRELATION_ID_FIELD_NAME, INT32, "A user-supplied integer value that will be passed back with the response")); - - private final ApiKeys apiKey; - private final short apiVersion; - private final String clientId; - private final int correlationId; - - public RequestHeader(Struct struct) { - short apiKey = struct.getShort(API_KEY_FIELD_NAME); - if (!ApiKeys.hasId(apiKey)) - throw new InvalidRequestException("Unknown API key " + apiKey); - - this.apiKey = ApiKeys.forId(apiKey); - apiVersion = struct.getShort(API_VERSION_FIELD_NAME); - - // only v0 of the controlled shutdown request is missing the clientId - if (struct.hasField(CLIENT_ID_FIELD_NAME)) - clientId = struct.getString(CLIENT_ID_FIELD_NAME); - else - clientId = ""; - correlationId = struct.getInt(CORRELATION_ID_FIELD_NAME); + private final RequestHeaderData data; + private final short headerVersion; + + public RequestHeader(Struct struct, short headerVersion) { + this(new RequestHeaderData(struct, headerVersion), headerVersion); + } + + public RequestHeader(ApiKeys requestApiKey, short requestVersion, String clientId, int correlationId) { + this(new RequestHeaderData(). + setRequestApiKey(requestApiKey.id). + setRequestApiVersion(requestVersion). + setClientId(clientId). + setCorrelationId(correlationId), + ApiKeys.forId(requestApiKey.id).headerVersion(requestVersion)); } - public RequestHeader(ApiKeys apiKey, short version, String clientId, int correlation) { - this.apiKey = requireNonNull(apiKey); - this.apiVersion = version; - this.clientId = clientId; - this.correlationId = correlation; + public RequestHeader(RequestHeaderData data, short headerVersion) { + this.data = data; + this.headerVersion = headerVersion; } public Struct toStruct() { - Schema schema = schema(apiKey.id, apiVersion); - Struct struct = new Struct(schema); - struct.set(API_KEY_FIELD_NAME, apiKey.id); - struct.set(API_VERSION_FIELD_NAME, apiVersion); - - // only v0 of the controlled shutdown request is missing the clientId - if (struct.hasField(CLIENT_ID_FIELD_NAME)) - struct.set(CLIENT_ID_FIELD_NAME, clientId); - struct.set(CORRELATION_ID_FIELD_NAME, correlationId); - return struct; + return this.data.toStruct(headerVersion); } public ApiKeys apiKey() { - return apiKey; + return ApiKeys.forId(data.requestApiKey()); } public short apiVersion() { - return apiVersion; + return data.requestApiVersion(); + } + + public short headerVersion() { + return headerVersion; } public String clientId() { - return clientId; + return data.clientId(); } public int correlationId() { - return correlationId; + return data.correlationId(); + } + + public RequestHeaderData data() { + return data; } public ResponseHeader toResponseHeader() { - return new ResponseHeader(correlationId); + return new ResponseHeader(data.correlationId(), headerVersion); } public static RequestHeader parse(ByteBuffer buffer) { + short apiKey = -1; try { - short apiKey = buffer.getShort(); + apiKey = buffer.getShort(); short apiVersion = buffer.getShort(); - Schema schema = schema(apiKey, apiVersion); + short headerVersion = ApiKeys.forId(apiKey).headerVersion(apiVersion); buffer.rewind(); - return new RequestHeader(schema.read(buffer)); - } catch (InvalidRequestException e) { - throw e; - } catch (Throwable ex) { + return new RequestHeader(new RequestHeaderData( + new ByteBufferAccessor(buffer), headerVersion), headerVersion); + } catch (UnsupportedVersionException e) { + throw new InvalidRequestException("Unknown API key " + apiKey, e); + } catch (Throwable ex) { throw new InvalidRequestException("Error parsing request header. Our best guess of the apiKey is: " + - buffer.getShort(0), ex); + apiKey, ex); } } @Override public String toString() { - return "RequestHeader(apiKey=" + apiKey + - ", apiVersion=" + apiVersion + - ", clientId=" + clientId + - ", correlationId=" + correlationId + + return "RequestHeader(apiKey=" + apiKey() + + ", apiVersion=" + apiVersion() + + ", clientId=" + clientId() + + ", correlationId=" + correlationId() + ")"; } @@ -141,28 +112,12 @@ public String toString() { public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - RequestHeader that = (RequestHeader) o; - return apiKey == that.apiKey && - apiVersion == that.apiVersion && - correlationId == that.correlationId && - Objects.equals(clientId, that.clientId); + return this.data.equals(that.data); } @Override public int hashCode() { - int result = apiKey.hashCode(); - result = 31 * result + (int) apiVersion; - result = 31 * result + (clientId != null ? clientId.hashCode() : 0); - result = 31 * result + correlationId; - return result; - } - - private static Schema schema(short apiKey, short version) { - if (apiKey == ApiKeys.CONTROLLED_SHUTDOWN.id && version == 0) - // This will be removed once we remove support for v0 of ControlledShutdownRequest, which - // depends on a non-standard request header (it does not have a clientId) - return CONTROLLED_SHUTDOWN_V0_SCHEMA; - return SCHEMA; + return this.data.hashCode(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java index fe452a2dc080..249b5d021081 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java @@ -16,31 +16,30 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.protocol.types.BoundField; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.message.ResponseHeaderData; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; -import static org.apache.kafka.common.protocol.types.Type.INT32; - /** * A response header in the kafka protocol. */ public class ResponseHeader extends AbstractRequestResponse { - public static final Schema SCHEMA = new Schema( - new Field("correlation_id", INT32, "The user-supplied value passed in with the request")); - private static final BoundField CORRELATION_KEY_FIELD = SCHEMA.get("correlation_id"); + private final ResponseHeaderData data; + private final short headerVersion; - private final int correlationId; + public ResponseHeader(Struct struct, short headerVersion) { + this(new ResponseHeaderData(struct, headerVersion), headerVersion); + } - public ResponseHeader(Struct struct) { - correlationId = struct.getInt(CORRELATION_KEY_FIELD); + public ResponseHeader(int correlationId, short headerVersion) { + this(new ResponseHeaderData().setCorrelationId(correlationId), headerVersion); } - public ResponseHeader(int correlationId) { - this.correlationId = correlationId; + public ResponseHeader(ResponseHeaderData data, short headerVersion) { + this.data = data; + this.headerVersion = headerVersion; } public int sizeOf() { @@ -48,17 +47,24 @@ public int sizeOf() { } public Struct toStruct() { - Struct struct = new Struct(SCHEMA); - struct.set(CORRELATION_KEY_FIELD, correlationId); - return struct; + return data.toStruct(headerVersion); } public int correlationId() { - return correlationId; + return this.data.correlationId(); } - public static ResponseHeader parse(ByteBuffer buffer) { - return new ResponseHeader(SCHEMA.read(buffer)); + public short headerVersion() { + return headerVersion; } + public ResponseHeaderData data() { + return data; + } + + public static ResponseHeader parse(ByteBuffer buffer, short headerVersion) { + return new ResponseHeader( + new ResponseHeaderData(new ByteBufferAccessor(buffer), headerVersion), + headerVersion); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java index bad9573a1575..e79228917fb3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java @@ -17,57 +17,27 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.StopReplicaRequestData; +import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionV0; +import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaTopic; +import org.apache.kafka.common.message.StopReplicaResponseData; +import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.FlattenedIterator; +import org.apache.kafka.common.utils.MappedIterator; import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; - -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; -import static org.apache.kafka.common.protocol.types.Type.INT32; +import java.util.stream.Collectors; public class StopReplicaRequest extends AbstractControlRequest { - private static final Field.Bool DELETE_PARTITIONS = new Field.Bool("delete_partitions", "Boolean which indicates if replica's partitions must be deleted."); - private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions", "The partitions"); - private static final Field.Array PARTITION_IDS = new Field.Array("partition_ids", INT32, "The partition ids of a topic"); - - private static final Field PARTITIONS_V0 = PARTITIONS.withFields( - TOPIC_NAME, - PARTITION_ID); - private static final Field PARTITIONS_V1 = PARTITIONS.withFields( - TOPIC_NAME, - PARTITION_IDS); - - private static final Schema STOP_REPLICA_REQUEST_V0 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - DELETE_PARTITIONS, - PARTITIONS_V0); - - // STOP_REPLICA_REQUEST_V1 added a broker_epoch Field. This field specifies the generation of the broker across - // bounces. It also normalizes partitions under each topic. - private static final Schema STOP_REPLICA_REQUEST_V1 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - BROKER_EPOCH, - DELETE_PARTITIONS, - PARTITIONS_V1); - - - public static Schema[] schemaVersions() { - return new Schema[] {STOP_REPLICA_REQUEST_V0, STOP_REPLICA_REQUEST_V1}; - } public static class Builder extends AbstractControlRequest.Builder { private final boolean deletePartitions; @@ -80,10 +50,31 @@ public Builder(short version, int controllerId, int controllerEpoch, long broker this.partitions = partitions; } - @Override public StopReplicaRequest build(short version) { - return new StopReplicaRequest(controllerId, controllerEpoch, brokerEpoch, - deletePartitions, partitions, version); + StopReplicaRequestData data = new StopReplicaRequestData() + .setControllerId(controllerId) + .setControllerEpoch(controllerEpoch) + .setBrokerEpoch(brokerEpoch) + .setDeletePartitions(deletePartitions); + + if (version >= 1) { + Map> topicPartitionsMap = CollectionUtils.groupPartitionsByTopic(partitions); + List topics = topicPartitionsMap.entrySet().stream().map(entry -> + new StopReplicaTopic() + .setName(entry.getKey()) + .setPartitionIndexes(entry.getValue()) + ).collect(Collectors.toList()); + data.setTopics(topics); + } else { + List requestPartitions = partitions.stream().map(tp -> + new StopReplicaPartitionV0() + .setTopicName(tp.topic()) + .setPartitionIndex(tp.partition()) + ).collect(Collectors.toList()); + data.setUngroupedPartitions(requestPartitions); + } + + return new StopReplicaRequest(data, version); } @Override @@ -100,54 +91,37 @@ public String toString() { } } - private final boolean deletePartitions; - private final Collection partitions; + private final StopReplicaRequestData data; - private StopReplicaRequest(int controllerId, int controllerEpoch, long brokerEpoch, boolean deletePartitions, - Collection partitions, short version) { - super(ApiKeys.STOP_REPLICA, version, controllerId, controllerEpoch, brokerEpoch); - this.deletePartitions = deletePartitions; - this.partitions = partitions; + private StopReplicaRequest(StopReplicaRequestData data, short version) { + super(ApiKeys.STOP_REPLICA, version); + this.data = data; } public StopReplicaRequest(Struct struct, short version) { - super(ApiKeys.STOP_REPLICA, struct, version); - - partitions = new HashSet<>(); - if (version > 0) { // V1 - for (Object topicObj : struct.get(PARTITIONS)) { - Struct topicData = (Struct) topicObj; - String topic = topicData.get(TOPIC_NAME); - for (Object partitionObj : topicData.get(PARTITION_IDS)) { - int partition = (Integer) partitionObj; - partitions.add(new TopicPartition(topic, partition)); - } - } - } else { // V0 - for (Object partitionDataObj : struct.get(PARTITIONS)) { - Struct partitionData = (Struct) partitionDataObj; - String topic = partitionData.get(TOPIC_NAME); - int partition = partitionData.get(PARTITION_ID); - partitions.add(new TopicPartition(topic, partition)); - } - } - deletePartitions = struct.get(DELETE_PARTITIONS); + this(new StopReplicaRequestData(struct, version), version); } @Override public StopReplicaResponse getErrorResponse(int throttleTimeMs, Throwable e) { Errors error = Errors.forException(e); - Map responses = new HashMap<>(partitions.size()); - for (TopicPartition partition : partitions) { - responses.put(partition, error); + StopReplicaResponseData data = new StopReplicaResponseData(); + data.setErrorCode(error.code()); + List partitions = new ArrayList<>(); + for (TopicPartition tp : partitions()) { + partitions.add(new StopReplicaPartitionError() + .setTopicName(tp.topic()) + .setPartitionIndex(tp.partition()) + .setErrorCode(error.code())); } + data.setPartitionErrors(partitions); short versionId = version(); switch (versionId) { case 0: case 1: - return new StopReplicaResponse(error, responses); + return new StopReplicaResponse(data); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ApiKeys.STOP_REPLICA.latestVersion())); @@ -155,11 +129,40 @@ public StopReplicaResponse getErrorResponse(int throttleTimeMs, Throwable e) { } public boolean deletePartitions() { - return deletePartitions; + return data.deletePartitions(); + } + + /** + * Note that this method has allocation overhead per iterated element, so callers should copy the result into + * another collection if they need to iterate more than once. + * + * Implementation note: we should strive to avoid allocation overhead per element, see + * `UpdateMetadataRequest.partitionStates()` for the preferred approach. That's not possible in this case and + * StopReplicaRequest should be relatively rare in comparison to other request types. + */ + public Iterable partitions() { + if (version() >= 1) { + return () -> new FlattenedIterator<>(data.topics().iterator(), topic -> + new MappedIterator<>(topic.partitionIndexes().iterator(), partition -> + new TopicPartition(topic.name(), partition))); + } + return () -> new MappedIterator<>(data.ungroupedPartitions().iterator(), + partition -> new TopicPartition(partition.topicName(), partition.partitionIndex())); + } + + @Override + public int controllerId() { + return data.controllerId(); + } + + @Override + public int controllerEpoch() { + return data.controllerEpoch(); } - public Collection partitions() { - return partitions; + @Override + public long brokerEpoch() { + return data.brokerEpoch(); } public static StopReplicaRequest parse(ByteBuffer buffer, short version) { @@ -168,34 +171,11 @@ public static StopReplicaRequest parse(ByteBuffer buffer, short version) { @Override protected Struct toStruct() { - Struct struct = new Struct(ApiKeys.STOP_REPLICA.requestSchema(version())); - - struct.set(CONTROLLER_ID, controllerId); - struct.set(CONTROLLER_EPOCH, controllerEpoch); - struct.setIfExists(BROKER_EPOCH, brokerEpoch); - struct.set(DELETE_PARTITIONS, deletePartitions); - - if (version() > 0) { // V1 - Map> topicPartitionsMap = CollectionUtils.groupPartitionsByTopic(partitions); - List topicsData = new ArrayList<>(topicPartitionsMap.size()); - for (Map.Entry> entry : topicPartitionsMap.entrySet()) { - Struct topicData = struct.instance(PARTITIONS); - topicData.set(TOPIC_NAME, entry.getKey()); - topicData.set(PARTITION_IDS, entry.getValue().toArray()); - topicsData.add(topicData); - } - struct.set(PARTITIONS, topicsData.toArray()); - - } else { // V0 - List partitionDatas = new ArrayList<>(partitions.size()); - for (TopicPartition partition : partitions) { - Struct partitionData = struct.instance(PARTITIONS); - partitionData.set(TOPIC_NAME, partition.topic()); - partitionData.set(PARTITION_ID, partition.partition()); - partitionDatas.add(partitionData); - } - struct.set(PARTITIONS, partitionDatas.toArray()); - } - return struct; + return data.toStruct(version()); } + + protected long size() { + return data.size(version()); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java index 6089bea8d4f0..7d6c7a074289 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java @@ -16,43 +16,19 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.StopReplicaResponseData; +import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; - -import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; +import java.util.stream.Collectors; public class StopReplicaResponse extends AbstractResponse { - private static final Field.ComplexArray PARTITIONS = new Field.ComplexArray("partitions", "Response for the requests partitions"); - - private static final Field PARTITIONS_V0 = PARTITIONS.withFields( - TOPIC_NAME, - PARTITION_ID, - ERROR_CODE); - private static final Schema STOP_REPLICA_RESPONSE_V0 = new Schema( - ERROR_CODE, - PARTITIONS_V0); - - private static final Schema STOP_REPLICA_RESPONSE_V1 = STOP_REPLICA_RESPONSE_V0; - - - public static Schema[] schemaVersions() { - return new Schema[] {STOP_REPLICA_RESPONSE_V0, STOP_REPLICA_RESPONSE_V1}; - } - - private final Map responses; /** * Possible error code: @@ -60,71 +36,44 @@ public static Schema[] schemaVersions() { * STALE_CONTROLLER_EPOCH (11) * STALE_BROKER_EPOCH (77) */ - private final Errors error; + private final StopReplicaResponseData data; - public StopReplicaResponse(Errors error, Map responses) { - this.responses = responses; - this.error = error; + public StopReplicaResponse(StopReplicaResponseData data) { + this.data = data; } - public StopReplicaResponse(Struct struct) { - responses = new HashMap<>(); - for (Object responseDataObj : struct.get(PARTITIONS)) { - Struct responseData = (Struct) responseDataObj; - String topic = responseData.get(TOPIC_NAME); - int partition = responseData.get(PARTITION_ID); - Errors error = Errors.forCode(responseData.get(ERROR_CODE)); - responses.put(new TopicPartition(topic, partition), error); - } - - error = Errors.forCode(struct.get(ERROR_CODE)); + public StopReplicaResponse(Struct struct, short version) { + data = new StopReplicaResponseData(struct, version); } - public Map responses() { - return responses; + public List partitionErrors() { + return data.partitionErrors(); } public Errors error() { - return error; + return Errors.forCode(data.errorCode()); } @Override public Map errorCounts() { - if (error != Errors.NONE) + if (data.errorCode() != Errors.NONE.code()) // Minor optimization since the top-level error applies to all partitions - return Collections.singletonMap(error, responses.size()); - return errorCounts(responses); + return Collections.singletonMap(error(), data.partitionErrors().size()); + return errorCounts(data.partitionErrors().stream().map(p -> Errors.forCode(p.errorCode())).collect(Collectors.toList())); } public static StopReplicaResponse parse(ByteBuffer buffer, short version) { - return new StopReplicaResponse(ApiKeys.STOP_REPLICA.parseResponse(version, buffer)); + return new StopReplicaResponse(ApiKeys.STOP_REPLICA.parseResponse(version, buffer), version); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ApiKeys.STOP_REPLICA.responseSchema(version)); - - List responseDatas = new ArrayList<>(responses.size()); - for (Map.Entry response : responses.entrySet()) { - Struct partitionData = struct.instance(PARTITIONS); - TopicPartition partition = response.getKey(); - partitionData.set(TOPIC_NAME, partition.topic()); - partitionData.set(PARTITION_ID, partition.partition()); - partitionData.set(ERROR_CODE, response.getValue().code()); - responseDatas.add(partitionData); - } - - struct.set(PARTITIONS, responseDatas.toArray()); - struct.set(ERROR_CODE, error.code()); - return struct; + return data.toStruct(version); } @Override public String toString() { - return "StopReplicaResponse(" + - "responses=" + responses + - ", error=" + error + - ")"; + return data.toString(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java index 1851f89f831d..256b9983671e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java @@ -88,7 +88,7 @@ public int throttleTimeMs() { @Override public Map errorCounts() { - return errorCounts(errors()); + return errorCounts(errors().values()); } public Map errors() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index b715c2aa65b6..b73fa5b5c1a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -16,177 +16,38 @@ */ 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.UpdateMetadataRequestData; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataBroker; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataEndpoint; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataTopicState; +import org.apache.kafka.common.message.UpdateMetadataResponseData; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.FlattenedIterator; import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; -import static org.apache.kafka.common.protocol.types.Type.INT32; +import static java.util.Collections.singletonList; public class UpdateMetadataRequest extends AbstractControlRequest { - private static final Field.ComplexArray TOPIC_STATES = new Field.ComplexArray("topic_states", "Topic states"); - private static final Field.ComplexArray PARTITION_STATES = new Field.ComplexArray("partition_states", "Partition states"); - private static final Field.ComplexArray LIVE_BROKERS = new Field.ComplexArray("live_brokers", "Live broekrs"); - - // PartitionState fields - private static final Field.Int32 LEADER = new Field.Int32("leader", "The broker id for the leader."); - private static final Field.Int32 LEADER_EPOCH = new Field.Int32("leader_epoch", "The leader epoch."); - private static final Field.Array ISR = new Field.Array("isr", INT32, "The in sync replica ids."); - private static final Field.Int32 ZK_VERSION = new Field.Int32("zk_version", "The ZK version."); - private static final Field.Array REPLICAS = new Field.Array("replicas", INT32, "The replica ids."); - private static final Field.Array OFFLINE_REPLICAS = new Field.Array("offline_replicas", INT32, "The offline replica ids"); - - // Live brokers fields - private static final Field.Int32 BROKER_ID = new Field.Int32("id", "The broker id"); - private static final Field.ComplexArray ENDPOINTS = new Field.ComplexArray("end_points", "The endpoints"); - private static final Field.NullableStr RACK = new Field.NullableStr("rack", "The rack"); - - // EndPoint fields - private static final Field.Str HOST = new Field.Str("host", "The hostname of the broker."); - private static final Field.Int32 PORT = new Field.Int32("port", "The port on which the broker accepts requests."); - private static final Field.Str LISTENER_NAME = new Field.Str("listener_name", "The listener name."); - private static final Field.Int16 SECURITY_PROTOCOL_TYPE = new Field.Int16("security_protocol_type", "The security protocol type."); - - private static final Field PARTITION_STATES_V0 = PARTITION_STATES.withFields( - TOPIC_NAME, - PARTITION_ID, - CONTROLLER_EPOCH, - LEADER, - LEADER_EPOCH, - ISR, - ZK_VERSION, - REPLICAS); - - // PARTITION_STATES_V4 added a per-partition offline_replicas field. This field specifies - // the list of replicas that are offline. - private static final Field PARTITION_STATES_V4 = PARTITION_STATES.withFields( - TOPIC_NAME, - PARTITION_ID, - CONTROLLER_EPOCH, - LEADER, - LEADER_EPOCH, - ISR, - ZK_VERSION, - REPLICAS, - OFFLINE_REPLICAS); - - private static final Field PARTITION_STATES_V5 = PARTITION_STATES.withFields( - PARTITION_ID, - CONTROLLER_EPOCH, - LEADER, - LEADER_EPOCH, - ISR, - ZK_VERSION, - REPLICAS, - OFFLINE_REPLICAS); - - // TOPIC_STATES_V5 normalizes TOPIC_STATES_V4 to - // make it more memory efficient - private static final Field TOPIC_STATES_V5 = TOPIC_STATES.withFields( - TOPIC_NAME, - PARTITION_STATES_V5); - - // for some reason, V1 sends `port` before `host` while V0 sends `host` before `port - private static final Field ENDPOINTS_V1 = ENDPOINTS.withFields( - PORT, - HOST, - SECURITY_PROTOCOL_TYPE); - - private static final Field ENDPOINTS_V3 = ENDPOINTS.withFields( - PORT, - HOST, - LISTENER_NAME, - SECURITY_PROTOCOL_TYPE); - - private static final Field LIVE_BROKERS_V0 = LIVE_BROKERS.withFields( - BROKER_ID, - HOST, - PORT); - - private static final Field LIVE_BROKERS_V1 = LIVE_BROKERS.withFields( - BROKER_ID, - ENDPOINTS_V1); - - private static final Field LIVE_BROKERS_V2 = LIVE_BROKERS.withFields( - BROKER_ID, - ENDPOINTS_V1, - RACK); - - private static final Field LIVE_BROKERS_V3 = LIVE_BROKERS.withFields( - BROKER_ID, - ENDPOINTS_V3, - RACK); - - private static final Schema UPDATE_METADATA_REQUEST_V0 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - PARTITION_STATES_V0, - LIVE_BROKERS_V0); - - private static final Schema UPDATE_METADATA_REQUEST_V1 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - PARTITION_STATES_V0, - LIVE_BROKERS_V1); - - private static final Schema UPDATE_METADATA_REQUEST_V2 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - PARTITION_STATES_V0, - LIVE_BROKERS_V2); - - - private static final Schema UPDATE_METADATA_REQUEST_V3 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - PARTITION_STATES_V0, - LIVE_BROKERS_V3); - - // UPDATE_METADATA_REQUEST_V4 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline. - private static final Schema UPDATE_METADATA_REQUEST_V4 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - PARTITION_STATES_V4, - LIVE_BROKERS_V3); - - // UPDATE_METADATA_REQUEST_V5 added a broker_epoch Field. This field specifies the generation of the broker across - // bounces. It also normalizes partitions under each topic. - private static final Schema UPDATE_METADATA_REQUEST_V5 = new Schema( - CONTROLLER_ID, - CONTROLLER_EPOCH, - BROKER_EPOCH, - TOPIC_STATES_V5, - LIVE_BROKERS_V3); - - public static Schema[] schemaVersions() { - return new Schema[] {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1, UPDATE_METADATA_REQUEST_V2, - UPDATE_METADATA_REQUEST_V3, UPDATE_METADATA_REQUEST_V4, UPDATE_METADATA_REQUEST_V5}; - } public static class Builder extends AbstractControlRequest.Builder { - private final Map partitionStates; - private final Set liveBrokers; + private final List partitionStates; + private final List liveBrokers; public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch, - Map partitionStates, Set liveBrokers) { + List partitionStates, List liveBrokers) { super(ApiKeys.UPDATE_METADATA, version, controllerId, controllerEpoch, brokerEpoch); this.partitionStates = partitionStates; this.liveBrokers = liveBrokers; @@ -194,14 +55,53 @@ public Builder(short version, int controllerId, int controllerEpoch, long broker @Override public UpdateMetadataRequest build(short version) { - if (version == 0) { - for (Broker broker : liveBrokers) { - if (broker.endPoints.size() != 1 || broker.endPoints.get(0).securityProtocol != SecurityProtocol.PLAINTEXT) { - throw new UnsupportedVersionException("UpdateMetadataRequest v0 only handles PLAINTEXT endpoints"); + if (version < 3) { + for (UpdateMetadataBroker broker : liveBrokers) { + if (version == 0) { + if (broker.endpoints().size() != 1) + throw new UnsupportedVersionException("UpdateMetadataRequest v0 requires a single endpoint"); + if (broker.endpoints().get(0).securityProtocol() != SecurityProtocol.PLAINTEXT.id) + throw new UnsupportedVersionException("UpdateMetadataRequest v0 only handles PLAINTEXT endpoints"); + // Don't null out `endpoints` since it's ignored by the generated code if version >= 1 + UpdateMetadataEndpoint endpoint = broker.endpoints().get(0); + broker.setV0Host(endpoint.host()); + broker.setV0Port(endpoint.port()); + } else { + if (broker.endpoints().stream().anyMatch(endpoint -> !endpoint.listener().isEmpty() && + !endpoint.listener().equals(listenerNameFromSecurityProtocol(endpoint)))) { + throw new UnsupportedVersionException("UpdateMetadataRequest v0-v3 does not support custom " + + "listeners, request version: " + version + ", endpoints: " + broker.endpoints()); + } } } } - return new UpdateMetadataRequest(version, controllerId, controllerEpoch, brokerEpoch, partitionStates, liveBrokers); + + UpdateMetadataRequestData data = new UpdateMetadataRequestData() + .setControllerId(controllerId) + .setControllerEpoch(controllerEpoch) + .setBrokerEpoch(brokerEpoch) + .setLiveBrokers(liveBrokers); + + if (version >= 5) { + Map topicStatesMap = groupByTopic(partitionStates); + data.setTopicStates(new ArrayList<>(topicStatesMap.values())); + } else { + data.setUngroupedPartitionStates(partitionStates); + } + + return new UpdateMetadataRequest(data, version); + } + + private static Map groupByTopic(List partitionStates) { + Map topicStates = new HashMap<>(); + for (UpdateMetadataPartitionState partition : partitionStates) { + // We don't null out the topic name in UpdateMetadataTopicState since it's ignored by the generated + // code if version >= 5 + UpdateMetadataTopicState topicState = topicStates.computeIfAbsent(partition.topicName(), + t -> new UpdateMetadataTopicState().setTopicName(partition.topicName())); + topicState.partitionStates().add(partition); + } + return topicStates; } @Override @@ -218,275 +118,110 @@ public String toString() { } } - public static final class PartitionState { - public final BasePartitionState basePartitionState; - public final List offlineReplicas; - - public PartitionState(int controllerEpoch, - int leader, - int leaderEpoch, - List isr, - int zkVersion, - List replicas, - List offlineReplicas) { - this.basePartitionState = new BasePartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas); - this.offlineReplicas = offlineReplicas; - } - - private PartitionState(Struct struct) { - int controllerEpoch = struct.get(CONTROLLER_EPOCH); - int leader = struct.get(LEADER); - int leaderEpoch = struct.get(LEADER_EPOCH); - - Object[] isrArray = struct.get(ISR); - List isr = new ArrayList<>(isrArray.length); - for (Object r : isrArray) - isr.add((Integer) r); - - int zkVersion = struct.get(ZK_VERSION); - - Object[] replicasArray = struct.get(REPLICAS); - List replicas = new ArrayList<>(replicasArray.length); - for (Object r : replicasArray) - replicas.add((Integer) r); + private final UpdateMetadataRequestData data; - this.basePartitionState = new BasePartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas); + UpdateMetadataRequest(UpdateMetadataRequestData data, short version) { + super(ApiKeys.UPDATE_METADATA, version); + this.data = data; + // Do this from the constructor to make it thread-safe (even though it's only needed when some methods are called) + normalize(); + } - this.offlineReplicas = new ArrayList<>(); - if (struct.hasField(OFFLINE_REPLICAS)) { - Object[] offlineReplicasArray = struct.get(OFFLINE_REPLICAS); - for (Object r : offlineReplicasArray) - offlineReplicas.add((Integer) r); + private void normalize() { + // Version 0 only supported a single host and port and the protocol was always plaintext + // Version 1 added support for multiple endpoints, each with its own security protocol + // Version 2 added support for rack + // Version 3 added support for listener name, which we can infer from the security protocol for older versions + if (version() < 3) { + for (UpdateMetadataBroker liveBroker : data.liveBrokers()) { + // Set endpoints so that callers can rely on it always being present + if (version() == 0 && liveBroker.endpoints().isEmpty()) { + SecurityProtocol securityProtocol = SecurityProtocol.PLAINTEXT; + liveBroker.setEndpoints(singletonList(new UpdateMetadataEndpoint() + .setHost(liveBroker.v0Host()) + .setPort(liveBroker.v0Port()) + .setSecurityProtocol(securityProtocol.id) + .setListener(ListenerName.forSecurityProtocol(securityProtocol).value()))); + } else { + for (UpdateMetadataEndpoint endpoint : liveBroker.endpoints()) { + // Set listener so that callers can rely on it always being present + if (endpoint.listener().isEmpty()) + endpoint.setListener(listenerNameFromSecurityProtocol(endpoint)); + } + } } } - @Override - public String toString() { - return "PartitionState(controllerEpoch=" + basePartitionState.controllerEpoch + - ", leader=" + basePartitionState.leader + - ", leaderEpoch=" + basePartitionState.leaderEpoch + - ", isr=" + Arrays.toString(basePartitionState.isr.toArray()) + - ", zkVersion=" + basePartitionState.zkVersion + - ", replicas=" + Arrays.toString(basePartitionState.replicas.toArray()) + - ", offlineReplicas=" + Arrays.toString(offlineReplicas.toArray()) + ")"; - } - - private void setStruct(Struct struct) { - struct.set(CONTROLLER_EPOCH, basePartitionState.controllerEpoch); - struct.set(LEADER, basePartitionState.leader); - struct.set(LEADER_EPOCH, basePartitionState.leaderEpoch); - struct.set(ISR, basePartitionState.isr.toArray()); - struct.set(ZK_VERSION, basePartitionState.zkVersion); - struct.set(REPLICAS, basePartitionState.replicas.toArray()); - struct.setIfExists(OFFLINE_REPLICAS, offlineReplicas.toArray()); + if (version() >= 5) { + for (UpdateMetadataTopicState topicState : data.topicStates()) { + for (UpdateMetadataPartitionState partitionState : topicState.partitionStates()) { + // Set the topic name so that we can always present the ungrouped view to callers + partitionState.setTopicName(topicState.topicName()); + } + } } } - public static final class Broker { - public final int id; - public final List endPoints; - public final String rack; // introduced in V2 - - public Broker(int id, List endPoints, String rack) { - this.id = id; - this.endPoints = endPoints; - this.rack = rack; - } - - @Override - public String toString() { - StringBuilder bld = new StringBuilder(); - bld.append("(id=").append(id); - bld.append(", endPoints=").append(Utils.join(endPoints, ",")); - bld.append(", rack=").append(rack); - bld.append(")"); - return bld.toString(); - } + private static String listenerNameFromSecurityProtocol(UpdateMetadataEndpoint endpoint) { + SecurityProtocol securityProtocol = SecurityProtocol.forId(endpoint.securityProtocol()); + return ListenerName.forSecurityProtocol(securityProtocol).value(); } - public static final class EndPoint { - public final String host; - public final int port; - public final SecurityProtocol securityProtocol; - public final ListenerName listenerName; // introduced in V3 - - public EndPoint(String host, int port, SecurityProtocol securityProtocol, ListenerName listenerName) { - this.host = host; - this.port = port; - this.securityProtocol = securityProtocol; - this.listenerName = listenerName; - } - - @Override - public String toString() { - return "(host=" + host + ", port=" + port + ", listenerName=" + listenerName + - ", securityProtocol=" + securityProtocol + ")"; - } + public UpdateMetadataRequest(Struct struct, short version) { + this(new UpdateMetadataRequestData(struct, version), version); } - private final Map partitionStates; - private final Set liveBrokers; - - private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch, long brokerEpoch, - Map partitionStates, Set liveBrokers) { - super(ApiKeys.UPDATE_METADATA, version, controllerId, controllerEpoch, brokerEpoch); - this.partitionStates = partitionStates; - this.liveBrokers = liveBrokers; + @Override + public int controllerId() { + return data.controllerId(); } - public UpdateMetadataRequest(Struct struct, short versionId) { - super(ApiKeys.UPDATE_METADATA, struct, versionId); - Map partitionStates = new HashMap<>(); - if (struct.hasField(TOPIC_STATES)) { - for (Object topicStatesDataObj : struct.get(TOPIC_STATES)) { - Struct topicStatesData = (Struct) topicStatesDataObj; - String topic = topicStatesData.get(TOPIC_NAME); - for (Object partitionStateDataObj : topicStatesData.get(PARTITION_STATES)) { - Struct partitionStateData = (Struct) partitionStateDataObj; - int partition = partitionStateData.get(PARTITION_ID); - PartitionState partitionState = new PartitionState(partitionStateData); - partitionStates.put(new TopicPartition(topic, partition), partitionState); - } - } - } else { - for (Object partitionStateDataObj : struct.get(PARTITION_STATES)) { - Struct partitionStateData = (Struct) partitionStateDataObj; - String topic = partitionStateData.get(TOPIC_NAME); - int partition = partitionStateData.get(PARTITION_ID); - PartitionState partitionState = new PartitionState(partitionStateData); - partitionStates.put(new TopicPartition(topic, partition), partitionState); - } - } - - Set liveBrokers = new HashSet<>(); - - for (Object brokerDataObj : struct.get(LIVE_BROKERS)) { - Struct brokerData = (Struct) brokerDataObj; - int brokerId = brokerData.get(BROKER_ID); + @Override + public int controllerEpoch() { + return data.controllerEpoch(); + } - // V0 - if (brokerData.hasField(HOST)) { - String host = brokerData.get(HOST); - int port = brokerData.get(PORT); - List endPoints = new ArrayList<>(1); - SecurityProtocol securityProtocol = SecurityProtocol.PLAINTEXT; - endPoints.add(new EndPoint(host, port, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol))); - liveBrokers.add(new Broker(brokerId, endPoints, null)); - } else { // V1, V2 or V3 - List endPoints = new ArrayList<>(); - for (Object endPointDataObj : brokerData.get(ENDPOINTS)) { - Struct endPointData = (Struct) endPointDataObj; - int port = endPointData.get(PORT); - String host = endPointData.get(HOST); - short protocolTypeId = endPointData.get(SECURITY_PROTOCOL_TYPE); - SecurityProtocol securityProtocol = SecurityProtocol.forId(protocolTypeId); - String listenerName; - if (endPointData.hasField(LISTENER_NAME)) // V3 - listenerName = endPointData.get(LISTENER_NAME); - else - listenerName = securityProtocol.name; - endPoints.add(new EndPoint(host, port, securityProtocol, new ListenerName(listenerName))); - } - String rack = null; - if (brokerData.hasField(RACK)) { // V2 - rack = brokerData.get(RACK); - } - liveBrokers.add(new Broker(brokerId, endPoints, rack)); - } - } - this.partitionStates = partitionStates; - this.liveBrokers = liveBrokers; + @Override + public long brokerEpoch() { + return data.brokerEpoch(); } @Override - protected Struct toStruct() { + public UpdateMetadataResponse getErrorResponse(int throttleTimeMs, Throwable e) { short version = version(); - Struct struct = new Struct(ApiKeys.UPDATE_METADATA.requestSchema(version)); - struct.set(CONTROLLER_ID, controllerId); - struct.set(CONTROLLER_EPOCH, controllerEpoch); - struct.setIfExists(BROKER_EPOCH, brokerEpoch); - - if (struct.hasField(TOPIC_STATES)) { - Map> topicStates = CollectionUtils.groupPartitionDataByTopic(partitionStates); - List topicStatesData = new ArrayList<>(topicStates.size()); - for (Map.Entry> entry : topicStates.entrySet()) { - Struct topicStateData = struct.instance(TOPIC_STATES); - topicStateData.set(TOPIC_NAME, entry.getKey()); - Map partitionMap = entry.getValue(); - List partitionStatesData = new ArrayList<>(partitionMap.size()); - for (Map.Entry partitionEntry : partitionMap.entrySet()) { - Struct partitionStateData = topicStateData.instance(PARTITION_STATES); - partitionStateData.set(PARTITION_ID, partitionEntry.getKey()); - partitionEntry.getValue().setStruct(partitionStateData); - partitionStatesData.add(partitionStateData); - } - topicStateData.set(PARTITION_STATES, partitionStatesData.toArray()); - topicStatesData.add(topicStateData); - } - struct.set(TOPIC_STATES, topicStatesData.toArray()); - } else { - List partitionStatesData = new ArrayList<>(partitionStates.size()); - for (Map.Entry entry : partitionStates.entrySet()) { - Struct partitionStateData = struct.instance(PARTITION_STATES); - TopicPartition topicPartition = entry.getKey(); - partitionStateData.set(TOPIC_NAME, topicPartition.topic()); - partitionStateData.set(PARTITION_ID, topicPartition.partition()); - entry.getValue().setStruct(partitionStateData); - partitionStatesData.add(partitionStateData); - } - struct.set(PARTITION_STATES, partitionStatesData.toArray()); - } - - List brokersData = new ArrayList<>(liveBrokers.size()); - for (Broker broker : liveBrokers) { - Struct brokerData = struct.instance(LIVE_BROKERS); - brokerData.set(BROKER_ID, broker.id); - - if (version == 0) { - EndPoint endPoint = broker.endPoints.get(0); - brokerData.set(HOST, endPoint.host); - brokerData.set(PORT, endPoint.port); - } else { - List endPointsData = new ArrayList<>(broker.endPoints.size()); - for (EndPoint endPoint : broker.endPoints) { - Struct endPointData = brokerData.instance(ENDPOINTS); - endPointData.set(PORT, endPoint.port); - endPointData.set(HOST, endPoint.host); - endPointData.set(SECURITY_PROTOCOL_TYPE, endPoint.securityProtocol.id); - if (version >= 3) - endPointData.set(LISTENER_NAME, endPoint.listenerName.value()); - endPointsData.add(endPointData); - - } - brokerData.set(ENDPOINTS, endPointsData.toArray()); - if (version >= 2) { - brokerData.set(RACK, broker.rack); - } - } + if (version <= 5) + return new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.forException(e).code())); + else + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + version, this.getClass().getSimpleName(), ApiKeys.UPDATE_METADATA.latestVersion())); + } - brokersData.add(brokerData); + public Iterable partitionStates() { + if (version() >= 5) { + return () -> new FlattenedIterator<>(data.topicStates().iterator(), + topicState -> topicState.partitionStates().iterator()); } - struct.set(LIVE_BROKERS, brokersData.toArray()); + return data.ungroupedPartitionStates(); + } - return struct; + public List liveBrokers() { + return data.liveBrokers(); } @Override - public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { - short versionId = version(); - if (versionId <= 5) - return new UpdateMetadataResponse(Errors.forException(e)); - else - throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ApiKeys.UPDATE_METADATA.latestVersion())); + protected Struct toStruct() { + return data.toStruct(version()); } - public Map partitionStates() { - return partitionStates; + protected ByteBuffer toBytes() { + ByteBuffer bytes = ByteBuffer.allocate(size()); + data.write(new ByteBufferAccessor(bytes), version()); + bytes.flip(); + return bytes; } - public Set liveBrokers() { - return liveBrokers; + protected int size() { + return data.size(version()); } public static UpdateMetadataRequest parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java index 478265b39586..c7d803f560eb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java @@ -16,62 +16,41 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.message.UpdateMetadataResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; import java.util.Map; -import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; - public class UpdateMetadataResponse extends AbstractResponse { - private static final Schema UPDATE_METADATA_RESPONSE_V0 = new Schema(ERROR_CODE); - private static final Schema UPDATE_METADATA_RESPONSE_V1 = UPDATE_METADATA_RESPONSE_V0; - private static final Schema UPDATE_METADATA_RESPONSE_V2 = UPDATE_METADATA_RESPONSE_V1; - private static final Schema UPDATE_METADATA_RESPONSE_V3 = UPDATE_METADATA_RESPONSE_V2; - private static final Schema UPDATE_METADATA_RESPONSE_V4 = UPDATE_METADATA_RESPONSE_V3; - private static final Schema UPDATE_METADATA_RESPONSE_V5 = UPDATE_METADATA_RESPONSE_V4; - - public static Schema[] schemaVersions() { - return new Schema[]{UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1, UPDATE_METADATA_RESPONSE_V2, - UPDATE_METADATA_RESPONSE_V3, UPDATE_METADATA_RESPONSE_V4, UPDATE_METADATA_RESPONSE_V5}; - } - /** - * Possible error code: - * - * STALE_CONTROLLER_EPOCH (11) - * STALE_BROKER_EPOCH (77) - */ - private final Errors error; + private final UpdateMetadataResponseData data; - public UpdateMetadataResponse(Errors error) { - this.error = error; + public UpdateMetadataResponse(UpdateMetadataResponseData data) { + this.data = data; } - public UpdateMetadataResponse(Struct struct) { - error = Errors.forCode(struct.get(ERROR_CODE)); + public UpdateMetadataResponse(Struct struct, short version) { + this(new UpdateMetadataResponseData(struct, version)); } public Errors error() { - return error; + return Errors.forCode(data.errorCode()); } @Override public Map errorCounts() { - return errorCounts(error); + return errorCounts(error()); } public static UpdateMetadataResponse parse(ByteBuffer buffer, short version) { - return new UpdateMetadataResponse(ApiKeys.UPDATE_METADATA.parseResponse(version, buffer)); + return new UpdateMetadataResponse(ApiKeys.UPDATE_METADATA.parseResponse(version, buffer), version); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ApiKeys.UPDATE_METADATA.responseSchema(version)); - struct.set(ERROR_CODE, error.code()); - return struct; + return data.toStruct(version); } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java index 3133f44a88f3..4bf56e80b3f6 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.errors.IllegalSaslStateException; import org.apache.kafka.common.errors.SaslAuthenticationException; import org.apache.kafka.common.errors.UnsupportedSaslMechanismException; +import org.apache.kafka.common.message.RequestHeaderData; import org.apache.kafka.common.message.SaslAuthenticateRequestData; import org.apache.kafka.common.message.SaslHandshakeRequestData; import org.apache.kafka.common.network.Authenticator; @@ -324,7 +325,14 @@ public Long reauthenticationLatencyMs() { private RequestHeader nextRequestHeader(ApiKeys apiKey, short version) { String clientId = (String) configs.get(CommonClientConfigs.CLIENT_ID_CONFIG); - currentRequestHeader = new RequestHeader(apiKey, version, clientId, correlationId++); + short requestApiKey = apiKey.id; + currentRequestHeader = new RequestHeader( + new RequestHeaderData(). + setRequestApiKey(requestApiKey). + setRequestApiVersion(version). + setClientId(clientId). + setCorrelationId(correlationId++), + apiKey.headerVersion(version)); return currentRequestHeader; } diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java index 92a70b1f1945..91280ca65a34 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java @@ -46,6 +46,7 @@ class KerberosRule { private final String toPattern; private final boolean repeat; private final boolean toLowerCase; + private final boolean toUpperCase; KerberosRule(String defaultRealm) { this.defaultRealm = defaultRealm; @@ -57,10 +58,11 @@ class KerberosRule { toPattern = null; repeat = false; toLowerCase = false; + toUpperCase = false; } KerberosRule(String defaultRealm, int numOfComponents, String format, String match, String fromPattern, - String toPattern, boolean repeat, boolean toLowerCase) { + String toPattern, boolean repeat, boolean toLowerCase, boolean toUpperCase) { this.defaultRealm = defaultRealm; isDefault = false; this.numOfComponents = numOfComponents; @@ -71,6 +73,7 @@ class KerberosRule { this.toPattern = toPattern; this.repeat = repeat; this.toLowerCase = toLowerCase; + this.toUpperCase = toUpperCase; } @Override @@ -102,6 +105,9 @@ public String toString() { if (toLowerCase) { buf.append("/L"); } + if (toUpperCase) { + buf.append("/U"); + } } return buf.toString(); } @@ -191,7 +197,10 @@ String apply(String[] params) throws IOException { } if (toLowerCase && result != null) { result = result.toLowerCase(Locale.ENGLISH); + } else if (toUpperCase && result != null) { + result = result.toUpperCase(Locale.ENGLISH); } + return result; } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosShortNamer.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosShortNamer.java index 69b4689009ba..96e01f17942b 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosShortNamer.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosShortNamer.java @@ -33,7 +33,7 @@ public class KerberosShortNamer { /** * A pattern for parsing a auth_to_local rule. */ - private static final Pattern RULE_PARSER = Pattern.compile("((DEFAULT)|((RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?(s/([^/]*)/([^/]*)/(g)?)?/?(L)?)))"); + private static final Pattern RULE_PARSER = Pattern.compile("((DEFAULT)|((RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?(s/([^/]*)/([^/]*)/(g)?)?/?(L|U)?)))"); /* Rules for the translation of the principal name into an operating system name */ private final List principalToLocalRules; @@ -66,7 +66,8 @@ private static List parseRules(String defaultRealm, List r matcher.group(10), matcher.group(11), "g".equals(matcher.group(12)), - "L".equals(matcher.group(13)))); + "L".equals(matcher.group(13)), + "U".equals(matcher.group(13)))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java index 6efe3112cdb4..ae9307506571 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java @@ -129,7 +129,7 @@ public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) { } /** - * Read an integer stored in variable-length format using zig-zag decoding from + * Read an integer stored in variable-length format using unsigned decoding from * Google Protocol Buffers. * * @param buffer The buffer to read from @@ -137,7 +137,7 @@ public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) { * * @throws IllegalArgumentException if variable-length value does not terminate after 5 bytes have been read */ - public static int readVarint(ByteBuffer buffer) { + public static int readUnsignedVarint(ByteBuffer buffer) { int value = 0; int i = 0; int b; @@ -148,11 +148,11 @@ public static int readVarint(ByteBuffer buffer) { throw illegalVarintException(value); } value |= b << i; - return (value >>> 1) ^ -(value & 1); + return value; } /** - * Read an integer stored in variable-length format using zig-zag decoding from + * Read an integer stored in variable-length format using unsigned decoding from * Google Protocol Buffers. * * @param in The input to read from @@ -161,7 +161,7 @@ public static int readVarint(ByteBuffer buffer) { * @throws IllegalArgumentException if variable-length value does not terminate after 5 bytes have been read * @throws IOException if {@link DataInput} throws {@link IOException} */ - public static int readVarint(DataInput in) throws IOException { + public static int readUnsignedVarint(DataInput in) throws IOException { int value = 0; int i = 0; int b; @@ -172,6 +172,35 @@ public static int readVarint(DataInput in) throws IOException { throw illegalVarintException(value); } value |= b << i; + return value; + } + + /** + * Read an integer stored in variable-length format using zig-zag decoding from + * Google Protocol Buffers. + * + * @param buffer The buffer to read from + * @return The integer read + * + * @throws IllegalArgumentException if variable-length value does not terminate after 5 bytes have been read + */ + public static int readVarint(ByteBuffer buffer) { + int value = readUnsignedVarint(buffer); + return (value >>> 1) ^ -(value & 1); + } + + /** + * Read an integer stored in variable-length format using zig-zag decoding from + * Google Protocol Buffers. + * + * @param in The input to read from + * @return The integer read + * + * @throws IllegalArgumentException if variable-length value does not terminate after 5 bytes have been read + * @throws IOException if {@link DataInput} throws {@link IOException} + */ + public static int readVarint(DataInput in) throws IOException { + int value = readUnsignedVarint(in); return (value >>> 1) ^ -(value & 1); } @@ -222,6 +251,40 @@ public static long readVarlong(ByteBuffer buffer) { return (value >>> 1) ^ -(value & 1); } + /** + * Write the given integer following the variable-length unsigned encoding from + * Google Protocol Buffers + * into the buffer. + * + * @param value The value to write + * @param buffer The output to write to + */ + public static void writeUnsignedVarint(int value, ByteBuffer buffer) { + while ((value & 0xffffff80) != 0L) { + byte b = (byte) ((value & 0x7f) | 0x80); + buffer.put(b); + value >>>= 7; + } + buffer.put((byte) value); + } + + /** + * Write the given integer following the variable-length unsigned encoding from + * Google Protocol Buffers + * into the buffer. + * + * @param value The value to write + * @param out The output to write to + */ + public static void writeUnsignedVarint(int value, DataOutput out) throws IOException { + while ((value & 0xffffff80) != 0L) { + byte b = (byte) ((value & 0x7f) | 0x80); + out.writeByte(b); + value >>>= 7; + } + out.writeByte((byte) value); + } + /** * Write the given integer following the variable-length zig-zag encoding from * Google Protocol Buffers @@ -231,12 +294,7 @@ public static long readVarlong(ByteBuffer buffer) { * @param out The output to write to */ public static void writeVarint(int value, DataOutput out) throws IOException { - int v = (value << 1) ^ (value >> 31); - while ((v & 0xffffff80) != 0L) { - out.writeByte((v & 0x7f) | 0x80); - v >>>= 7; - } - out.writeByte((byte) v); + writeUnsignedVarint((value << 1) ^ (value >> 31), out); } /** @@ -248,13 +306,7 @@ public static void writeVarint(int value, DataOutput out) throws IOException { * @param buffer The output to write to */ public static void writeVarint(int value, ByteBuffer buffer) { - int v = (value << 1) ^ (value >> 31); - while ((v & 0xffffff80) != 0L) { - byte b = (byte) ((v & 0x7f) | 0x80); - buffer.put(b); - v >>>= 7; - } - buffer.put((byte) v); + writeUnsignedVarint((value << 1) ^ (value >> 31), buffer); } /** @@ -293,20 +345,28 @@ public static void writeVarlong(long value, ByteBuffer buffer) { } /** - * Number of bytes needed to encode an integer in variable-length format. + * Number of bytes needed to encode an integer in unsigned variable-length format. * * @param value The signed value */ - public static int sizeOfVarint(int value) { - int v = (value << 1) ^ (value >> 31); + public static int sizeOfUnsignedVarint(int value) { int bytes = 1; - while ((v & 0xffffff80) != 0L) { + while ((value & 0xffffff80) != 0L) { bytes += 1; - v >>>= 7; + value >>>= 7; } return bytes; } + /** + * Number of bytes needed to encode an integer in variable-length format. + * + * @param value The signed value + */ + public static int sizeOfVarint(int value) { + return sizeOfUnsignedVarint((value << 1) ^ (value >> 31)); + } + /** * Number of bytes needed to encode a long in variable-length format. * diff --git a/clients/src/main/java/org/apache/kafka/common/utils/FlattenedIterator.java b/clients/src/main/java/org/apache/kafka/common/utils/FlattenedIterator.java new file mode 100644 index 000000000000..48bf3b7199e1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/utils/FlattenedIterator.java @@ -0,0 +1,45 @@ +/* + * 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.utils; + +import java.util.Iterator; +import java.util.function.Function; + +/** + * Provides a flattened iterator over the inner elements of an outer iterator. + */ +public final class FlattenedIterator extends AbstractIterator { + private final Iterator outerIterator; + private final Function> innerIteratorFunction; + private Iterator innerIterator; + + public FlattenedIterator(Iterator outerIterator, Function> innerIteratorFunction) { + this.outerIterator = outerIterator; + this.innerIteratorFunction = innerIteratorFunction; + } + + @Override + public I makeNext() { + while (innerIterator == null || !innerIterator.hasNext()) { + if (outerIterator.hasNext()) + innerIterator = innerIteratorFunction.apply(outerIterator.next()); + else + return allDone(); + } + return innerIterator.next(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/utils/MappedIterator.java b/clients/src/main/java/org/apache/kafka/common/utils/MappedIterator.java new file mode 100644 index 000000000000..f6eb270c56a2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/utils/MappedIterator.java @@ -0,0 +1,44 @@ +/* + * 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.utils; + +import java.util.Iterator; +import java.util.function.Function; + +/** + * An iterator that maps another iterator's elements from type `F` to type `T`. + */ +public final class MappedIterator implements Iterator { + private final Iterator underlyingIterator; + private final Function mapper; + + public MappedIterator(Iterator underlyingIterator, Function mapper) { + this.underlyingIterator = underlyingIterator; + this.mapper = mapper; + } + + @Override + public final boolean hasNext() { + return underlyingIterator.hasNext(); + } + + @Override + public final T next() { + return mapper.apply(underlyingIterator.next()); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/server/authorizer/AuthorizableRequestContext.java b/clients/src/main/java/org/apache/kafka/server/authorizer/AuthorizableRequestContext.java index 49c7290106af..f68b9381ce1d 100644 --- a/clients/src/main/java/org/apache/kafka/server/authorizer/AuthorizableRequestContext.java +++ b/clients/src/main/java/org/apache/kafka/server/authorizer/AuthorizableRequestContext.java @@ -32,7 +32,7 @@ public interface AuthorizableRequestContext { /** * Returns name of listener on which request was received. */ - String listener(); + String listenerName(); /** * Returns the security protocol for the listener on which request was received. diff --git a/clients/src/main/resources/common/message/CreateTopicsRequest.json b/clients/src/main/resources/common/message/CreateTopicsRequest.json index d2285668f4cb..e396ed209643 100644 --- a/clients/src/main/resources/common/message/CreateTopicsRequest.json +++ b/clients/src/main/resources/common/message/CreateTopicsRequest.json @@ -19,7 +19,8 @@ "name": "CreateTopicsRequest", // Version 1 adds validateOnly. // Version 4 makes partitions/replicationFactor optional even when assignments are not present (KIP-464) - "validVersions": "0-4", + // Version 5 returns topic configs in the response (KIP-525). + "validVersions": "0-5", "fields": [ { "name": "Topics", "type": "[]CreatableTopic", "versions": "0+", "about": "The topics to create.", "fields": [ diff --git a/clients/src/main/resources/common/message/CreateTopicsResponse.json b/clients/src/main/resources/common/message/CreateTopicsResponse.json index d56e642061e4..f646258bff5a 100644 --- a/clients/src/main/resources/common/message/CreateTopicsResponse.json +++ b/clients/src/main/resources/common/message/CreateTopicsResponse.json @@ -20,8 +20,9 @@ // Version 1 adds a per-topic error message string. // Version 2 adds the throttle time. // Starting in version 3, on quota violation, brokers send out responses before throttling. - // Version 4 makes partitions/replicationFactor optional even when assignments are not present (KIP-464) - "validVersions": "0-4", + // Version 4 makes partitions/replicationFactor optional even when assignments are not present (KIP-464). + // Version 5 returns topic configs in the response (KIP-525) + "validVersions": "0-5", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, "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." }, @@ -32,7 +33,26 @@ { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The error code, or 0 if there was no error." }, { "name": "ErrorMessage", "type": "string", "versions": "1+", "nullableVersions": "0+", "ignorable": true, - "about": "The error message, or null if there was no error." } + "about": "The error message, or null if there was no error." }, + { "name": "TopicConfigErrorCode", "type": "int16", "versions": "5+", "default": "0", + "about": "Optional topic config error returned if configs are not returned in the response." }, + { "name": "NumPartitions", "type": "int32", "versions": "5+", "default": "-1", + "about": "Number of partitions of the topic." }, + { "name": "ReplicationFactor", "type": "int16", "versions": "5+", "default": "-1", + "about": "Replicator factor of the topic." }, + { "name": "Configs", "type": "[]CreatableTopicConfigs", "versions": "5+", "nullableVersions": "5+", + "about": "Configuration of the topic.", "fields": [ + { "name": "Name", "type": "string", "versions": "5+", + "about": "The configuration name." }, + { "name": "Value", "type": "string", "versions": "5+", "nullableVersions": "0+", + "about": "The configuration value." }, + { "name": "ReadOnly", "type": "bool", "versions": "5+", + "about": "True if the configuration is read-only." }, + { "name": "ConfigSource", "type": "int8", "versions": "5+", "default": "-1", "ignorable": true, + "about": "The configuration source." }, + { "name": "IsSensitive", "type": "bool", "versions": "5+", + "about": "True if this configuration is sensitive." } + ]} ]} ] } diff --git a/clients/src/main/resources/common/message/LeaderAndIsrRequest.json b/clients/src/main/resources/common/message/LeaderAndIsrRequest.json index c43d2f4b1633..12f159a15740 100644 --- a/clients/src/main/resources/common/message/LeaderAndIsrRequest.json +++ b/clients/src/main/resources/common/message/LeaderAndIsrRequest.json @@ -30,16 +30,16 @@ "about": "The current controller epoch." }, { "name": "BrokerEpoch", "type": "int64", "versions": "2+", "ignorable": true, "default": "-1", "about": "The current broker epoch." }, - { "name": "PartitionStatesV0", "type": "[]LeaderAndIsrRequestPartition", "versions": "0-1", + { "name": "UngroupedPartitionStates", "type": "[]LeaderAndIsrPartitionState", "versions": "0-1", "about": "The state of each partition, in a v0 or v1 message." }, // In v0 or v1 requests, each partition is listed alongside its topic name. // In v2+ requests, partitions are organized by topic, so that each topic name // only needs to be listed once. - { "name": "TopicStates", "type": "[]LeaderAndIsrRequestTopicState", "versions": "2+", + { "name": "TopicStates", "type": "[]LeaderAndIsrTopicState", "versions": "2+", "about": "Each topic.", "fields": [ - { "name": "Name", "type": "string", "versions": "2+", "entityType": "topicName", + { "name": "TopicName", "type": "string", "versions": "2+", "entityType": "topicName", "about": "The topic name." }, - { "name": "PartitionStatesV0", "type": "[]LeaderAndIsrRequestPartition", "versions": "2+", + { "name": "PartitionStates", "type": "[]LeaderAndIsrPartitionState", "versions": "2+", "about": "The state of each partition" } ]}, { "name": "LiveLeaders", "type": "[]LeaderAndIsrLiveLeader", "versions": "0+", @@ -53,18 +53,18 @@ ]} ], "commonStructs": [ - { "name": "LeaderAndIsrRequestPartition", "versions": "0+", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0-1", "entityType": "topicName", + { "name": "LeaderAndIsrPartitionState", "versions": "0+", "fields": [ + { "name": "TopicName", "type": "string", "versions": "0-1", "entityType": "topicName", "ignorable": true, "about": "The topic name. This is only present in v0 or v1." }, { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index." }, { "name": "ControllerEpoch", "type": "int32", "versions": "0+", "about": "The controller epoch." }, - { "name": "LeaderKey", "type": "int32", "versions": "0+", "entityType": "brokerId", + { "name": "Leader", "type": "int32", "versions": "0+", "entityType": "brokerId", "about": "The broker ID of the leader." }, { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "about": "The leader epoch." }, - { "name": "IsrReplicas", "type": "[]int32", "versions": "0+", + { "name": "Isr", "type": "[]int32", "versions": "0+", "about": "The in-sync replica IDs." }, { "name": "ZkVersion", "type": "int32", "versions": "0+", "about": "The ZooKeeper version." }, diff --git a/clients/src/main/resources/common/message/LeaderAndIsrResponse.json b/clients/src/main/resources/common/message/LeaderAndIsrResponse.json index 06bb088e179b..f30055b97aac 100644 --- a/clients/src/main/resources/common/message/LeaderAndIsrResponse.json +++ b/clients/src/main/resources/common/message/LeaderAndIsrResponse.json @@ -26,7 +26,7 @@ "fields": [ { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The error code, or 0 if there was no error." }, - { "name": "Partitions", "type": "[]LeaderAndIsrResponsePartition", "versions": "0+", + { "name": "PartitionErrors", "type": "[]LeaderAndIsrPartitionError", "versions": "0+", "about": "Each partition.", "fields": [ { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name." }, diff --git a/clients/src/main/resources/common/message/OffsetDeleteRequest.json b/clients/src/main/resources/common/message/OffsetDeleteRequest.json new file mode 100644 index 000000000000..563594997097 --- /dev/null +++ b/clients/src/main/resources/common/message/OffsetDeleteRequest.json @@ -0,0 +1,37 @@ +// 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": 47, + "type": "request", + "name": "OffsetDeleteRequest", + "validVersions": "0", + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", + "about": "The unique group identifier." }, + { "name": "Topics", "type": "[]OffsetDeleteRequestTopic", "versions": "0+", + "about": "The topics to delete offsets for", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, + "about": "The topic name." }, + { "name": "Partitions", "type": "[]OffsetDeleteRequestPartition", "versions": "0+", + "about": "Each partition to delete offsets for.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." } + ] + } + ] + } + ] +} diff --git a/clients/src/main/resources/common/message/OffsetDeleteResponse.json b/clients/src/main/resources/common/message/OffsetDeleteResponse.json new file mode 100644 index 000000000000..f2be321f5d8f --- /dev/null +++ b/clients/src/main/resources/common/message/OffsetDeleteResponse.json @@ -0,0 +1,41 @@ +// 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": 47, + "type": "response", + "name": "OffsetDeleteResponse", + "validVersions": "0", + "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The top-level error code, or 0 if there was no error." }, + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, + "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": "Topics", "type": "[]OffsetDeleteResponseTopic", "versions": "0+", + "about": "The responses for each topic.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, + "about": "The topic name." }, + { "name": "Partitions", "type": "[]OffsetDeleteResponsePartition", "versions": "0+", + "about": "The responses for each partition in the topic.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true, + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." } + ] + } + ] + } + ] +} diff --git a/clients/src/main/resources/common/message/RequestHeader.json b/clients/src/main/resources/common/message/RequestHeader.json index d24dcf0e9875..a00658cc866d 100644 --- a/clients/src/main/resources/common/message/RequestHeader.json +++ b/clients/src/main/resources/common/message/RequestHeader.json @@ -16,7 +16,10 @@ { "type": "header", "name": "RequestHeader", - "validVersions": "0", + // Version 0 of the RequestHeader is only used by v0 of ControlledShutdownRequest. + // + // Version 1 is the first version with ClientId. + "validVersions": "0-1", "fields": [ { "name": "RequestApiKey", "type": "int16", "versions": "0+", "about": "The API key of this request." }, @@ -24,7 +27,7 @@ "about": "The API version of this request." }, { "name": "CorrelationId", "type": "int32", "versions": "0+", "about": "The correlation ID of this request." }, - { "name": "ClientId", "type": "string", "versions": "0+", + { "name": "ClientId", "type": "string", "versions": "1+", "nullableVersions": "1+", "ignorable": true, "about": "The client ID string." } ] } diff --git a/clients/src/main/resources/common/message/ResponseHeader.json b/clients/src/main/resources/common/message/ResponseHeader.json index d44825977da7..bc47b3f05f07 100644 --- a/clients/src/main/resources/common/message/ResponseHeader.json +++ b/clients/src/main/resources/common/message/ResponseHeader.json @@ -16,7 +16,8 @@ { "type": "header", "name": "ResponseHeader", - "validVersions": "0", + // Version 0 and version 1 are the same. + "validVersions": "0-1", "fields": [ { "name": "CorrelationId", "type": "int32", "versions": "0+", "about": "The correlation ID of this response." } diff --git a/clients/src/main/resources/common/message/StopReplicaRequest.json b/clients/src/main/resources/common/message/StopReplicaRequest.json index 12c3d7e8feb6..248faa58a1f8 100644 --- a/clients/src/main/resources/common/message/StopReplicaRequest.json +++ b/clients/src/main/resources/common/message/StopReplicaRequest.json @@ -29,14 +29,14 @@ "about": "The broker epoch." }, { "name": "DeletePartitions", "type": "bool", "versions": "0+", "about": "Whether these partitions should be deleted." }, - { "name": "PartitionsV0", "type": "[]StopReplicaRequestPartitionV0", "versions": "0", + { "name": "UngroupedPartitions", "type": "[]StopReplicaPartitionV0", "versions": "0", "about": "The partitions to stop.", "fields": [ { "name": "TopicName", "type": "string", "versions": "0", "entityType": "topicName", "about": "The topic name." }, { "name": "PartitionIndex", "type": "int32", "versions": "0", "about": "The partition index." } ]}, - { "name": "Topics", "type": "[]StopReplicaRequestTopic", "versions": "1+", + { "name": "Topics", "type": "[]StopReplicaTopic", "versions": "1+", "about": "The topics to stop.", "fields": [ { "name": "Name", "type": "string", "versions": "1+", "entityType": "topicName", "about": "The topic name." }, diff --git a/clients/src/main/resources/common/message/StopReplicaResponse.json b/clients/src/main/resources/common/message/StopReplicaResponse.json index 962cacbedd8b..759bffecf07c 100644 --- a/clients/src/main/resources/common/message/StopReplicaResponse.json +++ b/clients/src/main/resources/common/message/StopReplicaResponse.json @@ -22,7 +22,7 @@ "fields": [ { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The top-level error code, or 0 if there was no top-level error." }, - { "name": "Partitions", "type": "[]StopReplicaResponsePartition", "versions": "0+", + { "name": "PartitionErrors", "type": "[]StopReplicaPartitionError", "versions": "0+", "about": "The responses for each partition.", "fields": [ { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name." }, diff --git a/clients/src/main/resources/common/message/UpdateMetadataRequest.json b/clients/src/main/resources/common/message/UpdateMetadataRequest.json index d0873d1fd097..b1ac987b46c6 100644 --- a/clients/src/main/resources/common/message/UpdateMetadataRequest.json +++ b/clients/src/main/resources/common/message/UpdateMetadataRequest.json @@ -34,16 +34,16 @@ "about": "The controller epoch." }, { "name": "BrokerEpoch", "type": "int64", "versions": "5+", "ignorable": true, "default": "-1", "about": "The broker epoch." }, - { "name": "LegacyPartitionStates", "type": "[]UpdateMetadataPartitionState", "versions": "0-4", + { "name": "UngroupedPartitionStates", "type": "[]UpdateMetadataPartitionState", "versions": "0-4", "about": "In older versions of this RPC, each partition that we would like to update." }, - { "name": "TopicStates", "type": "[]UpdateMetadataRequestTopicState", "versions": "5+", + { "name": "TopicStates", "type": "[]UpdateMetadataTopicState", "versions": "5+", "about": "In newer versions of this RPC, each topic that we would like to update.", "fields": [ { "name": "TopicName", "type": "string", "versions": "5+", "entityType": "topicName", "about": "The topic name." }, { "name": "PartitionStates", "type": "[]UpdateMetadataPartitionState", "versions": "5+", "about": "The partition that we would like to update." } ]}, - { "name": "Brokers", "type": "[]UpdateMetadataRequestBroker", "versions": "0+", "fields": [ + { "name": "LiveBrokers", "type": "[]UpdateMetadataBroker", "versions": "0+", "fields": [ { "name": "Id", "type": "int32", "versions": "0+", "entityType": "brokerId", "about": "The broker id." }, // Version 0 of the protocol only allowed specifying a single host and @@ -52,13 +52,13 @@ "about": "The broker hostname." }, { "name": "V0Port", "type": "int32", "versions": "0", "ignorable": true, "about": "The broker port." }, - { "name": "Endpoints", "type": "[]UpdateMetadataRequestEndpoint", "versions": "1+", + { "name": "Endpoints", "type": "[]UpdateMetadataEndpoint", "versions": "1+", "ignorable": true, "about": "The broker endpoints.", "fields": [ { "name": "Port", "type": "int32", "versions": "1+", "about": "The port of this endpoint" }, { "name": "Host", "type": "string", "versions": "1+", "about": "The hostname of this endpoint" }, - { "name": "Listener", "type": "string", "versions": "3+", + { "name": "Listener", "type": "string", "versions": "3+", "ignorable": true, "about": "The listener name." }, { "name": "SecurityProtocol", "type": "int16", "versions": "1+", "about": "The security protocol type." } @@ -69,7 +69,7 @@ ], "commonStructs": [ { "name": "UpdateMetadataPartitionState", "versions": "0+", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0-4", "entityType": "topicName", + { "name": "TopicName", "type": "string", "versions": "0-4", "entityType": "topicName", "ignorable": true, "about": "In older versions of this RPC, the topic name." }, { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index." }, @@ -85,7 +85,7 @@ "about": "The Zookeeper version." }, { "name": "Replicas", "type": "[]int32", "versions": "0+", "entityType": "brokerId", "about": "All the replicas of this partition." }, - { "name": "OfflineReplicas", "type": "[]int32", "versions": "4+", "entityType": "brokerId", + { "name": "OfflineReplicas", "type": "[]int32", "versions": "4+", "entityType": "brokerId", "ignorable": true, "about": "The replicas of this partition which are offline." } ]} ] diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index b5bba716296b..ab11b95520f9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -45,6 +45,7 @@ import java.util.Collections; import java.util.List; +import static org.apache.kafka.common.protocol.ApiKeys.PRODUCE; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -155,16 +156,23 @@ public void testUnsupportedVersionDuringInternalMetadataRequest() { private void checkSimpleRequestResponse(NetworkClient networkClient) { awaitReady(networkClient, node); // has to be before creating any request, as it may send ApiVersionsRequest and its response is mocked with correlation id 0 - ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); + ProduceRequest.Builder builder = new ProduceRequest.Builder( + PRODUCE.latestVersion(), + PRODUCE.latestVersion(), + (short) 1, + 1000, + Collections.emptyMap(), + null); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = networkClient.newClientRequest( node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); networkClient.send(request, time.milliseconds()); networkClient.poll(1, time.milliseconds()); assertEquals(1, networkClient.inFlightRequestCount()); - ResponseHeader respHeader = new ResponseHeader(request.correlationId()); - Struct resp = new Struct(ApiKeys.PRODUCE.responseSchema(ApiKeys.PRODUCE.latestVersion())); + ResponseHeader respHeader = + new ResponseHeader(request.correlationId(), + request.apiKey().headerVersion(PRODUCE.latestVersion())); + Struct resp = new Struct(PRODUCE.responseSchema(PRODUCE.latestVersion())); resp.set("responses", new Object[0]); Struct responseHeaderStruct = respHeader.toStruct(); int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); @@ -183,7 +191,7 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { private void setExpectedApiVersionsResponse(ApiVersionsResponse response) { short apiVersionsResponseVersion = response.apiVersion(ApiKeys.API_VERSIONS.id).maxVersion; - ByteBuffer buffer = response.serialize(apiVersionsResponseVersion, new ResponseHeader(0)); + ByteBuffer buffer = response.serialize(ApiKeys.API_VERSIONS, apiVersionsResponseVersion, 0); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); } @@ -235,15 +243,22 @@ private void testRequestTimeout(ClientRequest request) { public void testConnectionThrottling() { // Instrument the test to return a response with a 100ms throttle delay. awaitReady(client, node); - ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); + ProduceRequest.Builder builder = new ProduceRequest.Builder( + PRODUCE.latestVersion(), + PRODUCE.latestVersion(), + (short) 1, + 1000, + Collections.emptyMap(), + null); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); - ResponseHeader respHeader = new ResponseHeader(request.correlationId()); - Struct resp = new Struct(ApiKeys.PRODUCE.responseSchema(ApiKeys.PRODUCE.latestVersion())); + ResponseHeader respHeader = + new ResponseHeader(request.correlationId(), + request.apiKey().headerVersion(PRODUCE.latestVersion())); + Struct resp = new Struct(PRODUCE.responseSchema(PRODUCE.latestVersion())); resp.set("responses", new Object[0]); resp.set(CommonFields.THROTTLE_TIME_MS, 100); Struct responseHeaderStruct = respHeader.toStruct(); @@ -288,7 +303,7 @@ private ApiVersionsResponse createExpectedApiVersionsResponse(ApiKeys key, short public void testThrottlingNotEnabledForConnectionToOlderBroker() { // Instrument the test so that the max protocol version for PRODUCE returned from the node is 5 and thus // client-side throttling is not enabled. Also, return a response with a 100ms throttle delay. - setExpectedApiVersionsResponse(createExpectedApiVersionsResponse(ApiKeys.PRODUCE, (short) 5)); + setExpectedApiVersionsResponse(createExpectedApiVersionsResponse(PRODUCE, (short) 5)); while (!client.ready(node, time.milliseconds())) client.poll(1, time.milliseconds()); selector.clear(); @@ -316,8 +331,7 @@ private int sendEmptyProduceRequest() { } - private void sendResponse(int correlationId, Struct response) { - ResponseHeader respHeader = new ResponseHeader(correlationId); + private void sendResponse(ResponseHeader respHeader, Struct response) { Struct responseHeaderStruct = respHeader.toStruct(); int size = responseHeaderStruct.sizeOf() + response.sizeOf(); ByteBuffer buffer = ByteBuffer.allocate(size); @@ -328,10 +342,12 @@ private void sendResponse(int correlationId, Struct response) { } private void sendThrottledProduceResponse(int correlationId, int throttleMs) { - Struct resp = new Struct(ApiKeys.PRODUCE.responseSchema(ApiKeys.PRODUCE.latestVersion())); + Struct resp = new Struct(PRODUCE.responseSchema(PRODUCE.latestVersion())); resp.set("responses", new Object[0]); resp.set(CommonFields.THROTTLE_TIME_MS, throttleMs); - sendResponse(correlationId, resp); + sendResponse(new ResponseHeader(correlationId, + PRODUCE.headerVersion(PRODUCE.latestVersion())), + resp); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 5d4d84b2d47c..b6fece991d24 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.clients.admin; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.MockClient; @@ -41,6 +43,7 @@ import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.GroupSubscribedToTopicException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.LeaderNotAvailableException; @@ -72,6 +75,11 @@ import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; +import org.apache.kafka.common.message.OffsetDeleteResponseData; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartition; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AlterPartitionReassignmentsResponse; import org.apache.kafka.common.requests.ApiError; @@ -98,6 +106,7 @@ import org.apache.kafka.common.requests.ListPartitionReassignmentsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.OffsetDeleteResponse; import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; @@ -242,6 +251,30 @@ public void testCloseAdminClient() { } } + private static OffsetDeleteResponse prepareOffsetDeleteResponse(Errors error) { + return new OffsetDeleteResponse( + new OffsetDeleteResponseData() + .setErrorCode(error.code()) + .setTopics(new OffsetDeleteResponseTopicCollection()) + ); + } + + private static OffsetDeleteResponse prepareOffsetDeleteResponse(String topic, int partition, Errors error) { + return new OffsetDeleteResponse( + new OffsetDeleteResponseData() + .setErrorCode(Errors.NONE.code()) + .setTopics(new OffsetDeleteResponseTopicCollection(Stream.of( + new OffsetDeleteResponseTopic() + .setName(topic) + .setPartitions(new OffsetDeleteResponsePartitionCollection(Collections.singletonList( + new OffsetDeleteResponsePartition() + .setPartitionIndex(partition) + .setErrorCode(error.code()) + ).iterator())) + ).collect(Collectors.toList()).iterator())) + ); + } + private static CreateTopicsResponse prepareCreateTopicsResponse(String topicName, Errors error) { CreateTopicsResponseData data = new CreateTopicsResponseData(); data.topics().add(new CreatableTopicResult(). @@ -1522,6 +1555,226 @@ public void testDeleteConsumerGroups() throws Exception { } } + @Test + public void testDeleteConsumerGroupOffsets() throws Exception { + // Happy path + + final Map nodes = new HashMap<>(); + nodes.put(0, new Node(0, "localhost", 8121)); + + final Cluster cluster = + new Cluster( + "mockClusterId", + nodes.values(), + Collections.emptyList(), + Collections.emptySet(), + Collections.emptySet(), nodes.get(0)); + + final String groupId = "group-0"; + final TopicPartition tp1 = new TopicPartition("foo", 0); + final TopicPartition tp2 = new TopicPartition("bar", 0); + final TopicPartition tp3 = new TopicPartition("foobar", 0); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse(new OffsetDeleteResponse( + new OffsetDeleteResponseData() + .setTopics(new OffsetDeleteResponseTopicCollection(Stream.of( + new OffsetDeleteResponseTopic() + .setName("foo") + .setPartitions(new OffsetDeleteResponsePartitionCollection(Collections.singletonList( + new OffsetDeleteResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code()) + ).iterator())), + new OffsetDeleteResponseTopic() + .setName("bar") + .setPartitions(new OffsetDeleteResponsePartitionCollection(Collections.singletonList( + new OffsetDeleteResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.GROUP_SUBSCRIBED_TO_TOPIC.code()) + ).iterator())) + ).collect(Collectors.toList()).iterator())) + ) + ); + + final DeleteConsumerGroupOffsetsResult errorResult = env.adminClient().deleteConsumerGroupOffsets( + groupId, Stream.of(tp1, tp2).collect(Collectors.toSet())); + + assertNull(errorResult.all().get()); + assertNull(errorResult.partitionResult(tp1).get()); + TestUtils.assertFutureError(errorResult.partitionResult(tp2), GroupSubscribedToTopicException.class); + TestUtils.assertFutureError(errorResult.partitionResult(tp3), IllegalArgumentException.class); + } + } + + @Test + public void testDeleteConsumerGroupOffsetsRetriableErrors() throws Exception { + // Retriable errors should be retried + + final Map nodes = new HashMap<>(); + nodes.put(0, new Node(0, "localhost", 8121)); + + final Cluster cluster = + new Cluster( + "mockClusterId", + nodes.values(), + Collections.emptyList(), + Collections.emptySet(), + Collections.emptySet(), nodes.get(0)); + + final String groupId = "group-0"; + final TopicPartition tp1 = new TopicPartition("foo", 0); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse( + FindCoordinatorResponse.prepareResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + prepareOffsetDeleteResponse(Errors.COORDINATOR_NOT_AVAILABLE)); + + env.kafkaClient().prepareResponse( + prepareOffsetDeleteResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS)); + + /* + * We need to return two responses here, one for NOT_COORDINATOR call when calling delete a consumer group + * api using coordinator that has moved. This will retry whole operation. So we need to again respond with a + * FindCoordinatorResponse. + */ + env.kafkaClient().prepareResponse( + prepareOffsetDeleteResponse(Errors.NOT_COORDINATOR)); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + prepareOffsetDeleteResponse("foo", 0, Errors.NONE)); + + final DeleteConsumerGroupOffsetsResult errorResult1 = env.adminClient() + .deleteConsumerGroupOffsets(groupId, Stream.of(tp1).collect(Collectors.toSet())); + + assertNull(errorResult1.all().get()); + assertNull(errorResult1.partitionResult(tp1).get()); + } + } + + @Test + public void testDeleteConsumerGroupOffsetsNonRetriableErrors() throws Exception { + // Non-retriable errors throw an exception + + final Map nodes = new HashMap<>(); + nodes.put(0, new Node(0, "localhost", 8121)); + + final Cluster cluster = + new Cluster( + "mockClusterId", + nodes.values(), + Collections.emptyList(), + Collections.emptySet(), + Collections.emptySet(), nodes.get(0)); + + final String groupId = "group-0"; + final TopicPartition tp1 = new TopicPartition("foo", 0); + final List retriableErrors = Arrays.asList( + Errors.GROUP_AUTHORIZATION_FAILED, Errors.INVALID_GROUP_ID, Errors.GROUP_ID_NOT_FOUND); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + for (Errors error : retriableErrors) { + env.kafkaClient().prepareResponse(FindCoordinatorResponse + .prepareResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + prepareOffsetDeleteResponse(error)); + + DeleteConsumerGroupOffsetsResult errorResult = env.adminClient() + .deleteConsumerGroupOffsets(groupId, Stream.of(tp1).collect(Collectors.toSet())); + + TestUtils.assertFutureError(errorResult.all(), error.exception().getClass()); + TestUtils.assertFutureError(errorResult.partitionResult(tp1), error.exception().getClass()); + } + } + } + + @Test + public void testDeleteConsumerGroupOffsetsFindCoordinatorRetriableErrors() throws Exception { + // Retriable FindCoordinatorResponse errors should be retried + + final Map nodes = new HashMap<>(); + nodes.put(0, new Node(0, "localhost", 8121)); + + final Cluster cluster = + new Cluster( + "mockClusterId", + nodes.values(), + Collections.emptyList(), + Collections.emptySet(), + Collections.emptySet(), nodes.get(0)); + + final String groupId = "group-0"; + final TopicPartition tp1 = new TopicPartition("foo", 0); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + env.kafkaClient().prepareResponse( + prepareOffsetDeleteResponse("foo", 0, Errors.NONE)); + + final DeleteConsumerGroupOffsetsResult result = env.adminClient() + .deleteConsumerGroupOffsets(groupId, Stream.of(tp1).collect(Collectors.toSet())); + + assertNull(result.all().get()); + assertNull(result.partitionResult(tp1).get()); + } + } + + @Test + public void testDeleteConsumerGroupOffsetsFindCoordinatorNonRetriableErrors() throws Exception { + // Non-retriable FindCoordinatorResponse errors throw an exception + + final Map nodes = new HashMap<>(); + nodes.put(0, new Node(0, "localhost", 8121)); + + final Cluster cluster = + new Cluster( + "mockClusterId", + nodes.values(), + Collections.emptyList(), + Collections.emptySet(), + Collections.emptySet(), nodes.get(0)); + + final String groupId = "group-0"; + final TopicPartition tp1 = new TopicPartition("foo", 0); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse( + prepareFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode())); + + final DeleteConsumerGroupOffsetsResult errorResult = env.adminClient() + .deleteConsumerGroupOffsets(groupId, Stream.of(tp1).collect(Collectors.toSet())); + + TestUtils.assertFutureError(errorResult.all(), GroupAuthorizationException.class); + TestUtils.assertFutureError(errorResult.partitionResult(tp1), GroupAuthorizationException.class); + } + } + @Test public void testIncrementalAlterConfigs() throws Exception { try (AdminClientUnitTestEnv env = mockClientEnv()) { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 86ba572f532d..017e447b7421 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -149,13 +149,13 @@ public DescribeClusterResult describeCluster(DescribeClusterOptions options) { @Override public CreateTopicsResult createTopics(Collection newTopics, CreateTopicsOptions options) { - Map> createTopicResult = new HashMap<>(); + Map> createTopicResult = new HashMap<>(); if (timeoutNextRequests > 0) { for (final NewTopic newTopic : newTopics) { String topicName = newTopic.name(); - KafkaFutureImpl future = new KafkaFutureImpl<>(); + KafkaFutureImpl future = new KafkaFutureImpl<>(); future.completeExceptionally(new TimeoutException()); createTopicResult.put(topicName, future); } @@ -165,7 +165,7 @@ public CreateTopicsResult createTopics(Collection newTopics, CreateTop } for (final NewTopic newTopic : newTopics) { - KafkaFutureImpl future = new KafkaFutureImpl<>(); + KafkaFutureImpl future = new KafkaFutureImpl<>(); String topicName = newTopic.name(); if (allTopics.containsKey(topicName)) { @@ -343,6 +343,11 @@ public DeleteConsumerGroupsResult deleteConsumerGroups(Collection groupI throw new UnsupportedOperationException("Not implemented yet"); } + @Override + public DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsets(String groupId, Set partitions, DeleteConsumerGroupOffsetsOptions options) { + throw new UnsupportedOperationException("Not implemented yet"); + } + @Deprecated @Override public ElectPreferredLeadersResult electPreferredLeaders(Collection partitions, ElectPreferredLeadersOptions options) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 7892d6d1c92d..a9c4d259aa4e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -696,7 +696,7 @@ public void testCommitsFetchedDuringAssign() { // fetch offset for one topic client.prepareResponseFrom(offsetResponse(Collections.singletonMap(tp0, offset1), Errors.NONE), coordinator); - assertEquals(offset1, consumer.committed(tp0).offset()); + assertEquals(offset1, consumer.committed(Collections.singleton(tp0)).get(tp0).offset()); consumer.assign(Arrays.asList(tp0, tp1)); @@ -704,12 +704,12 @@ public void testCommitsFetchedDuringAssign() { Map offsets = new HashMap<>(); offsets.put(tp0, offset1); client.prepareResponseFrom(offsetResponse(offsets, Errors.NONE), coordinator); - assertEquals(offset1, consumer.committed(tp0).offset()); + assertEquals(offset1, consumer.committed(Collections.singleton(tp0)).get(tp0).offset()); offsets.remove(tp0); offsets.put(tp1, offset2); client.prepareResponseFrom(offsetResponse(offsets, Errors.NONE), coordinator); - assertEquals(offset2, consumer.committed(tp1).offset()); + assertEquals(offset2, consumer.committed(Collections.singleton(tp1)).get(tp1).offset()); consumer.close(Duration.ofMillis(0)); } @@ -1137,7 +1137,7 @@ public void testManualAssignmentChangeWithAutoCommitEnabled() { // fetch offset for one topic client.prepareResponseFrom(offsetResponse(Collections.singletonMap(tp0, 0L), Errors.NONE), coordinator); - assertEquals(0, consumer.committed(tp0).offset()); + assertEquals(0, consumer.committed(Collections.singleton(tp0)).get(tp0).offset()); // verify that assignment immediately changes assertTrue(consumer.assignment().equals(singleton(tp0))); @@ -1195,7 +1195,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled() { client.prepareResponseFrom( offsetResponse(Collections.singletonMap(tp0, 0L), Errors.NONE), coordinator); - assertEquals(0, consumer.committed(tp0).offset()); + assertEquals(0, consumer.committed(Collections.singleton(tp0)).get(tp0).offset()); // verify that assignment immediately changes assertTrue(consumer.assignment().equals(singleton(tp0))); @@ -1256,12 +1256,12 @@ public void testOffsetOfPausedPartitions() { offsets.put(tp1, 0L); client.prepareResponseFrom(offsetResponse(offsets, Errors.NONE), coordinator); - assertEquals(0, consumer.committed(tp0).offset()); + assertEquals(0, consumer.committed(Collections.singleton(tp0)).get(tp0).offset()); offsets.remove(tp0); offsets.put(tp1, 0L); client.prepareResponseFrom(offsetResponse(offsets, Errors.NONE), coordinator); - assertEquals(0, consumer.committed(tp1).offset()); + assertEquals(0, consumer.committed(Collections.singleton(tp1)).get(tp1).offset()); // fetch and verify consumer's position in the two partitions final Map offsetResponse = new HashMap<>(); @@ -1356,7 +1356,7 @@ public void testOperationsBySubscribingConsumerWithDefaultGroupId() { } try { - newConsumer((String) null).committed(tp0); + newConsumer((String) null).committed(Collections.singleton(tp0)).get(tp0); fail("Expected an InvalidGroupIdException"); } catch (InvalidGroupIdException e) { // OK, expected @@ -1383,7 +1383,7 @@ public void testOperationsByAssigningConsumerWithDefaultGroupId() { consumer.assign(singleton(tp0)); try { - consumer.committed(tp0); + consumer.committed(Collections.singleton(tp0)).get(tp0); fail("Expected an InvalidGroupIdException"); } catch (InvalidGroupIdException e) { // OK, expected @@ -1636,7 +1636,7 @@ public void testCommitSyncAuthenticationFailure() { @Test(expected = AuthenticationException.class) public void testCommittedAuthenticationFaiure() { final KafkaConsumer consumer = consumerWithPendingAuthenticationError(); - consumer.committed(tp0); + consumer.committed(Collections.singleton(tp0)).get(tp0); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index aad4d2973a03..5a012b2cf67a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -55,9 +55,10 @@ public void testSimpleMock() { assertEquals(rec1, iter.next()); assertEquals(rec2, iter.next()); assertFalse(iter.hasNext()); - assertEquals(2L, consumer.position(new TopicPartition("test", 0))); + final TopicPartition tp = new TopicPartition("test", 0); + assertEquals(2L, consumer.position(tp)); consumer.commitSync(); - assertEquals(2L, consumer.committed(new TopicPartition("test", 0)).offset()); + assertEquals(2L, consumer.committed(Collections.singleton(tp)).get(tp).offset()); } @SuppressWarnings("deprecation") @@ -81,9 +82,10 @@ public void testSimpleMockDeprecated() { assertEquals(rec1, iter.next()); assertEquals(rec2, iter.next()); assertFalse(iter.hasNext()); - assertEquals(2L, consumer.position(new TopicPartition("test", 0))); + final TopicPartition tp = new TopicPartition("test", 0); + assertEquals(2L, consumer.position(tp)); consumer.commitSync(); - assertEquals(2L, consumer.committed(new TopicPartition("test", 0)).offset()); + assertEquals(2L, consumer.committed(Collections.singleton(tp)).get(tp).offset()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java index 118e60a3d128..e5c5073afee0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java @@ -17,15 +17,20 @@ package org.apache.kafka.clients.consumer; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription; +import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor; +import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor.MemberInfo; import org.apache.kafka.common.TopicPartition; +import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -33,39 +38,57 @@ public class RangeAssignorTest { private RangeAssignor assignor = new RangeAssignor(); - private String consumerId = "consumer"; - private String topic = "topic"; + + // For plural tests + private String topic1 = "topic1"; + private String topic2 = "topic2"; + private final String consumer1 = "consumer1"; + private final String instance1 = "instance1"; + private final String consumer2 = "consumer2"; + private final String instance2 = "instance2"; + private final String consumer3 = "consumer3"; + private final String instance3 = "instance3"; + + private List staticMemberInfos; + + @Before + public void setUp() { + staticMemberInfos = new ArrayList<>(); + staticMemberInfos.add(new MemberInfo(consumer1, Optional.of(instance1))); + staticMemberInfos.add(new MemberInfo(consumer2, Optional.of(instance2))); + staticMemberInfos.add(new MemberInfo(consumer3, Optional.of(instance3))); + } @Test public void testOneConsumerNoTopic() { Map partitionsPerTopic = new HashMap<>(); Map> assignment = assignor.assign(partitionsPerTopic, - Collections.singletonMap(consumerId, new Subscription(Collections.emptyList()))); + Collections.singletonMap(consumer1, new Subscription(Collections.emptyList()))); - assertEquals(Collections.singleton(consumerId), assignment.keySet()); - assertTrue(assignment.get(consumerId).isEmpty()); + assertEquals(Collections.singleton(consumer1), assignment.keySet()); + assertTrue(assignment.get(consumer1).isEmpty()); } @Test public void testOneConsumerNonexistentTopic() { Map partitionsPerTopic = new HashMap<>(); Map> assignment = assignor.assign(partitionsPerTopic, - Collections.singletonMap(consumerId, new Subscription(topics(topic)))); - assertEquals(Collections.singleton(consumerId), assignment.keySet()); - assertTrue(assignment.get(consumerId).isEmpty()); + Collections.singletonMap(consumer1, new Subscription(topics(topic1)))); + assertEquals(Collections.singleton(consumer1), assignment.keySet()); + assertTrue(assignment.get(consumer1).isEmpty()); } @Test public void testOneConsumerOneTopic() { Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic, 3); + partitionsPerTopic.put(topic1, 3); Map> assignment = assignor.assign(partitionsPerTopic, - Collections.singletonMap(consumerId, new Subscription(topics(topic)))); + Collections.singletonMap(consumer1, new Subscription(topics(topic1)))); - assertEquals(Collections.singleton(consumerId), assignment.keySet()); - assertAssignment(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId)); + assertEquals(Collections.singleton(consumer1), assignment.keySet()); + assertAssignment(partitions(tp(topic1, 0), tp(topic1, 1), tp(topic1, 2)), assignment.get(consumer1)); } @Test @@ -73,79 +96,58 @@ public void testOnlyAssignsPartitionsFromSubscribedTopics() { String otherTopic = "other"; Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic, 3); + partitionsPerTopic.put(topic1, 3); partitionsPerTopic.put(otherTopic, 3); Map> assignment = assignor.assign(partitionsPerTopic, - Collections.singletonMap(consumerId, new Subscription(topics(topic)))); - assertEquals(Collections.singleton(consumerId), assignment.keySet()); - assertAssignment(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId)); + Collections.singletonMap(consumer1, new Subscription(topics(topic1)))); + assertEquals(Collections.singleton(consumer1), assignment.keySet()); + assertAssignment(partitions(tp(topic1, 0), tp(topic1, 1), tp(topic1, 2)), assignment.get(consumer1)); } @Test public void testOneConsumerMultipleTopics() { - String topic1 = "topic1"; - String topic2 = "topic2"; - - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 1); - partitionsPerTopic.put(topic2, 2); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(1, 2); Map> assignment = assignor.assign(partitionsPerTopic, - Collections.singletonMap(consumerId, new Subscription(topics(topic1, topic2)))); + Collections.singletonMap(consumer1, new Subscription(topics(topic1, topic2)))); - assertEquals(Collections.singleton(consumerId), assignment.keySet()); - assertAssignment(partitions(tp(topic1, 0), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumerId)); + assertEquals(Collections.singleton(consumer1), assignment.keySet()); + assertAssignment(partitions(tp(topic1, 0), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumer1)); } @Test public void testTwoConsumersOneTopicOnePartition() { - String topic = "topic"; - String consumer1 = "consumer1"; - String consumer2 = "consumer2"; - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic, 1); + partitionsPerTopic.put(topic1, 1); Map consumers = new HashMap<>(); - consumers.put(consumer1, new Subscription(topics(topic))); - consumers.put(consumer2, new Subscription(topics(topic))); + consumers.put(consumer1, new Subscription(topics(topic1))); + consumers.put(consumer2, new Subscription(topics(topic1))); Map> assignment = assignor.assign(partitionsPerTopic, consumers); - assertAssignment(partitions(tp(topic, 0)), assignment.get(consumer1)); + assertAssignment(partitions(tp(topic1, 0)), assignment.get(consumer1)); assertAssignment(Collections.emptyList(), assignment.get(consumer2)); } @Test public void testTwoConsumersOneTopicTwoPartitions() { - String topic = "topic"; - String consumer1 = "consumer1"; - String consumer2 = "consumer2"; - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic, 2); + partitionsPerTopic.put(topic1, 2); Map consumers = new HashMap<>(); - consumers.put(consumer1, new Subscription(topics(topic))); - consumers.put(consumer2, new Subscription(topics(topic))); + consumers.put(consumer1, new Subscription(topics(topic1))); + consumers.put(consumer2, new Subscription(topics(topic1))); Map> assignment = assignor.assign(partitionsPerTopic, consumers); - assertAssignment(partitions(tp(topic, 0)), assignment.get(consumer1)); - assertAssignment(partitions(tp(topic, 1)), assignment.get(consumer2)); + assertAssignment(partitions(tp(topic1, 0)), assignment.get(consumer1)); + assertAssignment(partitions(tp(topic1, 1)), assignment.get(consumer2)); } @Test public void testMultipleConsumersMixedTopics() { - String topic1 = "topic1"; - String topic2 = "topic2"; - String consumer1 = "consumer1"; - String consumer2 = "consumer2"; - String consumer3 = "consumer3"; - - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 3); - partitionsPerTopic.put(topic2, 2); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 2); Map consumers = new HashMap<>(); consumers.put(consumer1, new Subscription(topics(topic1))); @@ -165,9 +167,7 @@ public void testTwoConsumersTwoTopicsSixPartitions() { String consumer1 = "consumer1"; String consumer2 = "consumer2"; - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 3); - partitionsPerTopic.put(topic2, 3); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 3); Map consumers = new HashMap<>(); consumers.put(consumer1, new Subscription(topics(topic1, topic2))); @@ -178,11 +178,155 @@ public void testTwoConsumersTwoTopicsSixPartitions() { assertAssignment(partitions(tp(topic1, 2), tp(topic2, 2)), assignment.get(consumer2)); } + @Test + public void testTwoStaticConsumersTwoTopicsSixPartitions() { + // although consumer high has a higher rank than consumer low, the comparison happens on + // instance id level. + String consumerIdLow = "consumer-b"; + String consumerIdHigh = "consumer-a"; + + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 3); + + Map consumers = new HashMap<>(); + Subscription consumerLowSubscription = new Subscription(topics(topic1, topic2), + null, + Collections.emptyList()); + consumerLowSubscription.setGroupInstanceId(Optional.of(instance1)); + consumers.put(consumerIdLow, consumerLowSubscription); + Subscription consumerHighSubscription = new Subscription(topics(topic1, topic2), + null, + Collections.emptyList()); + consumerHighSubscription.setGroupInstanceId(Optional.of(instance2)); + consumers.put(consumerIdHigh, consumerHighSubscription); + Map> assignment = assignor.assign(partitionsPerTopic, consumers); + assertAssignment(partitions(tp(topic1, 0), tp(topic1, 1), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumerIdLow)); + assertAssignment(partitions(tp(topic1, 2), tp(topic2, 2)), assignment.get(consumerIdHigh)); + } + + @Test + public void testOneStaticConsumerAndOneDynamicConsumerTwoTopicsSixPartitions() { + // although consumer high has a higher rank than low, consumer low will win the comparison + // because it has instance id while consumer 2 doesn't. + String consumerIdLow = "consumer-b"; + String consumerIdHigh = "consumer-a"; + + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 3); + + Map consumers = new HashMap<>(); + + Subscription consumerLowSubscription = new Subscription(topics(topic1, topic2), + null, + Collections.emptyList()); + consumerLowSubscription.setGroupInstanceId(Optional.of(instance1)); + consumers.put(consumerIdLow, consumerLowSubscription); + consumers.put(consumerIdHigh, new Subscription(topics(topic1, topic2))); + + Map> assignment = assignor.assign(partitionsPerTopic, consumers); + assertAssignment(partitions(tp(topic1, 0), tp(topic1, 1), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumerIdLow)); + assertAssignment(partitions(tp(topic1, 2), tp(topic2, 2)), assignment.get(consumerIdHigh)); + } + + @Test + public void testStaticMemberRangeAssignmentPersistent() { + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(5, 4); + + Map consumers = new HashMap<>(); + for (MemberInfo m : staticMemberInfos) { + Subscription subscription = new Subscription(topics(topic1, topic2), + null, + Collections.emptyList()); + subscription.setGroupInstanceId(m.groupInstanceId); + consumers.put(m.memberId, subscription); + } + // Consumer 4 is a dynamic member. + String consumer4 = "consumer4"; + consumers.put(consumer4, new Subscription(topics(topic1, topic2))); + + Map> expectedAssignment = new HashMap<>(); + // Have 3 static members instance1, instance2, instance3 to be persistent + // across generations. Their assignment shall be the same. + expectedAssignment.put(consumer1, partitions(tp(topic1, 0), tp(topic1, 1), tp(topic2, 0))); + expectedAssignment.put(consumer2, partitions(tp(topic1, 2), tp(topic2, 1))); + expectedAssignment.put(consumer3, partitions(tp(topic1, 3), tp(topic2, 2))); + expectedAssignment.put(consumer4, partitions(tp(topic1, 4), tp(topic2, 3))); + + Map> assignment = assignor.assign(partitionsPerTopic, consumers); + assertEquals(expectedAssignment, assignment); + + // Replace dynamic member 4 with a new dynamic member 5. + consumers.remove(consumer4); + String consumer5 = "consumer5"; + consumers.put(consumer5, new Subscription(topics(topic1, topic2))); + + expectedAssignment.remove(consumer4); + expectedAssignment.put(consumer5, partitions(tp(topic1, 4), tp(topic2, 3))); + assignment = assignor.assign(partitionsPerTopic, consumers); + assertEquals(expectedAssignment, assignment); + } + + @Test + public void testStaticMemberRangeAssignmentPersistentAfterMemberIdChanges() { + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(5, 5); + + Map consumers = new HashMap<>(); + for (MemberInfo m : staticMemberInfos) { + Subscription subscription = new Subscription(topics(topic1, topic2), + null, + Collections.emptyList()); + subscription.setGroupInstanceId(m.groupInstanceId); + consumers.put(m.memberId, subscription); + } + Map> expectedInstanceAssignment = new HashMap<>(); + expectedInstanceAssignment.put(instance1, + partitions(tp(topic1, 0), tp(topic1, 1), tp(topic2, 0), tp(topic2, 1))); + expectedInstanceAssignment.put(instance2, + partitions(tp(topic1, 2), tp(topic1, 3), tp(topic2, 2), tp(topic2, 3))); + expectedInstanceAssignment.put(instance3, + partitions(tp(topic1, 4), tp(topic2, 4))); + + Map> staticAssignment = + checkStaticAssignment(assignor, partitionsPerTopic, consumers); + assertEquals(expectedInstanceAssignment, staticAssignment); + + // Now switch the member.id fields for each member info, the assignment should + // stay the same as last time. + String consumer4 = "consumer4"; + String consumer5 = "consumer5"; + consumers.put(consumer4, consumers.get(consumer3)); + consumers.remove(consumer3); + consumers.put(consumer5, consumers.get(consumer2)); + consumers.remove(consumer2); + + Map> newStaticAssignment = + checkStaticAssignment(assignor, partitionsPerTopic, consumers); + assertEquals(staticAssignment, newStaticAssignment); + } + + static Map> checkStaticAssignment(AbstractPartitionAssignor assignor, + Map partitionsPerTopic, + Map consumers) { + Map> assignmentByMemberId = assignor.assign(partitionsPerTopic, consumers); + Map> assignmentByInstanceId = new HashMap<>(); + for (Map.Entry entry : consumers.entrySet()) { + String memberId = entry.getKey(); + Optional instanceId = entry.getValue().groupInstanceId(); + instanceId.ifPresent(id -> assignmentByInstanceId.put(id, assignmentByMemberId.get(memberId))); + } + return assignmentByInstanceId; + } + private void assertAssignment(List expected, List actual) { // order doesn't matter for assignment, so convert to a set assertEquals(new HashSet<>(expected), new HashSet<>(actual)); } + private Map setupPartitionsPerTopicWithTwoTopics(int numberOfPartitions1, int numberOfPartitions2) { + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, numberOfPartitions1); + partitionsPerTopic.put(topic2, numberOfPartitions2); + return partitionsPerTopic; + } + private static List topics(String... topics) { return Arrays.asList(topics); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java index e7622807834a..5358a814d430 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer; +import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor; import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor.MemberInfo; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription; import org.apache.kafka.common.TopicPartition; @@ -29,6 +30,7 @@ import java.util.Map; import java.util.Optional; +import static org.apache.kafka.clients.consumer.RangeAssignorTest.checkStaticAssignment; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -38,6 +40,9 @@ public class RoundRobinAssignorTest { private String topic = "topic"; private String consumerId = "consumer"; + private String topic1 = "topic1"; + private String topic2 = "topic2"; + @Test public void testOneConsumerNoTopic() { Map partitionsPerTopic = new HashMap<>(); @@ -83,12 +88,7 @@ public void testOnlyAssignsPartitionsFromSubscribedTopics() { @Test public void testOneConsumerMultipleTopics() { - String topic1 = "topic1"; - String topic2 = "topic2"; - - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 1); - partitionsPerTopic.put(topic2, 2); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(1, 2); Map> assignment = assignor.assign(partitionsPerTopic, Collections.singletonMap(consumerId, new Subscription(topics(topic1, topic2)))); @@ -137,9 +137,7 @@ public void testMultipleConsumersMixedTopics() { String consumer2 = "consumer2"; String consumer3 = "consumer3"; - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 3); - partitionsPerTopic.put(topic2, 2); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 2); Map consumers = new HashMap<>(); consumers.put(consumer1, new Subscription(topics(topic1))); @@ -159,9 +157,7 @@ public void testTwoDynamicConsumersTwoTopicsSixPartitions() { String consumer1 = "consumer1"; String consumer2 = "consumer2"; - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 3); - partitionsPerTopic.put(topic2, 3); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 3); Map consumers = new HashMap<>(); consumers.put(consumer1, new Subscription(topics(topic1, topic2))); @@ -183,9 +179,7 @@ public void testTwoStaticConsumersTwoTopicsSixPartitions() { String consumer2 = "consumer-a"; String instance2 = "instance2"; - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 3); - partitionsPerTopic.put(topic2, 3); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 3); Map consumers = new HashMap<>(); Subscription consumer1Subscription = new Subscription(topics(topic1, topic2), null); @@ -203,15 +197,11 @@ public void testTwoStaticConsumersTwoTopicsSixPartitions() { public void testOneStaticConsumerAndOneDynamicConsumerTwoTopicsSixPartitions() { // although consumer 2 has a higher rank than 1, consumer 1 will win the comparison // because it has instance id while consumer 2 doesn't. - String topic1 = "topic1"; - String topic2 = "topic2"; String consumer1 = "consumer-b"; String instance1 = "instance1"; String consumer2 = "consumer-a"; - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 3); - partitionsPerTopic.put(topic2, 3); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 3); Map consumers = new HashMap<>(); @@ -226,11 +216,9 @@ public void testOneStaticConsumerAndOneDynamicConsumerTwoTopicsSixPartitions() { } @Test - public void testStaticMemberAssignmentPersistent() { + public void testStaticMemberRoundRobinAssignmentPersistent() { // Have 3 static members instance1, instance2, instance3 to be persistent // across generations. Their assignment shall be the same. - String topic1 = "topic1"; - String topic2 = "topic2"; String consumer1 = "consumer1"; String instance1 = "instance1"; String consumer2 = "consumer2"; @@ -246,9 +234,8 @@ public void testStaticMemberAssignmentPersistent() { // Consumer 4 is a dynamic member. String consumer4 = "consumer4"; - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 3); - partitionsPerTopic.put(topic2, 3); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(3, 3); + Map consumers = new HashMap<>(); for (MemberInfo m : staticMemberInfos) { Subscription subscription = new Subscription(topics(topic1, topic2), null); @@ -278,9 +265,7 @@ public void testStaticMemberAssignmentPersistent() { } @Test - public void testStaticMemberAssignmentPersistentAfterMemberIdChanges() { - String topic1 = "topic1"; - String topic2 = "topic2"; + public void testStaticMemberRoundRobinAssignmentPersistentAfterMemberIdChanges() { String consumer1 = "consumer1"; String instance1 = "instance1"; String consumer2 = "consumer2"; @@ -292,37 +277,20 @@ public void testStaticMemberAssignmentPersistentAfterMemberIdChanges() { memberIdToInstanceId.put(consumer2, instance2); memberIdToInstanceId.put(consumer3, instance3); - List memberIdList = Arrays.asList(consumer1, consumer2, consumer3); - Map> staticAssignment = - checkStaticAssignment(topic1, topic2, memberIdList, memberIdToInstanceId); - memberIdToInstanceId.clear(); + Map partitionsPerTopic = setupPartitionsPerTopicWithTwoTopics(5, 5); - // Now switch the member.id fields for each member info, the assignment should - // stay the same as last time. - String consumer4 = "consumer4"; - String consumer5 = "consumer5"; - memberIdToInstanceId.put(consumer4, instance1); - memberIdToInstanceId.put(consumer5, instance2); - memberIdToInstanceId.put(consumer1, instance3); - memberIdList = Arrays.asList(consumer4, consumer5, consumer1); - Map> newStaticAssignment = - checkStaticAssignment(topic1, topic2, memberIdList, memberIdToInstanceId); - - assertEquals(staticAssignment, newStaticAssignment); - } - - private Map> checkStaticAssignment(String topic1, - String topic2, - List memberIdList, - Map memberIdToInstanceId) { - List staticMemberInfos = new ArrayList<>(); + Map> expectedInstanceAssignment = new HashMap<>(); + expectedInstanceAssignment.put(instance1, + partitions(tp(topic1, 0), tp(topic1, 3), tp(topic2, 1), tp(topic2, 4))); + expectedInstanceAssignment.put(instance2, + partitions(tp(topic1, 1), tp(topic1, 4), tp(topic2, 2))); + expectedInstanceAssignment.put(instance3, + partitions(tp(topic1, 2), tp(topic2, 0), tp(topic2, 3))); + + List staticMemberInfos = new ArrayList<>(); for (Map.Entry entry : memberIdToInstanceId.entrySet()) { - staticMemberInfos.add(new MemberInfo(entry.getKey(), Optional.of(entry.getValue()))); + staticMemberInfos.add(new AbstractPartitionAssignor.MemberInfo(entry.getKey(), Optional.of(entry.getValue()))); } - - Map partitionsPerTopic = new HashMap<>(); - partitionsPerTopic.put(topic1, 3); - partitionsPerTopic.put(topic2, 3); Map consumers = new HashMap<>(); for (MemberInfo m : staticMemberInfos) { Subscription subscription = new Subscription(topics(topic1, topic2), null); @@ -330,21 +298,23 @@ private Map> checkStaticAssignment(String topic1, consumers.put(m.memberId, subscription); } - Map> expectedInstanceAssignment = new HashMap<>(); - for (int i = 0; i < memberIdList.size(); i++) { - expectedInstanceAssignment.put(memberIdToInstanceId.get(memberIdList.get(i)), - partitions(tp(topic1, i), tp(topic2, i))); - } + Map> staticAssignment = + checkStaticAssignment(assignor, partitionsPerTopic, consumers); + assertEquals(expectedInstanceAssignment, staticAssignment); - Map> assignmentByMemberId = - assignor.assign(partitionsPerTopic, consumers); - Map> assignmentByInstanceId = new HashMap<>(); - for (String memberId : memberIdList) { - assignmentByInstanceId.put(memberIdToInstanceId.get(memberId), - assignmentByMemberId.get(memberId)); - } - assertEquals(expectedInstanceAssignment, assignmentByInstanceId); - return assignmentByInstanceId; + memberIdToInstanceId.clear(); + + // Now switch the member.id fields for each member info, the assignment should + // stay the same as last time. + String consumer4 = "consumer4"; + String consumer5 = "consumer5"; + consumers.put(consumer4, consumers.get(consumer3)); + consumers.remove(consumer3); + consumers.put(consumer5, consumers.get(consumer2)); + consumers.remove(consumer2); + Map> newStaticAssignment = + checkStaticAssignment(assignor, partitionsPerTopic, consumers); + assertEquals(staticAssignment, newStaticAssignment); } private static List topics(String... topics) { @@ -358,4 +328,11 @@ private static List partitions(TopicPartition... partitions) { private static TopicPartition tp(String topic, int partition) { return new TopicPartition(topic, partition); } + + private Map setupPartitionsPerTopicWithTwoTopics(int numberOfPartitions1, int numberOfPartitions2) { + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, numberOfPartitions1); + partitionsPerTopic.put(topic2, numberOfPartitions2); + return partitionsPerTopic; + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index e0264b3f3591..1936fcc3c310 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.message.SyncGroupResponseData; +import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -88,12 +89,13 @@ public class AbstractCoordinatorTest { private static final String GROUP_ID = "dummy-group"; private static final String METRIC_GROUP_PREFIX = "consumer"; - private MockClient mockClient; - private MockTime mockTime; private Node node; + private Metrics metrics; + private MockTime mockTime; private Node coordinatorNode; - private ConsumerNetworkClient consumerClient; + private MockClient mockClient; private DummyCoordinator coordinator; + private ConsumerNetworkClient consumerClient; private final String memberId = "memberId"; private final String leaderId = "leaderId"; @@ -124,7 +126,7 @@ false, false, new SubscriptionState(logContext, OffsetResetStrategy.EARLIEST), retryBackoffMs, REQUEST_TIMEOUT_MS, HEARTBEAT_INTERVAL_MS); - Metrics metrics = new Metrics(); + metrics = new Metrics(mockTime); mockClient.updateMetadata(TestUtils.metadataUpdateWith(1, emptyMap())); this.node = metadata.fetch().nodes().get(0); @@ -143,6 +145,89 @@ false, false, new SubscriptionState(logContext, OffsetResetStrategy.EARLIEST), mockTime); } + @Test + public void testMetrics() { + setupCoordinator(); + + assertNotNull(getMetric("heartbeat-response-time-max")); + assertNotNull(getMetric("heartbeat-rate")); + assertNotNull(getMetric("heartbeat-total")); + assertNotNull(getMetric("last-heartbeat-seconds-ago")); + assertNotNull(getMetric("join-time-avg")); + assertNotNull(getMetric("join-time-max")); + assertNotNull(getMetric("join-rate")); + assertNotNull(getMetric("join-total")); + assertNotNull(getMetric("sync-time-avg")); + assertNotNull(getMetric("sync-time-max")); + assertNotNull(getMetric("sync-rate")); + assertNotNull(getMetric("sync-total")); + assertNotNull(getMetric("rebalance-latency-avg")); + assertNotNull(getMetric("rebalance-latency-max")); + assertNotNull(getMetric("rebalance-rate-per-hour")); + assertNotNull(getMetric("rebalance-total")); + assertNotNull(getMetric("last-rebalance-seconds-ago")); + assertNotNull(getMetric("failed-rebalance-rate-per-hour")); + assertNotNull(getMetric("failed-rebalance-total")); + + metrics.sensor("heartbeat-latency").record(1.0d); + metrics.sensor("heartbeat-latency").record(6.0d); + metrics.sensor("heartbeat-latency").record(2.0d); + + assertEquals(6.0d, getMetric("heartbeat-response-time-max").metricValue()); + assertEquals(0.1d, getMetric("heartbeat-rate").metricValue()); + assertEquals(3.0d, getMetric("heartbeat-total").metricValue()); + + assertEquals(-1.0d, getMetric("last-heartbeat-seconds-ago").metricValue()); + coordinator.heartbeat().sentHeartbeat(mockTime.milliseconds()); + assertEquals(0.0d, getMetric("last-heartbeat-seconds-ago").metricValue()); + mockTime.sleep(10 * 1000L); + assertEquals(10.0d, getMetric("last-heartbeat-seconds-ago").metricValue()); + + metrics.sensor("join-latency").record(1.0d); + metrics.sensor("join-latency").record(6.0d); + metrics.sensor("join-latency").record(2.0d); + + assertEquals(3.0d, getMetric("join-time-avg").metricValue()); + assertEquals(6.0d, getMetric("join-time-max").metricValue()); + assertEquals(0.1d, getMetric("join-rate").metricValue()); + assertEquals(3.0d, getMetric("join-total").metricValue()); + + metrics.sensor("sync-latency").record(1.0d); + metrics.sensor("sync-latency").record(6.0d); + metrics.sensor("sync-latency").record(2.0d); + + assertEquals(3.0d, getMetric("sync-time-avg").metricValue()); + assertEquals(6.0d, getMetric("sync-time-max").metricValue()); + assertEquals(0.1d, getMetric("sync-rate").metricValue()); + assertEquals(3.0d, getMetric("sync-total").metricValue()); + + metrics.sensor("rebalance-latency").record(1.0d); + metrics.sensor("rebalance-latency").record(6.0d); + metrics.sensor("rebalance-latency").record(2.0d); + + assertEquals(3.0d, getMetric("rebalance-latency-avg").metricValue()); + assertEquals(6.0d, getMetric("rebalance-latency-max").metricValue()); + assertEquals(360.0d, getMetric("rebalance-rate-per-hour").metricValue()); + assertEquals(3.0d, getMetric("rebalance-total").metricValue()); + + metrics.sensor("failed-rebalance").record(1.0d); + metrics.sensor("failed-rebalance").record(6.0d); + metrics.sensor("failed-rebalance").record(2.0d); + + assertEquals(360.0d, getMetric("failed-rebalance-rate-per-hour").metricValue()); + assertEquals(3.0d, getMetric("failed-rebalance-total").metricValue()); + + assertEquals(-1.0d, getMetric("last-rebalance-seconds-ago").metricValue()); + coordinator.setLastRebalanceTime(mockTime.milliseconds()); + assertEquals(0.0d, getMetric("last-rebalance-seconds-ago").metricValue()); + mockTime.sleep(10 * 1000L); + assertEquals(10.0d, getMetric("last-rebalance-seconds-ago").metricValue()); + } + + private KafkaMetric getMetric(final String name) { + return metrics.metrics().get(metrics.metricName(name, "consumer-coordinator-metrics")); + } + @Test public void testCoordinatorDiscoveryBackoff() { setupCoordinator(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 91d3529ea10e..068f7bd8809e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -49,7 +49,9 @@ import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; import org.apache.kafka.common.message.SyncGroupResponseData; +import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.AbstractRequest; @@ -200,6 +202,58 @@ public void teardown() { this.coordinator.close(time.timer(0)); } + @Test + public void testMetrics() { + assertNotNull(getMetric("commit-latency-avg")); + assertNotNull(getMetric("commit-latency-max")); + assertNotNull(getMetric("commit-rate")); + assertNotNull(getMetric("commit-total")); + assertNotNull(getMetric("partition-revoked-latency-avg")); + assertNotNull(getMetric("partition-revoked-latency-max")); + assertNotNull(getMetric("partition-assigned-latency-avg")); + assertNotNull(getMetric("partition-assigned-latency-max")); + assertNotNull(getMetric("partition-lost-latency-avg")); + assertNotNull(getMetric("partition-lost-latency-max")); + assertNotNull(getMetric("assigned-partitions")); + + metrics.sensor("commit-latency").record(1.0d); + metrics.sensor("commit-latency").record(6.0d); + metrics.sensor("commit-latency").record(2.0d); + + assertEquals(3.0d, getMetric("commit-latency-avg").metricValue()); + assertEquals(6.0d, getMetric("commit-latency-max").metricValue()); + assertEquals(0.1d, getMetric("commit-rate").metricValue()); + assertEquals(3.0d, getMetric("commit-total").metricValue()); + + metrics.sensor("partition-revoked-latency").record(1.0d); + metrics.sensor("partition-revoked-latency").record(2.0d); + metrics.sensor("partition-assigned-latency").record(1.0d); + metrics.sensor("partition-assigned-latency").record(2.0d); + metrics.sensor("partition-lost-latency").record(1.0d); + metrics.sensor("partition-lost-latency").record(2.0d); + + assertEquals(1.5d, getMetric("partition-revoked-latency-avg").metricValue()); + assertEquals(2.0d, getMetric("partition-revoked-latency-max").metricValue()); + assertEquals(1.5d, getMetric("partition-assigned-latency-avg").metricValue()); + assertEquals(2.0d, getMetric("partition-assigned-latency-max").metricValue()); + assertEquals(1.5d, getMetric("partition-lost-latency-avg").metricValue()); + assertEquals(2.0d, getMetric("partition-lost-latency-max").metricValue()); + + assertEquals(0.0d, getMetric("assigned-partitions").metricValue()); + subscriptions.assignFromUser(Collections.singleton(t1p)); + assertEquals(1.0d, getMetric("assigned-partitions").metricValue()); + subscriptions.assignFromUser(Utils.mkSet(t1p, t2p)); + assertEquals(2.0d, getMetric("assigned-partitions").metricValue()); + } + + private KafkaMetric getMetric(final String name) { + return metrics.metrics().get(metrics.metricName(name, "consumer" + groupId + "-coordinator-metrics")); + } + + private Sensor getSensor(final String name) { + return metrics.sensor(name); + } + @Test public void testSelectRebalanceProtcol() { List assignors = new ArrayList<>(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 9e2621af5344..f2246d90d74b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -77,7 +77,6 @@ import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest; import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse; -import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.BytesDeserializer; import org.apache.kafka.common.serialization.Deserializer; @@ -1971,8 +1970,9 @@ public void testQuotaMetrics() { 1000, 1000, 64 * 1024, 64 * 1024, 1000, ClientDnsLookup.DEFAULT, time, true, new ApiVersions(), throttleTimeSensor, new LogContext()); - short apiVersionsResponseVersion = ApiKeys.API_VERSIONS.latestVersion(); - ByteBuffer buffer = ApiVersionsResponse.createApiVersionsResponse(400, RecordBatch.CURRENT_MAGIC_VALUE).serialize(apiVersionsResponseVersion, new ResponseHeader(0)); + ByteBuffer buffer = ApiVersionsResponse. + createApiVersionsResponse(400, RecordBatch.CURRENT_MAGIC_VALUE). + serialize(ApiKeys.API_VERSIONS, 0); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); while (!client.ready(node, time.milliseconds())) { client.poll(1, time.milliseconds()); @@ -1989,7 +1989,7 @@ public void testQuotaMetrics() { client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); FetchResponse response = fullFetchResponse(tp0, nextRecords, Errors.NONE, i, throttleTimeMs); - buffer = response.serialize(ApiKeys.FETCH.latestVersion(), new ResponseHeader(request.correlationId())); + buffer = response.serialize(ApiKeys.FETCH, request.correlationId()); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); client.poll(1, time.milliseconds()); // If a throttled response is received, advance the time to ensure progress. 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 4df89455bdd6..653faf21aa0a 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 @@ -64,7 +64,6 @@ import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.ProduceRequest; import org.apache.kafka.common.requests.ProduceResponse; -import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.requests.TransactionResult; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; @@ -277,8 +276,8 @@ public void testQuotaMetrics() throws Exception { 1000, 1000, 64 * 1024, 64 * 1024, 1000, ClientDnsLookup.DEFAULT, time, true, new ApiVersions(), throttleTimeSensor, logContext); - short apiVersionsResponseVersion = ApiKeys.API_VERSIONS.latestVersion(); - ByteBuffer buffer = ApiVersionsResponse.createApiVersionsResponse(400, RecordBatch.CURRENT_MAGIC_VALUE).serialize(apiVersionsResponseVersion, new ResponseHeader(0)); + ByteBuffer buffer = ApiVersionsResponse.createApiVersionsResponse(400, RecordBatch.CURRENT_MAGIC_VALUE). + serialize(ApiKeys.API_VERSIONS, 0); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); while (!client.ready(node, time.milliseconds())) { client.poll(1, time.milliseconds()); @@ -295,7 +294,7 @@ public void testQuotaMetrics() throws Exception { client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); ProduceResponse response = produceResponse(tp0, i, Errors.NONE, throttleTimeMs); - buffer = response.serialize(ApiKeys.PRODUCE.latestVersion(), new ResponseHeader(request.correlationId())); + buffer = response.serialize(ApiKeys.PRODUCE, request.correlationId()); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); client.poll(1, time.milliseconds()); // If a throttled response is received, advance the time to ensure progress. diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 701e9f44919b..9baa0345b4c5 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -648,7 +648,7 @@ public void testClassWithAlias() { // of the aliasing logic to suffice for this test. Thread.currentThread().setContextClassLoader(new ClassLoader(originalClassLoader) { @Override - public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { if (alias.equals(name)) { return NestedClass.class; } else { diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index bf060cdd161e..1af694cd55ea 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -37,7 +37,6 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Message; -import org.apache.kafka.common.protocol.types.ArrayOf; import org.apache.kafka.common.protocol.types.BoundField; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.SchemaException; @@ -249,33 +248,29 @@ public void testOffsetForLeaderEpochVersions() throws Exception { @Test public void testLeaderAndIsrVersions() throws Exception { // Version 3 adds two new fields - AddingReplicas and RemovingReplicas - LeaderAndIsrRequestData.LeaderAndIsrRequestTopicState partitionStateNoAddingRemovingReplicas = - new LeaderAndIsrRequestData.LeaderAndIsrRequestTopicState() - .setName("topic") - .setPartitionStatesV0( - Collections.singletonList( - new LeaderAndIsrRequestData.LeaderAndIsrRequestPartition() - .setPartitionIndex(0) - .setReplicas(Collections.singletonList(0)) - ) - ); - LeaderAndIsrRequestData.LeaderAndIsrRequestTopicState partitionStateWithAddingRemovingReplicas = - new LeaderAndIsrRequestData.LeaderAndIsrRequestTopicState() - .setName("topic") - .setPartitionStatesV0( - Collections.singletonList( - new LeaderAndIsrRequestData.LeaderAndIsrRequestPartition() - .setPartitionIndex(0) - .setReplicas(Collections.singletonList(0)) - .setAddingReplicas(Collections.singletonList(1)) - .setRemovingReplicas(Collections.singletonList(1)) - ) - ); + LeaderAndIsrRequestData.LeaderAndIsrTopicState partitionStateNoAddingRemovingReplicas = + new LeaderAndIsrRequestData.LeaderAndIsrTopicState() + .setTopicName("topic") + .setPartitionStates(Collections.singletonList( + new LeaderAndIsrRequestData.LeaderAndIsrPartitionState() + .setPartitionIndex(0) + .setReplicas(Collections.singletonList(0)) + )); + LeaderAndIsrRequestData.LeaderAndIsrTopicState partitionStateWithAddingRemovingReplicas = + new LeaderAndIsrRequestData.LeaderAndIsrTopicState() + .setTopicName("topic") + .setPartitionStates(Collections.singletonList( + new LeaderAndIsrRequestData.LeaderAndIsrPartitionState() + .setPartitionIndex(0) + .setReplicas(Collections.singletonList(0)) + .setAddingReplicas(Collections.singletonList(1)) + .setRemovingReplicas(Collections.singletonList(1)) + )); testAllMessageRoundTripsBetweenVersions( - (short) 2, - (short) 3, - new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateWithAddingRemovingReplicas)), - new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateNoAddingRemovingReplicas))); + (short) 2, + (short) 3, + new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateWithAddingRemovingReplicas)), + new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateNoAddingRemovingReplicas))); testAllMessageRoundTripsFromVersion((short) 3, new LeaderAndIsrRequestData().setTopicStates(Collections.singletonList(partitionStateWithAddingRemovingReplicas))); } @@ -699,9 +694,9 @@ private static void compareTypes(NamedType typeA, NamedType typeB) { entryA, entryA.type.isNullable() ? "nullable" : "non-nullable", entryB, entryB.type.isNullable() ? "nullable" : "non-nullable")); } - if (entryA.type instanceof ArrayOf) { - compareTypes(new NamedType(entryA.name, ((ArrayOf) entryA.type).type()), - new NamedType(entryB.name, ((ArrayOf) entryB.type).type())); + if (entryA.type.isArray()) { + compareTypes(new NamedType(entryA.name, entryA.type.arrayElementType().get()), + new NamedType(entryB.name, entryB.type.arrayElementType().get())); } } } diff --git a/clients/src/test/java/org/apache/kafka/common/message/TestUUIDDataTest.java b/clients/src/test/java/org/apache/kafka/common/message/TestUUIDDataTest.java new file mode 100644 index 000000000000..4b8e36b12719 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/message/TestUUIDDataTest.java @@ -0,0 +1,86 @@ +/* + * 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.message; + +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.types.Struct; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.UUID; + +public class TestUUIDDataTest { + + @Test + public void shouldStoreField() { + final UUID uuid = UUID.randomUUID(); + final TestUUIDData out = new TestUUIDData(); + out.setProcessId(uuid); + Assert.assertEquals(uuid, out.processId()); + } + + @Test + public void shouldDefaultField() { + final TestUUIDData out = new TestUUIDData(); + Assert.assertEquals(UUID.fromString("00000000-0000-0000-0000-000000000000"), out.processId()); + } + + @Test + public void shouldRoundTripFieldThroughStruct() { + final UUID uuid = UUID.randomUUID(); + final TestUUIDData out = new TestUUIDData(); + out.setProcessId(uuid); + + final Struct struct = out.toStruct((short) 1); + final TestUUIDData in = new TestUUIDData(); + in.fromStruct(struct, (short) 1); + + Assert.assertEquals(uuid, in.processId()); + } + + @Test + public void shouldRoundTripFieldThroughBuffer() { + final UUID uuid = UUID.randomUUID(); + final TestUUIDData out = new TestUUIDData(); + out.setProcessId(uuid); + + final ByteBuffer buffer = ByteBuffer.allocate(out.size((short) 1)); + out.write(new ByteBufferAccessor(buffer), (short) 1); + buffer.rewind(); + + final TestUUIDData in = new TestUUIDData(); + in.read(new ByteBufferAccessor(buffer), (short) 1); + + Assert.assertEquals(uuid, in.processId()); + } + + @Test + public void shouldImplementJVMMethods() { + final UUID uuid = UUID.randomUUID(); + final TestUUIDData a = new TestUUIDData(); + a.setProcessId(uuid); + + final TestUUIDData b = new TestUUIDData(); + b.setProcessId(uuid); + + Assert.assertEquals(a, b); + Assert.assertEquals(a.hashCode(), b.hashCode()); + // just tagging this on here + Assert.assertEquals(a.toString(), b.toString()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/MessageUtilTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/MessageUtilTest.java index 8620a1b03dc8..a28e15570e0b 100755 --- a/clients/src/test/java/org/apache/kafka/common/protocol/MessageUtilTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/MessageUtilTest.java @@ -21,9 +21,11 @@ import org.junit.Test; import org.junit.rules.Timeout; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; public final class MessageUtilTest { @@ -56,4 +58,12 @@ public void testDeepToString() { assertEquals("[foo]", MessageUtil.deepToString(Arrays.asList("foo").iterator())); } + + @Test + public void testByteBufferToArray() { + assertArrayEquals(new byte[] {1, 2, 3}, + MessageUtil.byteBufferToArray(ByteBuffer.wrap(new byte[] {1, 2, 3}))); + assertArrayEquals(new byte[] {}, + MessageUtil.byteBufferToArray(ByteBuffer.wrap(new byte[] {}))); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index 8a3b8ce01ac9..505772004a7c 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -68,31 +68,32 @@ public void setup() { @Test public void testSimple() { - check(Type.BOOLEAN, false); - check(Type.BOOLEAN, true); - check(Type.INT8, (byte) -111); - check(Type.INT16, (short) -11111); - check(Type.INT32, -11111111); - check(Type.INT64, -11111111111L); - check(Type.STRING, ""); - check(Type.STRING, "hello"); - check(Type.STRING, "A\u00ea\u00f1\u00fcC"); - check(Type.NULLABLE_STRING, null); - check(Type.NULLABLE_STRING, ""); - check(Type.NULLABLE_STRING, "hello"); - check(Type.BYTES, ByteBuffer.allocate(0)); - check(Type.BYTES, ByteBuffer.wrap("abcd".getBytes())); - check(Type.NULLABLE_BYTES, null); - check(Type.NULLABLE_BYTES, ByteBuffer.allocate(0)); - check(Type.NULLABLE_BYTES, ByteBuffer.wrap("abcd".getBytes())); - check(Type.VARINT, Integer.MAX_VALUE); - check(Type.VARINT, Integer.MIN_VALUE); - check(Type.VARLONG, Long.MAX_VALUE); - check(Type.VARLONG, Long.MIN_VALUE); - check(new ArrayOf(Type.INT32), new Object[] {1, 2, 3, 4}); - check(new ArrayOf(Type.STRING), new Object[] {}); - check(new ArrayOf(Type.STRING), new Object[] {"hello", "there", "beautiful"}); - check(ArrayOf.nullable(Type.STRING), null); + check(Type.BOOLEAN, false, "BOOLEAN"); + check(Type.BOOLEAN, true, "BOOLEAN"); + check(Type.INT8, (byte) -111, "INT8"); + check(Type.INT16, (short) -11111, "INT16"); + check(Type.INT32, -11111111, "INT32"); + check(Type.INT64, -11111111111L, "INT64"); + check(Type.STRING, "", "STRING"); + check(Type.STRING, "hello", "STRING"); + check(Type.STRING, "A\u00ea\u00f1\u00fcC", "STRING"); + check(Type.NULLABLE_STRING, null, "NULLABLE_STRING"); + check(Type.NULLABLE_STRING, "", "NULLABLE_STRING"); + check(Type.NULLABLE_STRING, "hello", "NULLABLE_STRING"); + check(Type.BYTES, ByteBuffer.allocate(0), "BYTES"); + check(Type.BYTES, ByteBuffer.wrap("abcd".getBytes()), "BYTES"); + check(Type.NULLABLE_BYTES, null, "NULLABLE_BYTES"); + check(Type.NULLABLE_BYTES, ByteBuffer.allocate(0), "NULLABLE_BYTES"); + check(Type.NULLABLE_BYTES, ByteBuffer.wrap("abcd".getBytes()), "NULLABLE_BYTES"); + check(Type.VARINT, Integer.MAX_VALUE, "VARINT"); + check(Type.VARINT, Integer.MIN_VALUE, "VARINT"); + check(Type.VARLONG, Long.MAX_VALUE, "VARLONG"); + check(Type.VARLONG, Long.MIN_VALUE, "VARLONG"); + check(new ArrayOf(Type.INT32), new Object[] {1, 2, 3, 4}, "ARRAY(INT32)"); + check(new ArrayOf(Type.STRING), new Object[] {}, "ARRAY(STRING)"); + check(new ArrayOf(Type.STRING), new Object[] {"hello", "there", "beautiful"}, + "ARRAY(STRING)"); + check(ArrayOf.nullable(Type.STRING), null, "ARRAY(STRING)"); } @Test @@ -105,7 +106,7 @@ public void testNulls() { if (!f.def.type.isNullable()) fail("Should not allow serialization of null value."); } catch (SchemaException e) { - assertFalse(f.def.type.isNullable()); + assertFalse(f.toString() + " should not be nullable", f.def.type.isNullable()); } finally { this.struct.set(f, o); } @@ -259,12 +260,13 @@ private Object roundtrip(Type type, Object obj) { return read; } - private void check(Type type, Object obj) { + private void check(Type type, Object obj, String expectedTypeName) { Object result = roundtrip(type, obj); if (obj instanceof Object[]) { obj = Arrays.asList((Object[]) obj); result = Arrays.asList((Object[]) result); } + assertEquals(expectedTypeName, type.toString()); assertEquals("The object read back should be the same as what was written.", obj, result); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ControlRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ControlRequestTest.java deleted file mode 100644 index 2f4689ed0a09..000000000000 --- a/clients/src/test/java/org/apache/kafka/common/requests/ControlRequestTest.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common.requests; - -import org.apache.kafka.common.TopicPartition; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Collections; -import java.util.Set; -import java.util.Map; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Random; - - -public class ControlRequestTest { - @Test - public void testLeaderAndIsrRequestNormalization() { - Set tps = generateRandomTopicPartitions(10, 10); - Map partitionStates = new HashMap<>(); - for (TopicPartition tp: tps) { - partitionStates.put(tp, new LeaderAndIsrRequest.PartitionState(0, 0, 0, - Collections.emptyList(), 0, Collections.emptyList(), false)); - } - LeaderAndIsrRequest.Builder builder = new LeaderAndIsrRequest.Builder((short) 2, 0, 0, 0, - partitionStates, Collections.emptySet()); - - Assert.assertTrue(builder.build((short) 2).size() < builder.build((short) 1).size()); - } - - @Test - public void testUpdateMetadataRequestNormalization() { - Set tps = generateRandomTopicPartitions(10, 10); - Map partitionStates = new HashMap<>(); - for (TopicPartition tp: tps) { - partitionStates.put(tp, new UpdateMetadataRequest.PartitionState(0, 0, 0, - Collections.emptyList(), 0, Collections.emptyList(), Collections.emptyList())); - } - UpdateMetadataRequest.Builder builder = new UpdateMetadataRequest.Builder((short) 5, 0, 0, 0, - partitionStates, Collections.emptySet()); - - Assert.assertTrue(builder.build((short) 5).size() < builder.build((short) 4).size()); - } - - @Test - public void testStopReplicaRequestNormalization() { - Set tps = generateRandomTopicPartitions(10, 10); - Map partitionStates = new HashMap<>(); - for (TopicPartition tp: tps) { - partitionStates.put(tp, new UpdateMetadataRequest.PartitionState(0, 0, 0, - Collections.emptyList(), 0, Collections.emptyList(), Collections.emptyList())); - } - StopReplicaRequest.Builder builder = new StopReplicaRequest.Builder((short) 5, 0, 0, 0, false, tps); - - Assert.assertTrue(builder.build((short) 1).size() < builder.build((short) 0).size()); - } - - private Set generateRandomTopicPartitions(int numTopic, int numPartitionPerTopic) { - Set tps = new HashSet<>(); - Random r = new Random(); - for (int i = 0; i < numTopic; i++) { - byte[] array = new byte[32]; - r.nextBytes(array); - String topic = new String(array); - for (int j = 0; j < numPartitionPerTopic; j++) { - tps.add(new TopicPartition(topic, j)); - } - } - return tps; - } - -} diff --git a/clients/src/test/java/org/apache/kafka/common/requests/LeaderAndIsrRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/LeaderAndIsrRequestTest.java new file mode 100644 index 000000000000..ffcc9cc09a8a --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/requests/LeaderAndIsrRequestTest.java @@ -0,0 +1,147 @@ +/* + * 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.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.LeaderAndIsrRequestData; +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrLiveLeader; +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static org.apache.kafka.common.protocol.ApiKeys.LEADER_AND_ISR; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class LeaderAndIsrRequestTest { + + /** + * Verifies the logic we have in LeaderAndIsrRequest to present a unified interface across the various versions + * works correctly. For example, `LeaderAndIsrPartitionState.topicName` is not serialiazed/deserialized in + * recent versions, but we set it manually so that we can always present the ungrouped partition states + * independently of the version. + */ + @Test + public void testVersionLogic() { + for (short version = LEADER_AND_ISR.oldestVersion(); version <= LEADER_AND_ISR.latestVersion(); version++) { + List partitionStates = asList( + new LeaderAndIsrPartitionState() + .setTopicName("topic0") + .setPartitionIndex(0) + .setControllerEpoch(2) + .setLeader(0) + .setLeaderEpoch(10) + .setIsr(asList(0, 1)) + .setZkVersion(10) + .setReplicas(asList(0, 1, 2)) + .setAddingReplicas(asList(3)) + .setRemovingReplicas(asList(2)), + new LeaderAndIsrPartitionState() + .setTopicName("topic0") + .setPartitionIndex(1) + .setControllerEpoch(2) + .setLeader(1) + .setLeaderEpoch(11) + .setIsr(asList(1, 2, 3)) + .setZkVersion(11) + .setReplicas(asList(1, 2, 3)) + .setAddingReplicas(emptyList()) + .setRemovingReplicas(emptyList()), + new LeaderAndIsrPartitionState() + .setTopicName("topic1") + .setPartitionIndex(0) + .setControllerEpoch(2) + .setLeader(2) + .setLeaderEpoch(11) + .setIsr(asList(2, 3, 4)) + .setZkVersion(11) + .setReplicas(asList(2, 3, 4)) + .setAddingReplicas(emptyList()) + .setRemovingReplicas(emptyList()) + ); + + List liveNodes = asList( + new Node(0, "host0", 9090), + new Node(1, "host1", 9091) + ); + LeaderAndIsrRequest request = new LeaderAndIsrRequest.Builder(version, 1, 2, 3, partitionStates, + liveNodes).build(); + + List liveLeaders = liveNodes.stream().map(n -> new LeaderAndIsrLiveLeader() + .setBrokerId(n.id()) + .setHostName(n.host()) + .setPort(n.port())).collect(Collectors.toList()); + assertEquals(new HashSet<>(partitionStates), iterableToSet(request.partitionStates())); + assertEquals(liveLeaders, request.liveLeaders()); + assertEquals(1, request.controllerId()); + assertEquals(2, request.controllerEpoch()); + assertEquals(3, request.brokerEpoch()); + + ByteBuffer byteBuffer = request.toBytes(); + LeaderAndIsrRequest deserializedRequest = new LeaderAndIsrRequest(new LeaderAndIsrRequestData( + new ByteBufferAccessor(byteBuffer), version), version); + + // Adding/removing replicas is only supported from version 3, so the deserialized request won't have + // them for earlier versions. + if (version < 3) { + partitionStates.get(0) + .setAddingReplicas(emptyList()) + .setRemovingReplicas(emptyList()); + } + + assertEquals(new HashSet<>(partitionStates), iterableToSet(deserializedRequest.partitionStates())); + assertEquals(liveLeaders, deserializedRequest.liveLeaders()); + assertEquals(1, request.controllerId()); + assertEquals(2, request.controllerEpoch()); + assertEquals(3, request.brokerEpoch()); + } + } + + @Test + public void testTopicPartitionGroupingSizeReduction() { + Set tps = TestUtils.generateRandomTopicPartitions(10, 10); + List partitionStates = new ArrayList<>(); + for (TopicPartition tp : tps) { + partitionStates.add(new LeaderAndIsrPartitionState() + .setTopicName(tp.topic()) + .setPartitionIndex(tp.partition())); + } + LeaderAndIsrRequest.Builder builder = new LeaderAndIsrRequest.Builder((short) 2, 0, 0, 0, + partitionStates, Collections.emptySet()); + + LeaderAndIsrRequest v2 = builder.build((short) 2); + LeaderAndIsrRequest v1 = builder.build((short) 1); + assertTrue("Expected v2 < v1: v2=" + v2.size() + ", v1=" + v1.size(), v2.size() < v1.size()); + } + + private Set iterableToSet(Iterable iterable) { + return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toSet()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/requests/LeaderAndIsrResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/LeaderAndIsrResponseTest.java index 74e097191c11..edb962b2c580 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/LeaderAndIsrResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/LeaderAndIsrResponseTest.java @@ -16,16 +16,19 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState; +import org.apache.kafka.common.message.LeaderAndIsrResponseData; +import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.junit.Test; +import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; +import java.util.List; import java.util.Map; +import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -33,32 +36,50 @@ public class LeaderAndIsrResponseTest { @Test public void testErrorCountsFromGetErrorResponse() { - HashMap partitionStates = new HashMap<>(); - partitionStates.put(new TopicPartition("foo", 0), new LeaderAndIsrRequest.PartitionState(15, 1, 10, - Collections.singletonList(10), 20, Collections.singletonList(10), false)); - partitionStates.put(new TopicPartition("foo", 1), new LeaderAndIsrRequest.PartitionState(15, 1, 10, - Collections.singletonList(10), 20, Collections.singletonList(10), false)); + List partitionStates = new ArrayList<>(); + partitionStates.add(new LeaderAndIsrPartitionState() + .setTopicName("foo") + .setPartitionIndex(0) + .setControllerEpoch(15) + .setLeader(1) + .setLeaderEpoch(10) + .setIsr(Collections.singletonList(10)) + .setZkVersion(20) + .setReplicas(Collections.singletonList(10)) + .setIsNew(false)); + partitionStates.add(new LeaderAndIsrPartitionState() + .setTopicName("foo") + .setPartitionIndex(1) + .setControllerEpoch(15) + .setLeader(1) + .setLeaderEpoch(10) + .setIsr(Collections.singletonList(10)) + .setZkVersion(20) + .setReplicas(Collections.singletonList(10)) + .setIsNew(false)); LeaderAndIsrRequest request = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion(), - 15, 20, 0, partitionStates, Collections.emptySet()).build(); + 15, 20, 0, partitionStates, Collections.emptySet()).build(); LeaderAndIsrResponse response = request.getErrorResponse(0, Errors.CLUSTER_AUTHORIZATION_FAILED.exception()); assertEquals(Collections.singletonMap(Errors.CLUSTER_AUTHORIZATION_FAILED, 2), response.errorCounts()); } @Test public void testErrorCountsWithTopLevelError() { - Map errors = new HashMap<>(); - errors.put(new TopicPartition("foo", 0), Errors.NONE); - errors.put(new TopicPartition("foo", 1), Errors.NOT_LEADER_FOR_PARTITION); - LeaderAndIsrResponse response = new LeaderAndIsrResponse(Errors.UNKNOWN_SERVER_ERROR, errors); + List partitions = createPartitions("foo", + asList(Errors.NONE, Errors.NOT_LEADER_FOR_PARTITION)); + LeaderAndIsrResponse response = new LeaderAndIsrResponse(new LeaderAndIsrResponseData() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) + .setPartitionErrors(partitions)); assertEquals(Collections.singletonMap(Errors.UNKNOWN_SERVER_ERROR, 2), response.errorCounts()); } @Test public void testErrorCountsNoTopLevelError() { - Map errors = new HashMap<>(); - errors.put(new TopicPartition("foo", 0), Errors.NONE); - errors.put(new TopicPartition("foo", 1), Errors.CLUSTER_AUTHORIZATION_FAILED); - LeaderAndIsrResponse response = new LeaderAndIsrResponse(Errors.NONE, errors); + List partitions = createPartitions("foo", + asList(Errors.NONE, Errors.CLUSTER_AUTHORIZATION_FAILED)); + LeaderAndIsrResponse response = new LeaderAndIsrResponse(new LeaderAndIsrResponseData() + .setErrorCode(Errors.NONE.code()) + .setPartitionErrors(partitions)); Map errorCounts = response.errorCounts(); assertEquals(2, errorCounts.size()); assertEquals(1, errorCounts.get(Errors.NONE).intValue()); @@ -67,14 +88,27 @@ public void testErrorCountsNoTopLevelError() { @Test public void testToString() { - Map errors = new HashMap<>(); - errors.put(new TopicPartition("foo", 0), Errors.NONE); - errors.put(new TopicPartition("foo", 1), Errors.CLUSTER_AUTHORIZATION_FAILED); - LeaderAndIsrResponse response = new LeaderAndIsrResponse(Errors.NONE, errors); + List partitions = createPartitions("foo", + asList(Errors.NONE, Errors.CLUSTER_AUTHORIZATION_FAILED)); + LeaderAndIsrResponse response = new LeaderAndIsrResponse(new LeaderAndIsrResponseData() + .setErrorCode(Errors.NONE.code()) + .setPartitionErrors(partitions)); String responseStr = response.toString(); assertTrue(responseStr.contains(LeaderAndIsrResponse.class.getSimpleName())); - assertTrue(responseStr.contains(errors.toString())); - assertTrue(responseStr.contains(Errors.NONE.name())); + assertTrue(responseStr.contains(partitions.toString())); + assertTrue(responseStr.contains("errorCode=" + Errors.NONE.code())); + } + + private List createPartitions(String topicName, List errors) { + List partitions = new ArrayList<>(); + int partitionIndex = 0; + for (Errors error : errors) { + partitions.add(new LeaderAndIsrPartitionError() + .setTopicName(topicName) + .setPartitionIndex(partitionIndex++) + .setErrorCode(error.code())); + } + return partitions; } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java index 7c24d1f2e1d0..d9ac2c2d0d77 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java @@ -64,7 +64,7 @@ public void testSerdeUnsupportedApiVersionRequest() throws Exception { responseBuffer.flip(); responseBuffer.getInt(); // strip off the size - ResponseHeader responseHeader = ResponseHeader.parse(responseBuffer); + ResponseHeader responseHeader = ResponseHeader.parse(responseBuffer, header.headerVersion()); assertEquals(correlationId, responseHeader.correlationId()); Struct struct = ApiKeys.API_VERSIONS.parseResponse((short) 0, responseBuffer); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java index f73ee2f3823b..dd9dc327fc93 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java @@ -43,6 +43,7 @@ public void testSerdeControlledShutdownV0() { assertEquals(0, deserialized.apiVersion()); assertEquals(correlationId, deserialized.correlationId()); assertEquals("", deserialized.clientId()); + assertEquals(0, deserialized.headerVersion()); Struct serialized = deserialized.toStruct(); ByteBuffer serializedBuffer = toBuffer(serialized); @@ -54,23 +55,12 @@ public void testSerdeControlledShutdownV0() { } @Test - public void testRequestHeader() { + public void testRequestHeaderV1() { RequestHeader header = new RequestHeader(ApiKeys.FIND_COORDINATOR, (short) 1, "", 10); + assertEquals(1, header.headerVersion()); ByteBuffer buffer = toBuffer(header.toStruct()); + assertEquals(10, buffer.remaining()); RequestHeader deserialized = RequestHeader.parse(buffer); assertEquals(header, deserialized); } - - @Test - public void testRequestHeaderWithNullClientId() { - RequestHeader header = new RequestHeader(ApiKeys.FIND_COORDINATOR, (short) 1, null, 10); - Struct headerStruct = header.toStruct(); - ByteBuffer buffer = toBuffer(headerStruct); - RequestHeader deserialized = RequestHeader.parse(buffer); - assertEquals(header.apiKey(), deserialized.apiKey()); - assertEquals(header.apiVersion(), deserialized.apiVersion()); - assertEquals(header.correlationId(), deserialized.correlationId()); - assertEquals("", deserialized.clientId()); // null defaults to "" - } - } 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 ddf574937e3f..6f9f7466ecdf 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 @@ -76,19 +76,35 @@ import org.apache.kafka.common.message.InitProducerIdResponseData; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupResponseData; +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState; +import org.apache.kafka.common.message.LeaderAndIsrResponseData; import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.ListGroupsRequestData; import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; +import org.apache.kafka.common.message.OffsetDeleteRequestData; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestPartition; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopic; +import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection; +import org.apache.kafka.common.message.OffsetDeleteResponseData; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartition; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; +import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; import org.apache.kafka.common.message.RenewDelegationTokenResponseData; import org.apache.kafka.common.message.SaslAuthenticateRequestData; import org.apache.kafka.common.message.SaslAuthenticateResponseData; import org.apache.kafka.common.message.SaslHandshakeRequestData; import org.apache.kafka.common.message.SaslHandshakeResponseData; +import org.apache.kafka.common.message.StopReplicaResponseData; import org.apache.kafka.common.message.TxnOffsetCommitRequestData; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataBroker; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataEndpoint; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState; +import org.apache.kafka.common.message.UpdateMetadataResponseData; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; @@ -135,6 +151,7 @@ import java.util.Set; import static java.util.Arrays.asList; +import static org.apache.kafka.common.protocol.ApiKeys.FETCH; import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID; import static org.apache.kafka.test.TestUtils.toBuffer; import static org.junit.Assert.assertEquals; @@ -227,6 +244,8 @@ public void testSerialization() throws Exception { checkErrorResponse(createLeaderAndIsrRequest(0), new UnknownServerException(), false); checkRequest(createLeaderAndIsrRequest(1), true); checkErrorResponse(createLeaderAndIsrRequest(1), new UnknownServerException(), false); + checkRequest(createLeaderAndIsrRequest(2), true); + checkErrorResponse(createLeaderAndIsrRequest(2), new UnknownServerException(), false); checkResponse(createLeaderAndIsrResponse(), 0, true); checkRequest(createSaslHandshakeRequest(), true); checkErrorResponse(createSaslHandshakeRequest(), new UnknownServerException(), true); @@ -377,18 +396,21 @@ public void testSerialization() throws Exception { checkRequest(createListPartitionReassignmentsRequest(), true); checkErrorResponse(createListPartitionReassignmentsRequest(), new UnknownServerException(), true); checkResponse(createListPartitionReassignmentsResponse(), 0, true); + checkRequest(createOffsetDeleteRequest(), true); + checkErrorResponse(createOffsetDeleteRequest(), new UnknownServerException(), true); + checkResponse(createOffsetDeleteResponse(), 0, true); } @Test public void testResponseHeader() { - ResponseHeader header = createResponseHeader(); + ResponseHeader header = createResponseHeader((short) 1); ByteBuffer buffer = toBuffer(header.toStruct()); - ResponseHeader deserialized = ResponseHeader.parse(buffer); + ResponseHeader deserialized = ResponseHeader.parse(buffer, header.headerVersion()); assertEquals(header.correlationId(), deserialized.correlationId()); } private void checkOlderFetchVersions() throws Exception { - int latestVersion = ApiKeys.FETCH.latestVersion(); + int latestVersion = FETCH.latestVersion(); for (int i = 0; i < latestVersion; ++i) { checkErrorResponse(createFetchRequest(i), new UnknownServerException(), true); checkRequest(createFetchRequest(i), true); @@ -535,11 +557,12 @@ public void produceResponseV5Test() { ProduceResponse v5Response = new ProduceResponse(responseData, 10); short version = 5; + short headerVersion = ApiKeys.PRODUCE.headerVersion(version); - ByteBuffer buffer = v5Response.serialize(version, new ResponseHeader(0)); + ByteBuffer buffer = v5Response.serialize(ApiKeys.PRODUCE, version, 0); buffer.rewind(); - ResponseHeader.parse(buffer); // throw away. + ResponseHeader.parse(buffer, headerVersion); // throw away. Struct deserializedStruct = ApiKeys.PRODUCE.parseResponse(version, buffer); @@ -590,9 +613,9 @@ public void fetchResponseVersionTest() { FetchResponse v1Response = new FetchResponse<>(Errors.NONE, responseData, 10, INVALID_SESSION_ID); assertEquals("Throttle time must be zero", 0, v0Response.throttleTimeMs()); assertEquals("Throttle time must be 10", 10, v1Response.throttleTimeMs()); - assertEquals("Should use schema version 0", ApiKeys.FETCH.responseSchema((short) 0), + assertEquals("Should use schema version 0", FETCH.responseSchema((short) 0), v0Response.toStruct((short) 0).schema()); - assertEquals("Should use schema version 1", ApiKeys.FETCH.responseSchema((short) 1), + assertEquals("Should use schema version 1", FETCH.responseSchema((short) 1), v1Response.toStruct((short) 1).schema()); assertEquals("Response data does not match", responseData, v0Response.responseData()); assertEquals("Response data does not match", responseData, v1Response.responseData()); @@ -621,10 +644,10 @@ public void testFetchResponseV4() { @Test public void verifyFetchResponseFullWrites() throws Exception { - verifyFetchResponseFullWrite(ApiKeys.FETCH.latestVersion(), createFetchResponse(123)); - verifyFetchResponseFullWrite(ApiKeys.FETCH.latestVersion(), + verifyFetchResponseFullWrite(FETCH.latestVersion(), createFetchResponse(123)); + verifyFetchResponseFullWrite(FETCH.latestVersion(), createFetchResponse(Errors.FETCH_SESSION_ID_NOT_FOUND, 123)); - for (short version = 0; version <= ApiKeys.FETCH.latestVersion(); version++) { + for (short version = 0; version <= FETCH.latestVersion(); version++) { verifyFetchResponseFullWrite(version, createFetchResponse()); } } @@ -632,7 +655,8 @@ public void verifyFetchResponseFullWrites() throws Exception { private void verifyFetchResponseFullWrite(short apiVersion, FetchResponse fetchResponse) throws Exception { int correlationId = 15; - Send send = fetchResponse.toSend("1", new ResponseHeader(correlationId), apiVersion); + short headerVersion = FETCH.headerVersion(apiVersion); + Send send = fetchResponse.toSend("1", new ResponseHeader(correlationId, headerVersion), apiVersion); ByteBufferChannel channel = new ByteBufferChannel(send.size()); send.writeTo(channel); channel.close(); @@ -644,11 +668,11 @@ private void verifyFetchResponseFullWrite(short apiVersion, FetchResponse fetchR assertTrue(size > 0); // read the header - ResponseHeader responseHeader = ResponseHeader.parse(channel.buffer()); + ResponseHeader responseHeader = ResponseHeader.parse(channel.buffer(), headerVersion); assertEquals(correlationId, responseHeader.correlationId()); // read the body - Struct responseBody = ApiKeys.FETCH.responseSchema(apiVersion).read(buf); + Struct responseBody = FETCH.responseSchema(apiVersion).read(buf); assertEquals(fetchResponse.toStruct(apiVersion), responseBody); assertEquals(size, responseHeader.sizeOf() + responseBody.sizeOf()); @@ -745,8 +769,8 @@ public void testOffsetFetchRequestBuilderToString() { assertTrue(string.contains("group1")); } - private ResponseHeader createResponseHeader() { - return new ResponseHeader(10); + private ResponseHeader createResponseHeader(short headerVersion) { + return new ResponseHeader(10, headerVersion); } private FindCoordinatorRequest createFindCoordinatorRequest(int version) { @@ -1100,9 +1124,14 @@ private StopReplicaRequest createStopReplicaRequest(int version, boolean deleteP } private StopReplicaResponse createStopReplicaResponse() { - Map responses = new HashMap<>(); - responses.put(new TopicPartition("test", 0), Errors.NONE); - return new StopReplicaResponse(Errors.NONE, responses); + List partitions = new ArrayList<>(); + partitions.add(new StopReplicaResponseData.StopReplicaPartitionError() + .setTopicName("test") + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code())); + return new StopReplicaResponse(new StopReplicaResponseData() + .setErrorCode(Errors.NONE.code()) + .setPartitionErrors(partitions)); } private ControlledShutdownRequest createControlledShutdownRequest() { @@ -1140,15 +1169,39 @@ private ControlledShutdownResponse createControlledShutdownResponse() { } private LeaderAndIsrRequest createLeaderAndIsrRequest(int version) { - Map partitionStates = new HashMap<>(); + List partitionStates = new ArrayList<>(); List isr = asList(1, 2); List replicas = asList(1, 2, 3, 4); - partitionStates.put(new TopicPartition("topic5", 105), - new LeaderAndIsrRequest.PartitionState(0, 2, 1, new ArrayList<>(isr), 2, replicas, false)); - partitionStates.put(new TopicPartition("topic5", 1), - new LeaderAndIsrRequest.PartitionState(1, 1, 1, new ArrayList<>(isr), 2, replicas, false)); - partitionStates.put(new TopicPartition("topic20", 1), - new LeaderAndIsrRequest.PartitionState(1, 0, 1, new ArrayList<>(isr), 2, replicas, false)); + partitionStates.add(new LeaderAndIsrPartitionState() + .setTopicName("topic5") + .setPartitionIndex(105) + .setControllerEpoch(0) + .setLeader(2) + .setLeaderEpoch(1) + .setIsr(isr) + .setZkVersion(2) + .setReplicas(replicas) + .setIsNew(false)); + partitionStates.add(new LeaderAndIsrPartitionState() + .setTopicName("topic5") + .setPartitionIndex(1) + .setControllerEpoch(1) + .setLeader(1) + .setLeaderEpoch(1) + .setIsr(isr) + .setZkVersion(2) + .setReplicas(replicas) + .setIsNew(false)); + partitionStates.add(new LeaderAndIsrPartitionState() + .setTopicName("topic20") + .setPartitionIndex(1) + .setControllerEpoch(1) + .setLeader(0) + .setLeaderEpoch(1) + .setIsr(isr) + .setZkVersion(2) + .setReplicas(replicas) + .setIsNew(false)); Set leaders = Utils.mkSet( new Node(0, "test0", 1223), @@ -1158,49 +1211,97 @@ private LeaderAndIsrRequest createLeaderAndIsrRequest(int version) { } private LeaderAndIsrResponse createLeaderAndIsrResponse() { - Map responses = new HashMap<>(); - responses.put(new TopicPartition("test", 0), Errors.NONE); - return new LeaderAndIsrResponse(Errors.NONE, responses); + List partitions = new ArrayList<>(); + partitions.add(new LeaderAndIsrResponseData.LeaderAndIsrPartitionError() + .setTopicName("test") + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code())); + return new LeaderAndIsrResponse(new LeaderAndIsrResponseData() + .setErrorCode(Errors.NONE.code()) + .setPartitionErrors(partitions)); } private UpdateMetadataRequest createUpdateMetadataRequest(int version, String rack) { - Map partitionStates = new HashMap<>(); + List partitionStates = new ArrayList<>(); List isr = asList(1, 2); List replicas = asList(1, 2, 3, 4); List offlineReplicas = asList(); - partitionStates.put(new TopicPartition("topic5", 105), - new UpdateMetadataRequest.PartitionState(0, 2, 1, isr, 2, replicas, offlineReplicas)); - partitionStates.put(new TopicPartition("topic5", 1), - new UpdateMetadataRequest.PartitionState(1, 1, 1, isr, 2, replicas, offlineReplicas)); - partitionStates.put(new TopicPartition("topic20", 1), - new UpdateMetadataRequest.PartitionState(1, 0, 1, isr, 2, replicas, offlineReplicas)); + partitionStates.add(new UpdateMetadataPartitionState() + .setTopicName("topic5") + .setPartitionIndex(105) + .setControllerEpoch(0) + .setLeader(2) + .setLeaderEpoch(1) + .setIsr(isr) + .setZkVersion(2) + .setReplicas(replicas) + .setOfflineReplicas(offlineReplicas)); + partitionStates.add(new UpdateMetadataPartitionState() + .setTopicName("topic5") + .setPartitionIndex(1) + .setControllerEpoch(1) + .setLeader(1) + .setLeaderEpoch(1) + .setIsr(isr) + .setZkVersion(2) + .setReplicas(replicas) + .setOfflineReplicas(offlineReplicas)); + partitionStates.add(new UpdateMetadataPartitionState() + .setTopicName("topic20") + .setPartitionIndex(1) + .setControllerEpoch(1) + .setLeader(0) + .setLeaderEpoch(1) + .setIsr(isr) + .setZkVersion(2) + .setReplicas(replicas) + .setOfflineReplicas(offlineReplicas)); SecurityProtocol plaintext = SecurityProtocol.PLAINTEXT; - List endPoints1 = new ArrayList<>(); - endPoints1.add(new UpdateMetadataRequest.EndPoint("host1", 1223, plaintext, - ListenerName.forSecurityProtocol(plaintext))); - - List endPoints2 = new ArrayList<>(); - endPoints2.add(new UpdateMetadataRequest.EndPoint("host1", 1244, plaintext, - ListenerName.forSecurityProtocol(plaintext))); + List endpoints1 = new ArrayList<>(); + endpoints1.add(new UpdateMetadataEndpoint() + .setHost("host1") + .setPort(1223) + .setSecurityProtocol(plaintext.id) + .setListener(ListenerName.forSecurityProtocol(plaintext).value())); + + List endpoints2 = new ArrayList<>(); + endpoints2.add(new UpdateMetadataEndpoint() + .setHost("host1") + .setPort(1244) + .setSecurityProtocol(plaintext.id) + .setListener(ListenerName.forSecurityProtocol(plaintext).value())); if (version > 0) { SecurityProtocol ssl = SecurityProtocol.SSL; - endPoints2.add(new UpdateMetadataRequest.EndPoint("host2", 1234, ssl, - ListenerName.forSecurityProtocol(ssl))); - endPoints2.add(new UpdateMetadataRequest.EndPoint("host2", 1334, ssl, - new ListenerName("CLIENT"))); + endpoints2.add(new UpdateMetadataEndpoint() + .setHost("host2") + .setPort(1234) + .setSecurityProtocol(ssl.id) + .setListener(ListenerName.forSecurityProtocol(ssl).value())); + endpoints2.add(new UpdateMetadataEndpoint() + .setHost("host2") + .setPort(1334) + .setSecurityProtocol(ssl.id)); + if (version >= 3) + endpoints2.get(1).setListener("CLIENT"); } - Set liveBrokers = Utils.mkSet( - new UpdateMetadataRequest.Broker(0, endPoints1, rack), - new UpdateMetadataRequest.Broker(1, endPoints2, rack) + List liveBrokers = Arrays.asList( + new UpdateMetadataBroker() + .setId(0) + .setEndpoints(endpoints1) + .setRack(rack), + new UpdateMetadataBroker() + .setId(1) + .setEndpoints(endpoints2) + .setRack(rack) ); return new UpdateMetadataRequest.Builder((short) version, 1, 10, 0, partitionStates, - liveBrokers).build(); + liveBrokers).build(); } private UpdateMetadataResponse createUpdateMetadataResponse() { - return new UpdateMetadataResponse(Errors.NONE); + return new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.NONE.code())); } private SaslHandshakeRequest createSaslHandshakeRequest() { @@ -1724,4 +1825,43 @@ private ListPartitionReassignmentsResponse createListPartitionReassignmentsRespo )); return new ListPartitionReassignmentsResponse(data); } + + private OffsetDeleteRequest createOffsetDeleteRequest() { + OffsetDeleteRequestTopicCollection topics = new OffsetDeleteRequestTopicCollection(); + topics.add(new OffsetDeleteRequestTopic() + .setName("topic1") + .setPartitions(Collections.singletonList( + new OffsetDeleteRequestPartition() + .setPartitionIndex(0) + ) + ) + ); + + OffsetDeleteRequestData data = new OffsetDeleteRequestData(); + data.setGroupId("group1"); + data.setTopics(topics); + + return new OffsetDeleteRequest.Builder(data).build((short) 0); + } + + private OffsetDeleteResponse createOffsetDeleteResponse() { + OffsetDeleteResponsePartitionCollection partitions = new OffsetDeleteResponsePartitionCollection(); + partitions.add(new OffsetDeleteResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code()) + ); + + OffsetDeleteResponseTopicCollection topics = new OffsetDeleteResponseTopicCollection(); + topics.add(new OffsetDeleteResponseTopic() + .setName("topic1") + .setPartitions(partitions) + ); + + OffsetDeleteResponseData data = new OffsetDeleteResponseData(); + data.setErrorCode(Errors.NONE.code()); + data.setTopics(topics); + + return new OffsetDeleteResponse(data); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BasePartitionState.java b/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaRequestTest.java similarity index 53% rename from clients/src/main/java/org/apache/kafka/common/requests/BasePartitionState.java rename to clients/src/test/java/org/apache/kafka/common/requests/StopReplicaRequestTest.java index c9a461093caa..c06c9c3c6164 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BasePartitionState.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaRequestTest.java @@ -16,25 +16,21 @@ */ package org.apache.kafka.common.requests; -import java.util.List; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; -// This class contains the common fields shared between LeaderAndIsrRequest.PartitionState and UpdateMetadataRequest.PartitionState -public class BasePartitionState { +import java.util.Set; - public final int controllerEpoch; - public final int leader; - public final int leaderEpoch; - public final List isr; - public final int zkVersion; - public final List replicas; +import static org.junit.Assert.assertTrue; - BasePartitionState(int controllerEpoch, int leader, int leaderEpoch, List isr, int zkVersion, List replicas) { - this.controllerEpoch = controllerEpoch; - this.leader = leader; - this.leaderEpoch = leaderEpoch; - this.isr = isr; - this.zkVersion = zkVersion; - this.replicas = replicas; +public class StopReplicaRequestTest { + + @Test + public void testStopReplicaRequestNormalization() { + Set tps = TestUtils.generateRandomTopicPartitions(10, 10); + StopReplicaRequest.Builder builder = new StopReplicaRequest.Builder((short) 5, 0, 0, 0, false, tps); + assertTrue(builder.build((short) 1).size() < builder.build((short) 0).size()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaResponseTest.java index b6d4bdafb9b4..1463a0bc41cc 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaResponseTest.java @@ -17,13 +17,16 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.StopReplicaResponseData; +import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.utils.Utils; import org.junit.Test; +import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -41,19 +44,25 @@ public void testErrorCountsFromGetErrorResponse() { @Test public void testErrorCountsWithTopLevelError() { - Map errors = new HashMap<>(); - errors.put(new TopicPartition("foo", 0), Errors.NONE); - errors.put(new TopicPartition("foo", 1), Errors.NOT_LEADER_FOR_PARTITION); - StopReplicaResponse response = new StopReplicaResponse(Errors.UNKNOWN_SERVER_ERROR, errors); + List errors = new ArrayList<>(); + errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(0)); + errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(1) + .setErrorCode(Errors.NOT_LEADER_FOR_PARTITION.code())); + StopReplicaResponse response = new StopReplicaResponse(new StopReplicaResponseData() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) + .setPartitionErrors(errors)); assertEquals(Collections.singletonMap(Errors.UNKNOWN_SERVER_ERROR, 2), response.errorCounts()); } @Test public void testErrorCountsNoTopLevelError() { - Map errors = new HashMap<>(); - errors.put(new TopicPartition("foo", 0), Errors.NONE); - errors.put(new TopicPartition("foo", 1), Errors.CLUSTER_AUTHORIZATION_FAILED); - StopReplicaResponse response = new StopReplicaResponse(Errors.NONE, errors); + List errors = new ArrayList<>(); + errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(0)); + errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(1) + .setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code())); + StopReplicaResponse response = new StopReplicaResponse(new StopReplicaResponseData() + .setErrorCode(Errors.NONE.code()) + .setPartitionErrors(errors)); Map errorCounts = response.errorCounts(); assertEquals(2, errorCounts.size()); assertEquals(1, errorCounts.get(Errors.NONE).intValue()); @@ -62,14 +71,15 @@ public void testErrorCountsNoTopLevelError() { @Test public void testToString() { - Map errors = new HashMap<>(); - errors.put(new TopicPartition("foo", 0), Errors.NONE); - errors.put(new TopicPartition("foo", 1), Errors.CLUSTER_AUTHORIZATION_FAILED); - StopReplicaResponse response = new StopReplicaResponse(Errors.NONE, errors); + List errors = new ArrayList<>(); + errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(0)); + errors.add(new StopReplicaPartitionError().setTopicName("foo").setPartitionIndex(1) + .setErrorCode(Errors.CLUSTER_AUTHORIZATION_FAILED.code())); + StopReplicaResponse response = new StopReplicaResponse(new StopReplicaResponseData().setPartitionErrors(errors)); String responseStr = response.toString(); assertTrue(responseStr.contains(StopReplicaResponse.class.getSimpleName())); assertTrue(responseStr.contains(errors.toString())); - assertTrue(responseStr.contains(Errors.NONE.name())); + assertTrue(responseStr.contains("errorCode=" + Errors.NONE.code())); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/UpdateMetadataRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/UpdateMetadataRequestTest.java new file mode 100644 index 000000000000..9bc81fe86ed2 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/requests/UpdateMetadataRequestTest.java @@ -0,0 +1,187 @@ +/* + * 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.TopicPartition; +import org.apache.kafka.common.message.UpdateMetadataRequestData; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataBroker; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataEndpoint; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static org.apache.kafka.common.protocol.ApiKeys.UPDATE_METADATA; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class UpdateMetadataRequestTest { + + /** + * Verifies the logic we have in UpdateMetadataRequest to present a unified interface across the various versions + * works correctly. For example, `UpdateMetadataPartitionState.topicName` is not serialiazed/deserialized in + * recent versions, but we set it manually so that we can always present the ungrouped partition states + * independently of the version. + */ + @Test + public void testVersionLogic() { + for (short version = UPDATE_METADATA.oldestVersion(); version <= UPDATE_METADATA.latestVersion(); version++) { + List partitionStates = asList( + new UpdateMetadataPartitionState() + .setTopicName("topic0") + .setPartitionIndex(0) + .setControllerEpoch(2) + .setLeader(0) + .setLeaderEpoch(10) + .setIsr(asList(0, 1)) + .setZkVersion(10) + .setReplicas(asList(0, 1, 2)) + .setOfflineReplicas(asList(2)), + new UpdateMetadataPartitionState() + .setTopicName("topic0") + .setPartitionIndex(1) + .setControllerEpoch(2) + .setLeader(1) + .setLeaderEpoch(11) + .setIsr(asList(1, 2, 3)) + .setZkVersion(11) + .setReplicas(asList(1, 2, 3)) + .setOfflineReplicas(emptyList()), + new UpdateMetadataPartitionState() + .setTopicName("topic1") + .setPartitionIndex(0) + .setControllerEpoch(2) + .setLeader(2) + .setLeaderEpoch(11) + .setIsr(asList(2, 3)) + .setZkVersion(11) + .setReplicas(asList(2, 3, 4)) + .setOfflineReplicas(emptyList()) + ); + + List broker0Endpoints = new ArrayList<>(); + broker0Endpoints.add( + new UpdateMetadataEndpoint() + .setHost("host0") + .setPort(9090) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)); + + // Non plaintext endpoints are only supported from version 1 + if (version >= 1) { + broker0Endpoints.add(new UpdateMetadataEndpoint() + .setHost("host0") + .setPort(9091) + .setSecurityProtocol(SecurityProtocol.SSL.id)); + } + + // Custom listeners are only supported from version 3 + if (version >= 3) { + broker0Endpoints.get(0).setListener("listener0"); + broker0Endpoints.get(1).setListener("listener1"); + } + + List liveBrokers = asList( + new UpdateMetadataBroker() + .setId(0) + .setRack("rack0") + .setEndpoints(broker0Endpoints), + new UpdateMetadataBroker() + .setId(1) + .setEndpoints(asList( + new UpdateMetadataEndpoint() + .setHost("host1") + .setPort(9090) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener("PLAINTEXT") + )) + ); + + UpdateMetadataRequest request = new UpdateMetadataRequest.Builder(version, 1, 2, 3, + partitionStates, liveBrokers).build(); + + assertEquals(new HashSet<>(partitionStates), iterableToSet(request.partitionStates())); + assertEquals(liveBrokers, request.liveBrokers()); + assertEquals(1, request.controllerId()); + assertEquals(2, request.controllerEpoch()); + assertEquals(3, request.brokerEpoch()); + + ByteBuffer byteBuffer = request.toBytes(); + UpdateMetadataRequest deserializedRequest = new UpdateMetadataRequest(new UpdateMetadataRequestData( + new ByteBufferAccessor(byteBuffer), version), version); + + // Unset fields that are not supported in this version as the deserialized request won't have them + + // Rack is only supported from version 2 + if (version < 2) { + for (UpdateMetadataBroker liveBroker : liveBrokers) + liveBroker.setRack(""); + } + + // Non plaintext listener name is only supported from version 3 + if (version < 3) { + for (UpdateMetadataBroker liveBroker : liveBrokers) { + for (UpdateMetadataEndpoint endpoint : liveBroker.endpoints()) { + SecurityProtocol securityProtocol = SecurityProtocol.forId(endpoint.securityProtocol()); + endpoint.setListener(ListenerName.forSecurityProtocol(securityProtocol).value()); + } + } + } + + // Offline replicas are only supported from version 4 + if (version < 4) + partitionStates.get(0).setOfflineReplicas(emptyList()); + + assertEquals(new HashSet<>(partitionStates), iterableToSet(deserializedRequest.partitionStates())); + assertEquals(liveBrokers, deserializedRequest.liveBrokers()); + assertEquals(1, request.controllerId()); + assertEquals(2, request.controllerEpoch()); + assertEquals(3, request.brokerEpoch()); + } + } + + @Test + public void testTopicPartitionGroupingSizeReduction() { + Set tps = TestUtils.generateRandomTopicPartitions(10, 10); + List partitionStates = new ArrayList<>(); + for (TopicPartition tp : tps) { + partitionStates.add(new UpdateMetadataPartitionState() + .setTopicName(tp.topic()) + .setPartitionIndex(tp.partition())); + } + UpdateMetadataRequest.Builder builder = new UpdateMetadataRequest.Builder((short) 5, 0, 0, 0, + partitionStates, Collections.emptyList()); + + assertTrue(builder.build((short) 5).size() < builder.build((short) 4).size()); + } + + private Set iterableToSet(Iterable iterable) { + return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toSet()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index fc4eb031b35d..82a1f9792647 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -53,6 +53,7 @@ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs; import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.errors.SaslAuthenticationException; +import org.apache.kafka.common.message.RequestHeaderData; import org.apache.kafka.common.message.SaslAuthenticateRequestData; import org.apache.kafka.common.message.SaslHandshakeRequestData; import org.apache.kafka.common.network.CertStores; @@ -680,11 +681,16 @@ public void testApiVersionsRequestWithUnsupportedVersion() throws Exception { // Send ApiVersionsRequest with unsupported version and validate error response. String node = "1"; createClientConnection(SecurityProtocol.PLAINTEXT, node); - RequestHeader header = new RequestHeader(ApiKeys.API_VERSIONS, Short.MAX_VALUE, "someclient", 1); + + RequestHeader header = new RequestHeader(new RequestHeaderData(). + setRequestApiKey(ApiKeys.API_VERSIONS.id). + setRequestApiVersion(Short.MAX_VALUE). + setClientId("someclient"). + setCorrelationId(1), (short) 1); ApiVersionsRequest request = new ApiVersionsRequest.Builder().build(); selector.send(request.toSend(node, header)); ByteBuffer responseBuffer = waitForResponse(); - ResponseHeader.parse(responseBuffer); + ResponseHeader.parse(responseBuffer, header.headerVersion()); ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, (short) 0); assertEquals(Errors.UNSUPPORTED_VERSION, response.error()); diff --git a/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java b/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java index 687f9370a6c9..9dd44a14fb67 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java @@ -86,6 +86,35 @@ public void testToLowerCase() throws Exception { assertEquals("user", shortNamer.shortName(name)); } + @Test + public void testToUpperCase() throws Exception { + List rules = Arrays.asList( + "RULE:[1:$1]/U", + "RULE:[2:$1](Test.*)s/ABC///U", + "RULE:[2:$1](ABC.*)s/ABC/XYZ/g/U", + "RULE:[2:$1](App\\..*)s/App\\.(.*)/$1/g/U", + "RULE:[2:$1]/U", + "DEFAULT" + ); + + KerberosShortNamer shortNamer = KerberosShortNamer.fromUnparsedRules("REALM.COM", rules); + + KerberosName name = KerberosName.parse("User@REALM.COM"); + assertEquals("USER", shortNamer.shortName(name)); + + name = KerberosName.parse("TestABC/host@FOO.COM"); + assertEquals("TEST", shortNamer.shortName(name)); + + name = KerberosName.parse("ABC_User_ABC/host@FOO.COM"); + assertEquals("XYZ_USER_XYZ", shortNamer.shortName(name)); + + name = KerberosName.parse("App.SERVICE-name/example.com@REALM.COM"); + assertEquals("SERVICE-NAME", shortNamer.shortName(name)); + + name = KerberosName.parse("User/root@REALM.COM"); + assertEquals("USER", shortNamer.shortName(name)); + } + @Test public void testInvalidRules() { testInvalidRule(Arrays.asList("default")); @@ -94,6 +123,9 @@ public void testInvalidRules() { testInvalidRule(Arrays.asList("DEFAULT/g")); testInvalidRule(Arrays.asList("rule:[1:$1]")); + testInvalidRule(Arrays.asList("rule:[1:$1]/L/U")); + testInvalidRule(Arrays.asList("rule:[1:$1]/U/L")); + testInvalidRule(Arrays.asList("rule:[1:$1]/LU")); testInvalidRule(Arrays.asList("RULE:[1:$1/L")); testInvalidRule(Arrays.asList("RULE:[1:$1]/l")); testInvalidRule(Arrays.asList("RULE:[2:$1](ABC.*)s/ABC/XYZ/L/g")); diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java index ce23a3378a85..f15e26d222b0 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java @@ -33,11 +33,17 @@ public class ByteUtilsTest { private final byte x01 = 0x01; private final byte x02 = 0x02; private final byte x0F = 0x0f; + private final byte x07 = 0x07; + private final byte x08 = 0x08; + private final byte x3F = 0x3f; + private final byte x40 = 0x40; private final byte x7E = 0x7E; private final byte x7F = 0x7F; private final byte xFF = (byte) 0xff; private final byte x80 = (byte) 0x80; private final byte x81 = (byte) 0x81; + private final byte xBF = (byte) 0xbf; + private final byte xC0 = (byte) 0xc0; private final byte xFE = (byte) 0xfe; @Test @@ -111,6 +117,24 @@ public void testWriteUnsignedIntLEToOutputStream() throws IOException { assertArrayEquals(new byte[] {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4}, os2.toByteArray()); } + @Test + public void testUnsignedVarintSerde() throws Exception { + assertUnsignedVarintSerde(0, new byte[] {x00}); + assertUnsignedVarintSerde(-1, new byte[] {xFF, xFF, xFF, xFF, x0F}); + assertUnsignedVarintSerde(1, new byte[] {x01}); + assertUnsignedVarintSerde(63, new byte[] {x3F}); + assertUnsignedVarintSerde(-64, new byte[] {xC0, xFF, xFF, xFF, x0F}); + assertUnsignedVarintSerde(64, new byte[] {x40}); + assertUnsignedVarintSerde(8191, new byte[] {xFF, x3F}); + assertUnsignedVarintSerde(-8192, new byte[] {x80, xC0, xFF, xFF, x0F}); + assertUnsignedVarintSerde(8192, new byte[] {x80, x40}); + assertUnsignedVarintSerde(-8193, new byte[] {xFF, xBF, xFF, xFF, x0F}); + assertUnsignedVarintSerde(1048575, new byte[] {xFF, xFF, x3F}); + assertUnsignedVarintSerde(1048576, new byte[] {x80, x80, x40}); + assertUnsignedVarintSerde(Integer.MAX_VALUE, new byte[] {xFF, xFF, xFF, xFF, x07}); + assertUnsignedVarintSerde(Integer.MIN_VALUE, new byte[] {x80, x80, x80, x80, x08}); + } + @Test public void testVarintSerde() throws Exception { assertVarintSerde(0, new byte[] {x00}); @@ -197,6 +221,22 @@ public void testInvalidVarlong() { ByteUtils.readVarlong(buf); } + private void assertUnsignedVarintSerde(int value, byte[] expectedEncoding) throws IOException { + ByteBuffer buf = ByteBuffer.allocate(32); + ByteUtils.writeUnsignedVarint(value, buf); + buf.flip(); + assertArrayEquals(expectedEncoding, Utils.toArray(buf)); + assertEquals(value, ByteUtils.readUnsignedVarint(buf.duplicate())); + + buf.rewind(); + DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buf)); + ByteUtils.writeUnsignedVarint(value, out); + buf.flip(); + assertArrayEquals(expectedEncoding, Utils.toArray(buf)); + DataInputStream in = new DataInputStream(new ByteBufferInputStream(buf)); + assertEquals(value, ByteUtils.readUnsignedVarint(in)); + } + private void assertVarintSerde(int value, byte[] expectedEncoding) throws IOException { ByteBuffer buf = ByteBuffer.allocate(32); ByteUtils.writeVarint(value, buf); diff --git a/clients/src/test/java/org/apache/kafka/common/utils/FlattenedIteratorTest.java b/clients/src/test/java/org/apache/kafka/common/utils/FlattenedIteratorTest.java new file mode 100644 index 000000000000..95206cae473a --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/utils/FlattenedIteratorTest.java @@ -0,0 +1,115 @@ +/* + * 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.utils; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static org.junit.Assert.assertEquals; + +public class FlattenedIteratorTest { + + @Test + public void testNestedLists() { + List> list = asList( + asList("foo", "a", "bc"), + asList("ddddd"), + asList("", "bar2", "baz45")); + + Iterable flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator()); + List flattened = new ArrayList<>(); + flattenedIterable.forEach(flattened::add); + + assertEquals(list.stream().flatMap(l -> l.stream()).collect(Collectors.toList()), flattened); + + // Ensure we can iterate multiple times + List flattened2 = new ArrayList<>(); + flattenedIterable.forEach(flattened2::add); + + assertEquals(flattened, flattened2); + } + + @Test + public void testEmptyList() { + List> list = emptyList(); + + Iterable flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator()); + List flattened = new ArrayList<>(); + flattenedIterable.forEach(flattened::add); + + assertEquals(emptyList(), flattened); + } + + @Test + public void testNestedSingleEmptyList() { + List> list = asList(emptyList()); + + Iterable flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator()); + List flattened = new ArrayList<>(); + flattenedIterable.forEach(flattened::add); + + assertEquals(emptyList(), flattened); + } + + @Test + public void testEmptyListFollowedByNonEmpty() { + List> list = asList( + emptyList(), + asList("boo", "b", "de")); + + Iterable flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator()); + List flattened = new ArrayList<>(); + flattenedIterable.forEach(flattened::add); + + assertEquals(list.stream().flatMap(l -> l.stream()).collect(Collectors.toList()), flattened); + } + + @Test + public void testEmptyListInBetweenNonEmpty() { + List> list = asList( + asList("aadwdwdw"), + emptyList(), + asList("ee", "aa", "dd")); + + Iterable flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator()); + List flattened = new ArrayList<>(); + flattenedIterable.forEach(flattened::add); + + assertEquals(list.stream().flatMap(l -> l.stream()).collect(Collectors.toList()), flattened); + } + + @Test + public void testEmptyListAtTheEnd() { + List> list = asList( + asList("ee", "dd"), + asList("e"), + emptyList()); + + Iterable flattenedIterable = () -> new FlattenedIterator<>(list.iterator(), l -> l.iterator()); + List flattened = new ArrayList<>(); + flattenedIterable.forEach(flattened::add); + + assertEquals(list.stream().flatMap(l -> l.stream()).collect(Collectors.toList()), flattened); + } + +} + diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MappedIteratorTest.java b/clients/src/test/java/org/apache/kafka/common/utils/MappedIteratorTest.java new file mode 100644 index 000000000000..729bb25f70f8 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/utils/MappedIteratorTest.java @@ -0,0 +1,61 @@ +/* + * 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.utils; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static org.junit.Assert.assertEquals; + +public class MappedIteratorTest { + + @Test + public void testStringToInteger() { + List list = asList("foo", "", "bar2", "baz45"); + Function mapper = s -> s.length(); + + Iterable mappedIterable = () -> new MappedIterator<>(list.iterator(), mapper); + List mapped = new ArrayList<>(); + mappedIterable.forEach(mapped::add); + + assertEquals(list.stream().map(mapper).collect(Collectors.toList()), mapped); + + // Ensure that we can iterate a second time + List mapped2 = new ArrayList<>(); + mappedIterable.forEach(mapped2::add); + assertEquals(mapped, mapped2); + } + + @Test + public void testEmptyList() { + List list = emptyList(); + Function mapper = s -> s.length(); + + Iterable mappedIterable = () -> new MappedIterator<>(list.iterator(), mapper); + List mapped = new ArrayList<>(); + mappedIterable.forEach(mapped::add); + + assertEquals(emptyList(), mapped); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 5966cbe4a2f0..5858a16e4a50 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -446,6 +446,17 @@ public static ByteBuffer toBuffer(Struct struct) { return buffer; } + public static Set generateRandomTopicPartitions(int numTopic, int numPartitionPerTopic) { + Set tps = new HashSet<>(); + for (int i = 0; i < numTopic; i++) { + String topic = randomString(32); + for (int j = 0; j < numPartitionPerTopic; j++) { + tps.add(new TopicPartition(topic, j)); + } + } + return tps; + } + public static void assertFutureError(Future future, Class exceptionClass) throws InterruptedException { try { diff --git a/clients/src/test/resources/common/message/TestUUID.json b/clients/src/test/resources/common/message/TestUUID.json new file mode 100644 index 000000000000..843a0402bb56 --- /dev/null +++ b/clients/src/test/resources/common/message/TestUUID.json @@ -0,0 +1,22 @@ +// 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. +{ + "name": "TestUUID", + "validVersions": "1", + "fields": [ + { "name": "processId", "versions": "1+", "type": "uuid" } + ], + "type": "header" +} \ No newline at end of file diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/Converter.java b/connect/api/src/main/java/org/apache/kafka/connect/storage/Converter.java index 507489345b9e..29300c74eda4 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/Converter.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/storage/Converter.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.storage; +import org.apache.kafka.common.header.Headers; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -44,6 +45,23 @@ public interface Converter { */ byte[] fromConnectData(String topic, Schema schema, Object value); + /** + * Convert a Kafka Connect data object to a native object for serialization, + * potentially using the supplied topic and headers in the record as necessary. + * + *

    Connect uses this method directly, and for backward compatibility reasons this method + * by default will call the {@link #fromConnectData(String, Schema, Object)} method. + * Override this method to make use of the supplied headers.

    + * @param topic the topic associated with the data + * @param headers the headers associated with the data + * @param schema the schema for the value + * @param value the value to convert + * @return the serialized value + */ + default byte[] fromConnectData(String topic, Headers headers, Schema schema, Object value) { + return fromConnectData(topic, schema, value); + } + /** * Convert a native object to a Kafka Connect data object. * @param topic the topic associated with the data @@ -51,4 +69,20 @@ public interface Converter { * @return an object containing the {@link Schema} and the converted value */ SchemaAndValue toConnectData(String topic, byte[] value); + + /** + * Convert a native object to a Kafka Connect data object, + * potentially using the supplied topic and headers in the record as necessary. + * + *

    Connect uses this method directly, and for backward compatibility reasons this method + * by default will call the {@link #toConnectData(String, byte[])} method. + * Override this method to make use of the supplied headers.

    + * @param topic the topic associated with the data + * @param headers the headers associated with the data + * @param value the value to convert + * @return an object containing the {@link Schema} and the converted value + */ + default SchemaAndValue toConnectData(String topic, Headers headers, byte[] value) { + return toConnectData(topic, value); + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java index d8912a135021..0672f4e9c7f7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java @@ -121,6 +121,6 @@ public boolean isDlqContextHeadersEnabled() { } public static void main(String[] args) { - System.out.println(config.toHtmlTable()); + System.out.println(config.toHtml()); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java index ad891b694503..a0ee46893aca 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java @@ -34,6 +34,6 @@ public SourceConnectorConfig(Plugins plugins, Map props) { } public static void main(String[] args) { - System.out.println(config.toHtmlTable()); + System.out.println(config.toHtml()); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index 395c93e0f772..79760c0bf46a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -481,10 +481,10 @@ private void convertMessages(ConsumerRecords msgs) { } private SinkRecord convertAndTransformRecord(final ConsumerRecord msg) { - SchemaAndValue keyAndSchema = retryWithToleranceOperator.execute(() -> keyConverter.toConnectData(msg.topic(), msg.key()), + SchemaAndValue keyAndSchema = retryWithToleranceOperator.execute(() -> keyConverter.toConnectData(msg.topic(), msg.headers(), msg.key()), Stage.KEY_CONVERTER, keyConverter.getClass()); - SchemaAndValue valueAndSchema = retryWithToleranceOperator.execute(() -> valueConverter.toConnectData(msg.topic(), msg.value()), + SchemaAndValue valueAndSchema = retryWithToleranceOperator.execute(() -> valueConverter.toConnectData(msg.topic(), msg.headers(), msg.value()), Stage.VALUE_CONVERTER, valueConverter.getClass()); Headers headers = retryWithToleranceOperator.execute(() -> convertHeadersFor(msg), Stage.HEADER_CONVERTER, headerConverter.getClass()); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index 9ca79753a474..41b56c8900f6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -278,10 +278,10 @@ private ProducerRecord convertTransformedRecord(SourceRecord rec RecordHeaders headers = retryWithToleranceOperator.execute(() -> convertHeaderFor(record), Stage.HEADER_CONVERTER, headerConverter.getClass()); - byte[] key = retryWithToleranceOperator.execute(() -> keyConverter.fromConnectData(record.topic(), record.keySchema(), record.key()), + byte[] key = retryWithToleranceOperator.execute(() -> keyConverter.fromConnectData(record.topic(), headers, record.keySchema(), record.key()), Stage.KEY_CONVERTER, keyConverter.getClass()); - byte[] value = retryWithToleranceOperator.execute(() -> valueConverter.fromConnectData(record.topic(), record.valueSchema(), record.value()), + byte[] value = retryWithToleranceOperator.execute(() -> valueConverter.fromConnectData(record.topic(), headers, record.valueSchema(), record.value()), Stage.VALUE_CONVERTER, valueConverter.getClass()); if (retryWithToleranceOperator.failed()) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java index 0d66c35f6817..f383482d9fdd 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java @@ -317,6 +317,6 @@ public DistributedConfig(Map props) { } public static void main(String[] args) { - System.out.println(CONFIG.toHtmlTable()); + System.out.println(CONFIG.toHtml()); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java index b76e7d49b5d5..77d7728e2434 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -74,7 +74,7 @@ private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { out.println("

    "); - out.println(docInfo.configDef.toHtmlTable()); + out.println(docInfo.configDef.toHtml()); out.println(""); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TestConverterWithHeaders.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TestConverterWithHeaders.java new file mode 100644 index 000000000000..91e0999d2904 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TestConverterWithHeaders.java @@ -0,0 +1,80 @@ +/* + * 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.connect.runtime; + +import java.io.UnsupportedEncodingException; +import java.util.Map; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.storage.Converter; + +/** + * This is a simple Converter implementation that uses "encoding" header to encode/decode strings via provided charset name + */ +public class TestConverterWithHeaders implements Converter { + private static final String HEADER_ENCODING = "encoding"; + + @Override + public void configure(Map configs, boolean isKey) { + + } + + @Override + public SchemaAndValue toConnectData(String topic, Headers headers, byte[] value) { + String encoding = extractEncoding(headers); + + try { + return new SchemaAndValue(Schema.STRING_SCHEMA, new String(value, encoding)); + } catch (UnsupportedEncodingException e) { + throw new DataException("Unsupported encoding: " + encoding, e); + } + } + + @Override + public byte[] fromConnectData(String topic, Headers headers, Schema schema, Object value) { + String encoding = extractEncoding(headers); + + try { + return ((String) value).getBytes(encoding); + } catch (UnsupportedEncodingException e) { + throw new DataException("Unsupported encoding: " + encoding, e); + } + } + + private String extractEncoding(Headers headers) { + Header header = headers.lastHeader(HEADER_ENCODING); + if (header == null) { + throw new DataException("Header '" + HEADER_ENCODING + "' is required!"); + } + + return new String(header.value()); + } + + + @Override + public SchemaAndValue toConnectData(String topic, byte[] value) { + throw new DataException("Headers are required for this converter!"); + } + + @Override + public byte[] fromConnectData(String topic, Schema schema, Object value) { + throw new DataException("Headers are required for this converter!"); + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index 4d629709efe8..8c93528b7652 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.connect.runtime; +import java.util.Arrays; +import java.util.Iterator; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -25,6 +27,9 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.MockTime; @@ -42,6 +47,7 @@ import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.ConnectorTaskId; import org.easymock.Capture; import org.easymock.CaptureType; @@ -162,6 +168,10 @@ public void setUp() { } private void createTask(TargetState initialState) { + createTask(initialState, keyConverter, valueConverter, headerConverter); + } + + private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { workerTask = new WorkerSinkTask( taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, keyConverter, valueConverter, headerConverter, @@ -1264,6 +1274,85 @@ public void testMetricsGroup() { assertEquals(30, metrics.currentMetricValueAsDouble(group1.metricGroup(), "put-batch-max-time-ms"), 0.001d); } + @Test + public void testHeaders() throws Exception { + Headers headers = new RecordHeaders(); + headers.add("header_key", "header_value".getBytes()); + + createTask(initialState); + + expectInitializeTask(); + expectPollInitialAssignment(); + + expectConsumerPoll(1, headers); + expectConversionAndTransformation(1, null, headers); + sinkTask.put(EasyMock.>anyObject()); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + workerTask.iteration(); // iter 1 -- initial assignment + workerTask.iteration(); // iter 2 -- deliver 1 record + + PowerMock.verifyAll(); + } + + @Test + public void testHeadersWithCustomConverter() throws Exception { + StringConverter stringConverter = new StringConverter(); + TestConverterWithHeaders testConverter = new TestConverterWithHeaders(); + + createTask(initialState, stringConverter, testConverter, stringConverter); + + expectInitializeTask(); + expectPollInitialAssignment(); + + String keyA = "a"; + String valueA = "Árvíztűrő tükörfúrógép"; + Headers headersA = new RecordHeaders(); + String encodingA = "latin2"; + headersA.add("encoding", encodingA.getBytes()); + + String keyB = "b"; + String valueB = "Тестовое сообщение"; + Headers headersB = new RecordHeaders(); + String encodingB = "koi8_r"; + headersB.add("encoding", encodingB.getBytes()); + + expectConsumerPoll(Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, + 0L, 0, 0, keyA.getBytes(), valueA.getBytes(encodingA), headersA), + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 2, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, + 0L, 0, 0, keyB.getBytes(), valueB.getBytes(encodingB), headersB) + )); + + expectTransformation(2, null); + + Capture> records = EasyMock.newCapture(CaptureType.ALL); + sinkTask.put(EasyMock.capture(records)); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + workerTask.iteration(); // iter 1 -- initial assignment + workerTask.iteration(); // iter 2 -- deliver 1 record + + Iterator iterator = records.getValue().iterator(); + + SinkRecord recordA = iterator.next(); + assertEquals(keyA, recordA.key()); + assertEquals(valueA, (String) recordA.value()); + + SinkRecord recordB = iterator.next(); + assertEquals(keyB, recordB.key()); + assertEquals(valueB, (String) recordB.value()); + + PowerMock.verifyAll(); + } + private void expectInitializeTask() throws Exception { consumer.subscribe(EasyMock.eq(asList(TOPIC)), EasyMock.capture(rebalanceListener)); PowerMock.expectLastCall(); @@ -1346,17 +1435,25 @@ private void expectConsumerWakeup() { } private void expectConsumerPoll(final int numMessages) { - expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE); + expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, emptyHeaders()); + } + + private void expectConsumerPoll(final int numMessages, Headers headers) { + expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, headers); } private void expectConsumerPoll(final int numMessages, final long timestamp, final TimestampType timestampType) { + expectConsumerPoll(numMessages, timestamp, timestampType, emptyHeaders()); + } + + private void expectConsumerPoll(final int numMessages, final long timestamp, final TimestampType timestampType, Headers headers) { EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( new IAnswer>() { @Override public ConsumerRecords answer() throws Throwable { List> records = new ArrayList<>(); for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE)); + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE, headers)); recordsReturnedTp1 += numMessages; return new ConsumerRecords<>( numMessages > 0 ? @@ -1367,14 +1464,40 @@ public ConsumerRecords answer() throws Throwable { }); } + private void expectConsumerPoll(List> records) { + EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( + new IAnswer>() { + @Override + public ConsumerRecords answer() throws Throwable { + return new ConsumerRecords<>( + records.isEmpty() ? + Collections.>>emptyMap() : + Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) + ); + } + }); + } + private void expectConversionAndTransformation(final int numMessages) { expectConversionAndTransformation(numMessages, null); } private void expectConversionAndTransformation(final int numMessages, final String topicPrefix) { - EasyMock.expect(keyConverter.toConnectData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).times(numMessages); - EasyMock.expect(valueConverter.toConnectData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).times(numMessages); + expectConversionAndTransformation(numMessages, topicPrefix, emptyHeaders()); + } + private void expectConversionAndTransformation(final int numMessages, final String topicPrefix, final Headers headers) { + EasyMock.expect(keyConverter.toConnectData(TOPIC, headers, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).times(numMessages); + EasyMock.expect(valueConverter.toConnectData(TOPIC, headers, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).times(numMessages); + + for (Header header : headers) { + EasyMock.expect(headerConverter.toConnectHeader(TOPIC, header.key(), header.value())).andReturn(new SchemaAndValue(VALUE_SCHEMA, new String(header.value()))).times(1); + } + + expectTransformation(numMessages, topicPrefix); + } + + private void expectTransformation(final int numMessages, final String topicPrefix) { final Capture recordCapture = EasyMock.newCapture(); EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))) .andAnswer(new IAnswer() { @@ -1389,7 +1512,8 @@ public SinkRecord answer() { origRecord.key(), origRecord.valueSchema(), origRecord.value(), - origRecord.timestamp() + origRecord.timestamp(), + origRecord.headers() ) : origRecord; } @@ -1472,6 +1596,10 @@ private void assertMetrics(int minimumPollCountExpected) { double sendTotal = metrics.currentMetricValueAsDouble(sinkTaskGroup, "sink-record-send-total"); } + private RecordHeaders emptyHeaders() { + return new RecordHeaders(); + } + private abstract static class TestSinkTask extends SinkTask { } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 6e2b01ce7bc1..ab20dce2d26b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; @@ -572,8 +573,8 @@ public ConsumerRecords answer() throws Throwable { return records; } }); - EasyMock.expect(keyConverter.toConnectData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).anyTimes(); - EasyMock.expect(valueConverter.toConnectData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).anyTimes(); + EasyMock.expect(keyConverter.toConnectData(TOPIC, emptyHeaders(), RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).anyTimes(); + EasyMock.expect(valueConverter.toConnectData(TOPIC, emptyHeaders(), RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).anyTimes(); final Capture recordCapture = EasyMock.newCapture(); EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))).andAnswer( @@ -606,8 +607,8 @@ public ConsumerRecords answer() throws Throwable { return records; } }); - EasyMock.expect(keyConverter.toConnectData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); - EasyMock.expect(valueConverter.toConnectData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); + EasyMock.expect(keyConverter.toConnectData(TOPIC, emptyHeaders(), RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); + EasyMock.expect(valueConverter.toConnectData(TOPIC, emptyHeaders(), RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); sinkTask.put(EasyMock.anyObject(Collection.class)); return EasyMock.expectLastCall(); } @@ -651,8 +652,8 @@ public ConsumerRecords answer() throws Throwable { consumer.seek(TOPIC_PARTITION, startOffset); EasyMock.expectLastCall(); - EasyMock.expect(keyConverter.toConnectData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); - EasyMock.expect(valueConverter.toConnectData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); + EasyMock.expect(keyConverter.toConnectData(TOPIC, emptyHeaders(), RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); + EasyMock.expect(valueConverter.toConnectData(TOPIC, emptyHeaders(), RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); sinkTask.put(EasyMock.anyObject(Collection.class)); return EasyMock.expectLastCall(); } @@ -694,6 +695,10 @@ public Object answer() throws Throwable { return capturedCallback; } + private RecordHeaders emptyHeaders() { + return new RecordHeaders(); + } + private static abstract class TestSinkTask extends SinkTask { } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 1f5cc438ef71..272a6d56e6a2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -16,15 +16,21 @@ */ package org.apache.kafka.connect.runtime; +import java.nio.ByteBuffer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.record.InvalidRecordException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.WorkerSourceTask.SourceTaskMetricsGroup; @@ -39,6 +45,7 @@ import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.OffsetStorageReader; import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.ThreadedTest; @@ -152,6 +159,10 @@ private void createWorkerTask() { } private void createWorkerTask(TargetState initialState) { + createWorkerTask(initialState, keyConverter, valueConverter, headerConverter); + } + + private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter, transformationChain, producer, offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, RetryWithToleranceOperatorTest.NOOP_OPERATOR); @@ -682,6 +693,80 @@ public void testMetricsGroup() { assertEquals(1800.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-active-count"), 0.001d); } + @Test + public void testHeaders() throws Exception { + Headers headers = new RecordHeaders(); + headers.add("header_key", "header_value".getBytes()); + + org.apache.kafka.connect.header.Headers connectHeaders = new ConnectHeaders(); + connectHeaders.add("header_key", new SchemaAndValue(Schema.STRING_SCHEMA, "header_value")); + + createWorkerTask(); + + List records = new ArrayList<>(); + records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, null, connectHeaders)); + + Capture> sent = expectSendRecord(true, false, true, true, true, headers); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", records); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(SERIALIZED_KEY, sent.getValue().key()); + assertEquals(SERIALIZED_RECORD, sent.getValue().value()); + assertEquals(headers, sent.getValue().headers()); + + PowerMock.verifyAll(); + } + + @Test + public void testHeadersWithCustomConverter() throws Exception { + StringConverter stringConverter = new StringConverter(); + TestConverterWithHeaders testConverter = new TestConverterWithHeaders(); + + createWorkerTask(TargetState.STARTED, stringConverter, testConverter, stringConverter); + + List records = new ArrayList<>(); + + String stringA = "Árvíztűrő tükörfúrógép"; + org.apache.kafka.connect.header.Headers headersA = new ConnectHeaders(); + String encodingA = "latin2"; + headersA.addString("encoding", encodingA); + + records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "a", Schema.STRING_SCHEMA, stringA, null, headersA)); + + String stringB = "Тестовое сообщение"; + org.apache.kafka.connect.header.Headers headersB = new ConnectHeaders(); + String encodingB = "koi8_r"; + headersB.addString("encoding", encodingB); + + records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "b", Schema.STRING_SCHEMA, stringB, null, headersB)); + + Capture> sentRecordA = expectSendRecord(false, false, true, true, false, null); + Capture> sentRecordB = expectSendRecord(false, false, true, true, false, null); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", records); + Whitebox.invokeMethod(workerTask, "sendRecords"); + + assertEquals(ByteBuffer.wrap("a".getBytes()), ByteBuffer.wrap(sentRecordA.getValue().key())); + assertEquals( + ByteBuffer.wrap(stringA.getBytes(encodingA)), + ByteBuffer.wrap(sentRecordA.getValue().value()) + ); + assertEquals(encodingA, new String(sentRecordA.getValue().headers().lastHeader("encoding").value())); + + assertEquals(ByteBuffer.wrap("b".getBytes()), ByteBuffer.wrap(sentRecordB.getValue().key())); + assertEquals( + ByteBuffer.wrap(stringB.getBytes(encodingB)), + ByteBuffer.wrap(sentRecordB.getValue().value()) + ); + assertEquals(encodingB, new String(sentRecordB.getValue().headers().lastHeader("encoding").value())); + + PowerMock.verifyAll(); + } + private CountDownLatch expectPolls(int minimum, final AtomicInteger count) throws InterruptedException { final CountDownLatch latch = new CountDownLatch(minimum); // Note that we stub these to allow any number of calls because the thread will continue to @@ -708,7 +793,7 @@ private CountDownLatch expectPolls(int count) throws InterruptedException { @SuppressWarnings("unchecked") private void expectSendRecordSyncFailure(Throwable error) throws InterruptedException { - expectConvertKeyValue(false); + expectConvertHeadersAndKeyValue(false); expectApplyTransformationChain(false); offsetWriter.offset(PARTITION, OFFSET); @@ -729,24 +814,34 @@ private Capture> expectSendRecordOnce(boolean isR } private Capture> expectSendRecordProducerCallbackFail() throws InterruptedException { - return expectSendRecord(false, false, false, false); + return expectSendRecord(false, false, false, false, true, emptyHeaders()); } private Capture> expectSendRecordTaskCommitRecordSucceed(boolean anyTimes, boolean isRetry) throws InterruptedException { - return expectSendRecord(anyTimes, isRetry, true, true); + return expectSendRecord(anyTimes, isRetry, true, true, true, emptyHeaders()); } private Capture> expectSendRecordTaskCommitRecordFail(boolean anyTimes, boolean isRetry) throws InterruptedException { - return expectSendRecord(anyTimes, isRetry, true, false); + return expectSendRecord(anyTimes, isRetry, true, false, true, emptyHeaders()); + } + + private Capture> expectSendRecord(boolean anyTimes, boolean isRetry, boolean succeed) throws InterruptedException { + return expectSendRecord(anyTimes, isRetry, succeed, true, true, emptyHeaders()); } + @SuppressWarnings("unchecked") private Capture> expectSendRecord( boolean anyTimes, boolean isRetry, boolean sendSuccess, - boolean commitSuccess + boolean commitSuccess, + boolean isMockedConverters, + Headers headers ) throws InterruptedException { - expectConvertKeyValue(anyTimes); + if (isMockedConverters) { + expectConvertHeadersAndKeyValue(anyTimes, headers); + } + expectApplyTransformationChain(anyTimes); Capture> sent = EasyMock.newCapture(); @@ -794,13 +889,24 @@ public Future answer() throws Throwable { return sent; } - private void expectConvertKeyValue(boolean anyTimes) { - IExpectationSetters convertKeyExpect = EasyMock.expect(keyConverter.fromConnectData(TOPIC, KEY_SCHEMA, KEY)); + private void expectConvertHeadersAndKeyValue(boolean anyTimes) { + expectConvertHeadersAndKeyValue(anyTimes, emptyHeaders()); + } + + private void expectConvertHeadersAndKeyValue(boolean anyTimes, Headers headers) { + for (Header header : headers) { + IExpectationSetters convertHeaderExpect = EasyMock.expect(headerConverter.fromConnectHeader(TOPIC, header.key(), Schema.STRING_SCHEMA, new String(header.value()))); + if (anyTimes) + convertHeaderExpect.andStubReturn(header.value()); + else + convertHeaderExpect.andReturn(header.value()); + } + IExpectationSetters convertKeyExpect = EasyMock.expect(keyConverter.fromConnectData(TOPIC, headers, KEY_SCHEMA, KEY)); if (anyTimes) convertKeyExpect.andStubReturn(SERIALIZED_KEY); else convertKeyExpect.andReturn(SERIALIZED_KEY); - IExpectationSetters convertValueExpect = EasyMock.expect(valueConverter.fromConnectData(TOPIC, RECORD_SCHEMA, RECORD)); + IExpectationSetters convertValueExpect = EasyMock.expect(valueConverter.fromConnectData(TOPIC, headers, RECORD_SCHEMA, RECORD)); if (anyTimes) convertValueExpect.andStubReturn(SERIALIZED_RECORD); else @@ -902,6 +1008,10 @@ private void assertPollMetrics(int minimumPollCountExpected) { } } + private RecordHeaders emptyHeaders() { + return new RecordHeaders(); + } + private abstract static class TestSourceTask extends SourceTask { } diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 1b74eb6fc6e3..9203334e4615 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -128,7 +128,7 @@ object ConfigCommand extends Config { if (dynamicBrokerConfigs.nonEmpty) { val perBrokerConfig = entityName != ConfigEntityName.Default val errorMessage = s"--bootstrap-server option must be specified to update broker configs $dynamicBrokerConfigs." - val info = "Broker configuraton updates using ZooKeeper are supported for bootstrapping before brokers" + + val info = "Broker configuration updates using ZooKeeper are supported for bootstrapping before brokers" + " are started to enable encrypted password configs to be stored in ZooKeeper." if (perBrokerConfig) { adminZkClient.parseBroker(entityName).foreach { brokerId => @@ -522,9 +522,9 @@ object ConfigCommand extends Config { val nl = System.getProperty("line.separator") val addConfig = parser.accepts("add-config", "Key Value pairs of configs to add. Square brackets can be used to group values which contain commas: 'k1=v1,k2=[v1,v2,v2],k3=v3'. The following is a list of valid configurations: " + "For entity-type '" + ConfigType.Topic + "': " + LogConfig.configNames.map("\t" + _).mkString(nl, nl, nl) + - "For entity-type '" + ConfigType.Broker + "': " + DynamicConfig.Broker.names.asScala.map("\t" + _).mkString(nl, nl, nl) + - "For entity-type '" + ConfigType.User + "': " + DynamicConfig.User.names.asScala.map("\t" + _).mkString(nl, nl, nl) + - "For entity-type '" + ConfigType.Client + "': " + DynamicConfig.Client.names.asScala.map("\t" + _).mkString(nl, nl, nl) + + "For entity-type '" + ConfigType.Broker + "': " + DynamicConfig.Broker.names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + + "For entity-type '" + ConfigType.User + "': " + DynamicConfig.User.names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + + "For entity-type '" + ConfigType.Client + "': " + DynamicConfig.Client.names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + s"Entity types '${ConfigType.User}' and '${ConfigType.Client}' may be specified together to update config for clients of a specific user.") .withRequiredArg .ofType(classOf[String]) diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 8e8ca0b74930..1a3362508c10 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -90,7 +90,9 @@ object ApiVersion { // Introduced static membership. KAFKA_2_3_IV0, // Add rack_id to FetchRequest, preferred_read_replica to FetchResponse, and replica_id to OffsetsForLeaderRequest - KAFKA_2_3_IV1 + KAFKA_2_3_IV1, + // Add adding_replicas and removing_replicas fields to LeaderAndIsrRequest + KAFKA_2_4_IV0 ) // Map keys are the union of the short and full versions @@ -316,6 +318,13 @@ case object KAFKA_2_3_IV1 extends DefaultApiVersion { val id: Int = 23 } +case object KAFKA_2_4_IV0 extends DefaultApiVersion { + val shortVersion: String = "2.4" + val subVersion = "IV0" + val recordVersion = RecordVersion.V2 + val id: Int = 24 +} + object ApiVersionValidator extends Validator { override def ensureValid(name: String, value: Any): Unit = { diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 2ca8148153b1..e8e621f03555 100755 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -29,6 +29,13 @@ import org.apache.kafka.server.authorizer.AuthorizerServerInfo import scala.collection.Seq import scala.collection.JavaConverters._ +object Broker { + private[cluster] case class ServerInfo(clusterResource: ClusterResource, + brokerId: Int, + endpoints: util.List[Endpoint], + interBrokerEndpoint: Endpoint) extends AuthorizerServerInfo +} + /** * A Kafka broker. * A broker has an id, a collection of end-points, an optional rack and a listener to security protocol map. @@ -75,15 +82,8 @@ case class Broker(id: Int, endPoints: Seq[EndPoint], rack: Option[String]) { def toServerInfo(clusterId: String, config: KafkaConfig): AuthorizerServerInfo = { val clusterResource: ClusterResource = new ClusterResource(clusterId) - val interBrokerEndpoint: Endpoint = endPoint(config.interBrokerListenerName) - val brokerEndpoints: util.List[Endpoint] = endPoints.toList.map(_.asInstanceOf[Endpoint]).asJava - BrokerEndpointInfo(clusterResource, id, brokerEndpoints, interBrokerEndpoint) + val interBrokerEndpoint: Endpoint = endPoint(config.interBrokerListenerName).toJava + val brokerEndpoints: util.List[Endpoint] = endPoints.toList.map(_.toJava).asJava + Broker.ServerInfo(clusterResource, id, brokerEndpoints, interBrokerEndpoint) } - - case class BrokerEndpointInfo(clusterResource: ClusterResource, - brokerId: Int, - endpoints: util.List[Endpoint], - interBrokerEndpoint: Endpoint) - extends AuthorizerServerInfo - } diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 4fbae70baadd..2f8229a38865 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -62,8 +62,7 @@ object EndPoint { /** * Part of the broker definition - matching host/port pair to a protocol */ -case class EndPoint(override val host: String, override val port: Int, listenerName: ListenerName, override val securityProtocol: SecurityProtocol) - extends JEndpoint(Option(listenerName).map(_.value).orNull, securityProtocol, host, port) { +case class EndPoint(host: String, port: Int, listenerName: ListenerName, securityProtocol: SecurityProtocol) { def connectionString: String = { val hostport = if (host == null) @@ -73,7 +72,7 @@ case class EndPoint(override val host: String, override val port: Int, listenerN listenerName.value + "://" + hostport } - // to keep spotbugs happy - override def equals(o: scala.Any): Boolean = super.equals(o) - override def hashCode(): Int = super.hashCode() + def toJava: JEndpoint = { + new JEndpoint(listenerName.value, securityProtocol, host, port) + } } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 03395a4e4e56..c3282b778f1e 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -19,6 +19,7 @@ package kafka.cluster import com.yammer.metrics.core.Gauge import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.{Optional, Properties} + import kafka.api.{ApiVersion, LeaderAndIsr, Request} import kafka.common.UnexpectedAppendOffsetException import kafka.controller.KafkaController @@ -31,6 +32,7 @@ import kafka.utils._ import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors._ +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors._ import org.apache.kafka.common.record.FileRecords.TimestampAndOffset @@ -38,6 +40,7 @@ import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} import org.apache.kafka.common.requests.EpochEndOffset._ import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.Time + import scala.collection.JavaConverters._ import scala.collection.{Map, Seq} @@ -404,8 +407,9 @@ class Partition(val topicPartition: TopicPartition, this.log = Some(log) } - def remoteReplicas: Set[Replica] = - remoteReplicasMap.values.toSet + // remoteReplicas will be called in the hot path, and must be inexpensive + def remoteReplicas: Iterable[Replica] = + remoteReplicasMap.values def futureReplicaDirChanged(newDestinationDir: String): Boolean = { inReadLock(leaderIsrUpdateLock) { @@ -475,29 +479,29 @@ class Partition(val topicPartition: TopicPartition, * If the leader replica id does not change, return false to indicate the replica manager. */ def makeLeader(controllerId: Int, - partitionStateInfo: LeaderAndIsrRequest.PartitionState, + partitionState: LeaderAndIsrPartitionState, correlationId: Int, highWatermarkCheckpoints: OffsetCheckpoints): Boolean = { val (leaderHWIncremented, isNewLeader) = inWriteLock(leaderIsrUpdateLock) { // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path - controllerEpoch = partitionStateInfo.basePartitionState.controllerEpoch + controllerEpoch = partitionState.controllerEpoch updateAssignmentAndIsr( - assignment = partitionStateInfo.basePartitionState.replicas.asScala.iterator.map(_.toInt).toSeq, - isr = partitionStateInfo.basePartitionState.isr.asScala.iterator.map(_.toInt).toSet + assignment = partitionState.replicas.asScala.iterator.map(_.toInt).toSeq, + isr = partitionState.isr.asScala.iterator.map(_.toInt).toSet ) - createLogIfNotExists(localBrokerId, partitionStateInfo.isNew, isFutureReplica = false, highWatermarkCheckpoints) + createLogIfNotExists(localBrokerId, partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints) val leaderLog = localLogOrException val leaderEpochStartOffset = leaderLog.logEndOffset - info(s"$topicPartition starts at Leader Epoch ${partitionStateInfo.basePartitionState.leaderEpoch} from " + + info(s"$topicPartition starts at Leader Epoch ${partitionState.leaderEpoch} from " + s"offset $leaderEpochStartOffset. Previous Leader Epoch was: $leaderEpoch") //We cache the leader epoch here, persisting it only if it's local (hence having a log dir) - leaderEpoch = partitionStateInfo.basePartitionState.leaderEpoch + leaderEpoch = partitionState.leaderEpoch leaderEpochStartOffsetOpt = Some(leaderEpochStartOffset) - zkVersion = partitionStateInfo.basePartitionState.zkVersion + zkVersion = partitionState.zkVersion // 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 @@ -545,25 +549,25 @@ class Partition(val topicPartition: TopicPartition, * replica manager that state is already correct and the become-follower steps can be skipped */ def makeFollower(controllerId: Int, - partitionStateInfo: LeaderAndIsrRequest.PartitionState, + partitionState: LeaderAndIsrPartitionState, correlationId: Int, highWatermarkCheckpoints: OffsetCheckpoints): Boolean = { inWriteLock(leaderIsrUpdateLock) { - val newLeaderBrokerId = partitionStateInfo.basePartitionState.leader + val newLeaderBrokerId = partitionState.leader val oldLeaderEpoch = leaderEpoch // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path - controllerEpoch = partitionStateInfo.basePartitionState.controllerEpoch + controllerEpoch = partitionState.controllerEpoch updateAssignmentAndIsr( - assignment = partitionStateInfo.basePartitionState.replicas.asScala.iterator.map(_.toInt).toSeq, + assignment = partitionState.replicas.asScala.iterator.map(_.toInt).toSeq, isr = Set.empty[Int] ) - createLogIfNotExists(localBrokerId, partitionStateInfo.isNew, isFutureReplica = false, highWatermarkCheckpoints) + createLogIfNotExists(localBrokerId, partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints) - leaderEpoch = partitionStateInfo.basePartitionState.leaderEpoch + leaderEpoch = partitionState.leaderEpoch leaderEpochStartOffsetOpt = None - zkVersion = partitionStateInfo.basePartitionState.zkVersion + zkVersion = partitionState.zkVersion if (leaderReplicaIdOpt.contains(newLeaderBrokerId) && leaderEpoch == oldLeaderEpoch) { false @@ -585,31 +589,41 @@ class Partition(val topicPartition: TopicPartition, followerStartOffset: Long, followerFetchTimeMs: Long, leaderEndOffset: Long): Boolean = { - getReplica(followerId) match { case Some(followerReplica) => // No need to calculate low watermark if there is no delayed DeleteRecordsRequest val oldLeaderLW = if (delayedOperations.numDelayedDelete > 0) lowWatermarkIfLeader else -1L + val prevFollowerEndOffset = followerReplica.logEndOffset followerReplica.updateFetchState( followerFetchOffsetMetadata, followerStartOffset, followerFetchTimeMs, leaderEndOffset) + val newLeaderLW = if (delayedOperations.numDelayedDelete > 0) lowWatermarkIfLeader else -1L // check if the LW of the partition has incremented // since the replica's logStartOffset may have incremented val leaderLWIncremented = newLeaderLW > oldLeaderLW + // check if we need to expand ISR to include this replica // if it is not in the ISR yet - val followerFetchOffset = followerFetchOffsetMetadata.messageOffset - val leaderHWIncremented = maybeExpandIsr(followerReplica, followerFetchTimeMs) + if (!inSyncReplicaIds(followerId)) + maybeExpandIsr(followerReplica, followerFetchTimeMs) + + // check if the HW of the partition can now be incremented + // since the replica may already be in the ISR and its LEO has just incremented + val leaderHWIncremented = if (prevFollowerEndOffset != followerReplica.logEndOffset) { + leaderLogIfLocal.exists(leaderLog => maybeIncrementLeaderHW(leaderLog, followerFetchTimeMs)) + } else { + false + } // some delayed operations may be unblocked after HW or LW changed if (leaderLWIncremented || leaderHWIncremented) tryCompleteDelayedRequests() debug(s"Recorded replica $followerId log end offset (LEO) position " + - s"$followerFetchOffset and log start offset $followerStartOffset.") + s"${followerFetchOffsetMetadata.messageOffset} and log start offset $followerStartOffset.") true case None => @@ -654,27 +668,20 @@ class Partition(val topicPartition: TopicPartition, * whether a replica is in-sync, we only check HW. * * This function can be triggered when a replica's LEO has incremented. - * - * @return true if the high watermark has been updated */ - private def maybeExpandIsr(followerReplica: Replica, followerFetchTimeMs: Long): Boolean = { + private def maybeExpandIsr(followerReplica: Replica, followerFetchTimeMs: Long): Unit = { inWriteLock(leaderIsrUpdateLock) { // check if this replica needs to be added to the ISR - leaderLogIfLocal match { - case Some(leaderLog) => - val leaderHighwatermark = leaderLog.highWatermark - if (!inSyncReplicaIds.contains(followerReplica.brokerId) && isFollowerInSync(followerReplica, leaderHighwatermark)) { - val newInSyncReplicaIds = inSyncReplicaIds + followerReplica.brokerId - info(s"Expanding ISR from ${inSyncReplicaIds.mkString(",")} " + - s"to ${newInSyncReplicaIds.mkString(",")}") - - // update ISR in ZK and cache - expandIsr(newInSyncReplicaIds) - } - // check if the HW of the partition can now be incremented - // since the replica may already be in the ISR and its LEO has just incremented - maybeIncrementLeaderHW(leaderLog, followerFetchTimeMs) - case None => false // nothing to do if no longer leader + leaderLogIfLocal.foreach { leaderLog => + val leaderHighwatermark = leaderLog.highWatermark + if (!inSyncReplicaIds.contains(followerReplica.brokerId) && isFollowerInSync(followerReplica, leaderHighwatermark)) { + val newInSyncReplicaIds = inSyncReplicaIds + followerReplica.brokerId + info(s"Expanding ISR from ${inSyncReplicaIds.mkString(",")} " + + s"to ${newInSyncReplicaIds.mkString(",")}") + + // update ISR in ZK and cache + expandIsr(newInSyncReplicaIds) + } } } } @@ -749,25 +756,35 @@ class Partition(val topicPartition: TopicPartition, * since all callers of this private API acquire that lock */ private def maybeIncrementLeaderHW(leaderLog: Log, curTime: Long = time.milliseconds): Boolean = { - val replicaLogEndOffsets = remoteReplicas.filter { replica => - curTime - replica.lastCaughtUpTimeMs <= replicaLagTimeMaxMs || inSyncReplicaIds.contains(replica.brokerId) - }.map(_.logEndOffsetMetadata) - val newHighWatermark = (replicaLogEndOffsets + leaderLog.logEndOffsetMetadata).min(new LogOffsetMetadata.OffsetOrdering) - leaderLog.maybeIncrementHighWatermark(newHighWatermark) match { - case Some(oldHighWatermark) => - debug(s"High watermark updated from $oldHighWatermark to $newHighWatermark") - true - - case None => - def logEndOffsetString: ((Int, LogOffsetMetadata)) => String = { - case (brokerId, logEndOffsetMetadata) => s"replica $brokerId: $logEndOffsetMetadata" + inReadLock(leaderIsrUpdateLock) { + // maybeIncrementLeaderHW is in the hot path, the following code is written to + // avoid unnecessary collection generation + var newHighWatermark = leaderLog.logEndOffsetMetadata + remoteReplicasMap.values.foreach { replica => + if (replica.logEndOffsetMetadata.messageOffset < newHighWatermark.messageOffset && + (curTime - replica.lastCaughtUpTimeMs <= replicaLagTimeMaxMs || inSyncReplicaIds.contains(replica.brokerId))) { + newHighWatermark = replica.logEndOffsetMetadata } + } - val replicaInfo = remoteReplicas.map(replica => (replica.brokerId, replica.logEndOffsetMetadata)) - val localLogInfo = (localBrokerId, localLogOrException.logEndOffsetMetadata) - trace(s"Skipping update high watermark since new hw $newHighWatermark is not larger than old value. " + - s"All current LEOs are ${(replicaInfo + localLogInfo).map(logEndOffsetString)}") - false + leaderLog.maybeIncrementHighWatermark(newHighWatermark) match { + case Some(oldHighWatermark) => + debug(s"High watermark updated from $oldHighWatermark to $newHighWatermark") + true + + case None => + def logEndOffsetString: ((Int, LogOffsetMetadata)) => String = { + case (brokerId, logEndOffsetMetadata) => s"replica $brokerId: $logEndOffsetMetadata" + } + + if (isTraceEnabled) { + val replicaInfo = remoteReplicas.map(replica => (replica.brokerId, replica.logEndOffsetMetadata)).toSet + val localLogInfo = (localBrokerId, localLogOrException.logEndOffsetMetadata) + trace(s"Skipping update high watermark since new hw $newHighWatermark is not larger than old value. " + + s"All current LEOs are ${(replicaInfo + localLogInfo).map(logEndOffsetString)}") + } + false + } } } @@ -779,15 +796,21 @@ class Partition(val topicPartition: TopicPartition, def lowWatermarkIfLeader: Long = { if (!isLeader) throw new NotLeaderForPartitionException(s"Leader not local for partition $topicPartition on broker $localBrokerId") - val logStartOffsets = remoteReplicas.collect { - case replica if metadataCache.getAliveBroker(replica.brokerId).nonEmpty => replica.logStartOffset - } + localLogOrException.logStartOffset + + // lowWatermarkIfLeader may be called many times when a DeleteRecordsRequest is outstanding, + // care has been taken to avoid generating unnecessary collections in this code + var lowWaterMark = localLogOrException.logStartOffset + remoteReplicas.foreach { replica => + if (metadataCache.getAliveBroker(replica.brokerId).nonEmpty && replica.logStartOffset < lowWaterMark) { + lowWaterMark = replica.logStartOffset + } + } futureLog match { case Some(partitionFutureLog) => - CoreUtils.min(logStartOffsets + partitionFutureLog.logStartOffset, 0L) + Math.min(lowWaterMark, partitionFutureLog.logStartOffset) case None => - CoreUtils.min(logStartOffsets, 0L) + lowWaterMark } } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index e91e48d80cf1..6c7450a0c7a9 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -26,10 +26,11 @@ import kafka.metrics.KafkaMetricsGroup import kafka.server.KafkaConfig import kafka.utils._ import org.apache.kafka.clients._ +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network._ -import org.apache.kafka.common.protocol.ApiKeys -import org.apache.kafka.common.requests.UpdateMetadataRequest.EndPoint +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol @@ -341,10 +342,10 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, controllerContext: ControllerContext, stateChangeLogger: StateChangeLogger) extends Logging { val controllerId: Int = config.brokerId - val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrRequest.PartitionState]] + val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrPartitionState]] val stopReplicaRequestMap = mutable.Map.empty[Int, ListBuffer[StopReplicaRequestInfo]] val updateMetadataRequestBrokerSet = mutable.Set.empty[Int] - val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, UpdateMetadataRequest.PartitionState] + val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, UpdateMetadataPartitionState] def sendEvent(event: ControllerEvent): Unit @@ -382,15 +383,19 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, brokerIds.filter(_ >= 0).foreach { brokerId => val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty) val alreadyNew = result.get(topicPartition).exists(_.isNew) - result.put(topicPartition, new LeaderAndIsrRequest.PartitionState(leaderIsrAndControllerEpoch.controllerEpoch, - leaderIsrAndControllerEpoch.leaderAndIsr.leader, - leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch, - leaderIsrAndControllerEpoch.leaderAndIsr.isr.map(Integer.valueOf).asJava, - leaderIsrAndControllerEpoch.leaderAndIsr.zkVersion, - replicaAssignment.replicas.map(Integer.valueOf).asJava, - replicaAssignment.addingReplicas.map(Integer.valueOf).asJava, - replicaAssignment.removingReplicas.map(Integer.valueOf).asJava, - isNew || alreadyNew)) + val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr + result.put(topicPartition, new LeaderAndIsrPartitionState() + .setTopicName(topicPartition.topic) + .setPartitionIndex(topicPartition.partition) + .setControllerEpoch(leaderIsrAndControllerEpoch.controllerEpoch) + .setLeader(leaderAndIsr.leader) + .setLeaderEpoch(leaderAndIsr.leaderEpoch) + .setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava) + .setZkVersion(leaderAndIsr.zkVersion) + .setReplicas(replicaAssignment.replicas.map(Integer.valueOf).asJava) + .setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava) + .setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava) + .setIsNew(isNew || alreadyNew)) } addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition)) @@ -410,25 +415,24 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, partitions: collection.Set[TopicPartition]): Unit = { def updateMetadataRequestPartitionInfo(partition: TopicPartition, beingDeleted: Boolean): Unit = { - val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition) - leaderIsrAndControllerEpochOpt match { - case Some(l @ LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) => + controllerContext.partitionLeadershipInfo.get(partition) match { + case Some(LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) => val replicas = controllerContext.partitionReplicaAssignment(partition) val offlineReplicas = replicas.filter(!controllerContext.isReplicaOnline(_, partition)) - val leaderIsrAndControllerEpoch = if (beingDeleted) { - val leaderDuringDelete = LeaderAndIsr.duringDelete(leaderAndIsr.isr) - LeaderIsrAndControllerEpoch(leaderDuringDelete, controllerEpoch) - } else { - l - } - - val partitionStateInfo = new UpdateMetadataRequest.PartitionState(leaderIsrAndControllerEpoch.controllerEpoch, - leaderIsrAndControllerEpoch.leaderAndIsr.leader, - leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch, - leaderIsrAndControllerEpoch.leaderAndIsr.isr.map(Integer.valueOf).asJava, - leaderIsrAndControllerEpoch.leaderAndIsr.zkVersion, - replicas.map(Integer.valueOf).asJava, - offlineReplicas.map(Integer.valueOf).asJava) + val updatedLeaderAndIsr = + if (beingDeleted) LeaderAndIsr.duringDelete(leaderAndIsr.isr) + else leaderAndIsr + + val partitionStateInfo = new UpdateMetadataPartitionState() + .setTopicName(partition.topic) + .setPartitionIndex(partition.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(updatedLeaderAndIsr.leader) + .setLeaderEpoch(updatedLeaderAndIsr.leaderEpoch) + .setIsr(updatedLeaderAndIsr.isr.map(Integer.valueOf).asJava) + .setZkVersion(updatedLeaderAndIsr.zkVersion) + .setReplicas(replicas.map(Integer.valueOf).asJava) + .setOfflineReplicas(offlineReplicas.map(Integer.valueOf).asJava) updateMetadataRequestPartitionInfoMap.put(partition, partitionStateInfo) case None => @@ -443,26 +447,28 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, private def sendLeaderAndIsrRequest(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { val leaderAndIsrRequestVersion: Short = - if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 2 + if (config.interBrokerProtocolVersion >= KAFKA_2_4_IV0) 3 + else if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 2 else if (config.interBrokerProtocolVersion >= KAFKA_1_0_IV0) 1 else 0 leaderAndIsrRequestMap.filterKeys(controllerContext.liveOrShuttingDownBrokerIds.contains).foreach { case (broker, leaderAndIsrPartitionStates) => - leaderAndIsrPartitionStates.foreach { - case (topicPartition, state) => + if (stateChangeLog.isTraceEnabled) { + leaderAndIsrPartitionStates.foreach { case (topicPartition, state) => val typeOfRequest = - if (broker == state.basePartitionState.leader) "become-leader" + if (broker == state.leader) "become-leader" else "become-follower" stateChangeLog.trace(s"Sending $typeOfRequest LeaderAndIsr request $state to broker $broker for partition $topicPartition") + } } - val leaderIds = leaderAndIsrPartitionStates.map(_._2.basePartitionState.leader).toSet + val leaderIds = leaderAndIsrPartitionStates.map(_._2.leader).toSet val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map { _.node(config.interBrokerListenerName) } val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker) val leaderAndIsrRequestBuilder = new LeaderAndIsrRequest.Builder(leaderAndIsrRequestVersion, controllerId, controllerEpoch, - brokerEpoch, leaderAndIsrPartitionStates.asJava, leaders.asJava) + brokerEpoch, leaderAndIsrPartitionStates.values.toBuffer.asJava, leaders.asJava) sendRequest(broker, leaderAndIsrRequestBuilder, (r: AbstractResponse) => sendEvent(LeaderAndIsrResponseReceived(r, broker))) } @@ -475,7 +481,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, s"for partition $tp") } - val partitionStates = Map.empty ++ updateMetadataRequestPartitionInfoMap + val partitionStates = updateMetadataRequestPartitionInfoMap.values.toBuffer val updateMetadataRequestVersion: Short = if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 5 else if (config.interBrokerProtocolVersion >= KAFKA_1_0_IV0) 4 @@ -484,23 +490,31 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, else if (config.interBrokerProtocolVersion >= KAFKA_0_9_0) 1 else 0 - val liveBrokers = if (updateMetadataRequestVersion == 0) { - // Version 0 of UpdateMetadataRequest only supports PLAINTEXT. - controllerContext.liveOrShuttingDownBrokers.map { broker => + val liveBrokers = controllerContext.liveOrShuttingDownBrokers.iterator.map { broker => + val endpoints = if (updateMetadataRequestVersion == 0) { + // Version 0 of UpdateMetadataRequest only supports PLAINTEXT val securityProtocol = SecurityProtocol.PLAINTEXT val listenerName = ListenerName.forSecurityProtocol(securityProtocol) val node = broker.node(listenerName) - val endPoints = Seq(new EndPoint(node.host, node.port, securityProtocol, listenerName)) - new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull) - } - } else { - controllerContext.liveOrShuttingDownBrokers.map { broker => - val endPoints = broker.endPoints.map { endPoint => - new UpdateMetadataRequest.EndPoint(endPoint.host, endPoint.port, endPoint.securityProtocol, endPoint.listenerName) + Seq(new UpdateMetadataEndpoint() + .setHost(node.host) + .setPort(node.port) + .setSecurityProtocol(securityProtocol.id) + .setListener(listenerName.value)) + } else { + broker.endPoints.map { endpoint => + new UpdateMetadataEndpoint() + .setHost(endpoint.host) + .setPort(endpoint.port) + .setSecurityProtocol(endpoint.securityProtocol.id) + .setListener(endpoint.listenerName.value) } - new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull) } - } + new UpdateMetadataBroker() + .setId(broker.id) + .setEndpoints(endpoints.asJava) + .setRack(broker.rack.orNull) + }.toBuffer updateMetadataRequestBrokerSet.intersect(controllerContext.liveOrShuttingDownBrokerIds).foreach { broker => val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker) @@ -519,9 +533,9 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, def stopReplicaPartitionDeleteResponseCallback(brokerId: Int)(response: AbstractResponse): Unit = { val stopReplicaResponse = response.asInstanceOf[StopReplicaResponse] - val partitionErrorsForDeletingTopics = stopReplicaResponse.responses.asScala.filterKeys { partition => - controllerContext.isTopicDeletionInProgress(partition.topic) - }.toMap + val partitionErrorsForDeletingTopics = stopReplicaResponse.partitionErrors.asScala.iterator.filter { pe => + controllerContext.isTopicDeletionInProgress(pe.topicName) + }.map(pe => new TopicPartition(pe.topicName, pe.partitionIndex) -> Errors.forCode(pe.errorCode)).toMap if (partitionErrorsForDeletingTopics.nonEmpty) sendEvent(TopicDeletionStopReplicaResponseReceived(brokerId, stopReplicaResponse.error, partitionErrorsForDeletingTopics)) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 04703e68aa51..eae02feab714 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -42,6 +42,7 @@ import org.apache.zookeeper.KeeperException.Code import scala.collection.JavaConverters._ import scala.collection.{Map, Seq, Set, immutable, mutable} +import scala.collection.mutable.ArrayBuffer import scala.util.{Failure, Try} sealed trait ElectionTrigger @@ -1312,12 +1313,17 @@ class KafkaController(val config: KafkaConfig, return } - val offlineReplicas = leaderAndIsrResponse.responses.asScala.collect { - case (tp, error) if error == Errors.KAFKA_STORAGE_ERROR => tp - } - val onlineReplicas = leaderAndIsrResponse.responses.asScala.collect { - case (tp, error) if error == Errors.NONE => tp + val offlineReplicas = new ArrayBuffer[TopicPartition]() + val onlineReplicas = new ArrayBuffer[TopicPartition]() + + leaderAndIsrResponse.partitions.asScala.foreach { partition => + val tp = new TopicPartition(partition.topicName, partition.partitionIndex) + if (partition.errorCode == Errors.KAFKA_STORAGE_ERROR.code) + offlineReplicas += tp + else if (partition.errorCode == Errors.NONE.code) + onlineReplicas += tp } + val previousOfflineReplicas = controllerContext.replicasOnOfflineDirs.getOrElse(brokerId, Set.empty[TopicPartition]) val currentOfflineReplicas = previousOfflineReplicas -- onlineReplicas ++ offlineReplicas controllerContext.replicasOnOfflineDirs.put(brokerId, currentOfflineReplicas) diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 9e52e516b05b..0be2a48867e3 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -30,6 +30,7 @@ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity +import org.apache.kafka.common.metrics.stats.Meter import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.RecordBatch.{NO_PRODUCER_EPOCH, NO_PRODUCER_ID} import org.apache.kafka.common.requests._ @@ -63,6 +64,27 @@ class GroupCoordinator(val brokerId: Int, type JoinCallback = JoinGroupResult => Unit type SyncCallback = SyncGroupResult => Unit + /* setup metrics */ + val offsetDeletionSensor = metrics.sensor("OffsetDeletions") + + offsetDeletionSensor.add(new Meter( + metrics.metricName("offset-deletion-rate", + "group-coordinator-metrics", + "The rate of administrative deleted offsets"), + metrics.metricName("offset-deletion-count", + "group-coordinator-metrics", + "The total number of administrative deleted offsets"))) + + val groupCompletedRebalanceSensor = metrics.sensor("OffsetDeletions") + + groupCompletedRebalanceSensor.add(new Meter( + metrics.metricName("group-completed-rebalance-rate", + "group-coordinator-metrics", + "The rate of completed rebalance"), + metrics.metricName("group-completed-rebalance-count", + "group-coordinator-metrics", + "The total number of completed rebalance"))) + this.logIdent = "[GroupCoordinator " + brokerId + "]: " private val isActive = new AtomicBoolean(false) @@ -407,6 +429,7 @@ class GroupCoordinator(val brokerId: Int, } } }) + groupCompletedRebalanceSensor.record() } case Stable => @@ -519,6 +542,70 @@ class GroupCoordinator(val brokerId: Int, groupErrors } + def handleDeleteOffsets(groupId: String, partitions: Seq[TopicPartition]): (Errors, Map[TopicPartition, Errors]) = { + var groupError: Errors = Errors.NONE + var partitionErrors: Map[TopicPartition, Errors] = Map() + var partitionEligibleForDeletion: Seq[TopicPartition] = Seq() + + validateGroupStatus(groupId, ApiKeys.OFFSET_DELETE) match { + case Some(error) => + groupError = error + + case None => + groupManager.getGroup(groupId) match { + case None => + groupError = if (groupManager.groupNotExists(groupId)) + Errors.GROUP_ID_NOT_FOUND else Errors.NOT_COORDINATOR + + case Some(group) => + group.inLock { + group.currentState match { + case Dead => + groupError = if (groupManager.groupNotExists(groupId)) + Errors.GROUP_ID_NOT_FOUND else Errors.NOT_COORDINATOR + + case Empty => + val (knownPartitions, unknownPartitions) = + partitions.partition(tp => group.offset(tp).nonEmpty) + + partitionEligibleForDeletion = knownPartitions + partitionErrors = unknownPartitions.map(_ -> Errors.UNKNOWN_TOPIC_OR_PARTITION).toMap + + case PreparingRebalance | CompletingRebalance | Stable if group.isConsumerGroup => + val (knownPartitions, unknownPartitions) = + partitions.partition(tp => group.offset(tp).nonEmpty) + + val (consumed, notConsumed) = + knownPartitions.partition(tp => group.isSubscribedToTopic(tp.topic())) + + partitionEligibleForDeletion = notConsumed + partitionErrors = consumed.map(_ -> Errors.GROUP_SUBSCRIBED_TO_TOPIC).toMap ++ + unknownPartitions.map(_ -> Errors.UNKNOWN_TOPIC_OR_PARTITION).toMap + + case _ => + groupError = Errors.NON_EMPTY_GROUP + } + } + + if (partitionEligibleForDeletion.nonEmpty) { + val offsetsRemoved = groupManager.cleanupGroupMetadata(Seq(group), group => { + group.removeOffsets(partitionEligibleForDeletion) + }) + + partitionErrors ++= partitionEligibleForDeletion.map(_ -> Errors.NONE).toMap + + offsetDeletionSensor.record(offsetsRemoved) + + info(s"The following offsets of the group $groupId were deleted: ${partitionEligibleForDeletion.mkString(", ")}. " + + s"A total of $offsetsRemoved offsets were removed.") + } + } + } + + // If there is a group error, the partition errors is empty + groupError -> partitionErrors + } + def handleHeartbeat(groupId: String, memberId: String, groupInstanceId: Option[String], diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala index ca42b0f098c0..b8237686fecb 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala @@ -16,17 +16,21 @@ */ package kafka.coordinator.group +import java.nio.ByteBuffer import java.util.UUID import java.util.concurrent.locks.ReentrantLock import kafka.common.OffsetAndMetadata import kafka.utils.{CoreUtils, Logging, nonthreadsafe} +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.types.SchemaException import org.apache.kafka.common.utils.Time import scala.collection.{Seq, immutable, mutable} +import scala.collection.JavaConverters._ private[group] sealed trait GroupState @@ -136,6 +140,7 @@ private object GroupMetadata { group.addStaticMember(member.groupInstanceId, member.memberId) } }) + group.subscribedTopics = group.computeSubscribedTopics() group } @@ -204,6 +209,10 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState private var receivedTransactionalOffsetCommits = false private var receivedConsumerOffsetCommits = false + // When protocolType == `consumer`, a set of subscribed topics is maintained. The set is + // computed when a new generation is created or when the group is restored from the log. + private var subscribedTopics: Option[Set[String]] = None + var newMemberAdded: Boolean = false def inLock[T](fun: => T): T = CoreUtils.inLock(lock)(fun) @@ -219,6 +228,8 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState def protocolOrNull: String = protocol.orNull def currentStateTimestampOrDefault: Long = currentStateTimestamp.getOrElse(-1) + def isConsumerGroup: Boolean = protocolType.contains(ConsumerProtocol.PROTOCOL_TYPE) + def add(member: MemberMetadata, callback: JoinCallback = null): Unit = { if (members.isEmpty) this.protocolType = Some(member.protocolType) @@ -423,6 +434,54 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState protocolType.contains(memberProtocolType) && memberProtocols.exists(supportedProtocols(_) == members.size) } + def getSubscribedTopics: Option[Set[String]] = subscribedTopics + + /** + * Returns true if the consumer group is actively subscribed to the topic. When the consumer + * group does not know, because the information is not available yet or because the it has + * failed to parse the Consumer Protocol, it returns true to be safe. + */ + def isSubscribedToTopic(topic: String): Boolean = subscribedTopics match { + case Some(topics) => topics.contains(topic) + case None => true + } + + /** + * Collects the set of topics that the members are subscribed to when the Protocol Type is equal + * to 'consumer'. None is returned if + * - the protocol type is not equal to 'consumer'; + * - the protocol is not defined yet; or + * - the protocol metadata does not comply with the schema. + */ + private[group] def computeSubscribedTopics(): Option[Set[String]] = { + protocolType match { + case Some(ConsumerProtocol.PROTOCOL_TYPE) if members.nonEmpty && protocol.isDefined => + try { + Some( + members.map { case (_, member) => + // The consumer protocol is parsed with V0 which is the based prefix of all versions. + // This way the consumer group manager does not depend on any specific existing or + // future versions of the consumer protocol. VO must prefix all new versions. + val buffer = ByteBuffer.wrap(member.metadata(protocol.get)) + ConsumerProtocol.deserializeVersion(buffer) + ConsumerProtocol.deserializeSubscriptionV0(buffer).topics.asScala.toSet + }.reduceLeft(_ ++ _) + ) + } catch { + case e: SchemaException => { + warn(s"Failed to parse Consumer Protocol ${ConsumerProtocol.PROTOCOL_TYPE}:${protocol.get} " + + s"of group $groupId. Consumer group coordinator is not aware of the subscribed topics.", e) + None + } + } + + case Some(ConsumerProtocol.PROTOCOL_TYPE) if members.isEmpty => + Option(Set.empty) + + case _ => None + } + } + def updateMember(member: MemberMetadata, protocols: List[(String, Array[Byte])], callback: JoinCallback) = { @@ -465,10 +524,12 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState if (members.nonEmpty) { generationId += 1 protocol = Some(selectProtocol) + subscribedTopics = computeSubscribedTopics() transitionTo(CompletingRebalance) } else { generationId += 1 protocol = None + subscribedTopics = computeSubscribedTopics() transitionTo(Empty) } receivedConsumerOffsetCommits = false @@ -630,9 +691,11 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState def removeExpiredOffsets(currentTimestamp: Long, offsetRetentionMs: Long): Map[TopicPartition, OffsetAndMetadata] = { - def getExpiredOffsets(baseTimestamp: CommitRecordMetadataAndOffset => Long): Map[TopicPartition, OffsetAndMetadata] = { + def getExpiredOffsets(baseTimestamp: CommitRecordMetadataAndOffset => Long, + subscribedTopics: Set[String] = Set.empty): Map[TopicPartition, OffsetAndMetadata] = { offsets.filter { case (topicPartition, commitRecordMetadataAndOffset) => + !subscribedTopics.contains(topicPartition.topic()) && !pendingOffsetCommits.contains(topicPartition) && { commitRecordMetadataAndOffset.offsetAndMetadata.expireTimestamp match { case None => @@ -656,8 +719,20 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState // expire all offsets with no pending offset commit; // - if there is no current state timestamp (old group metadata schema) and retention period has passed // since the last commit timestamp, expire the offset - getExpiredOffsets(commitRecordMetadataAndOffset => - currentStateTimestamp.getOrElse(commitRecordMetadataAndOffset.offsetAndMetadata.commitTimestamp)) + getExpiredOffsets( + commitRecordMetadataAndOffset => currentStateTimestamp + .getOrElse(commitRecordMetadataAndOffset.offsetAndMetadata.commitTimestamp) + ) + + case Some(ConsumerProtocol.PROTOCOL_TYPE) if subscribedTopics.isDefined => + // consumers exist in the group => + // - if the group is aware of the subscribed topics and retention period had passed since the + // the last commit timestamp, expire the offset. offset with pending offset commit are not + // expired + getExpiredOffsets( + _.offsetAndMetadata.commitTimestamp, + subscribedTopics.get + ) case None => // protocolType is None => standalone (simple) consumer, that uses Kafka for offset storage only diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index 330ccd066846..2cbf7c8af29b 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -36,8 +36,9 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.metrics.stats.Meter import org.apache.kafka.common.metrics.stats.{Avg, Max} -import org.apache.kafka.common.metrics.{MetricConfig, Metrics} +import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.types.Type._ import org.apache.kafka.common.protocol.types._ @@ -95,6 +96,26 @@ class GroupMetadataManager(brokerId: Int, "group-coordinator-metrics", "The avg time it took to load the partitions in the last 30sec"), new Avg()) + val offsetCommitsSensor = metrics.sensor("OffsetCommits") + + offsetCommitsSensor.add(new Meter( + metrics.metricName("offset-commit-rate", + "group-coordinator-metrics", + "The rate of committed offsets"), + metrics.metricName("offset-commit-count", + "group-coordinator-metrics", + "The total number of committed offsets"))) + + val offsetExpiredSensor = metrics.sensor("OffsetExpired") + + offsetExpiredSensor.add(new Meter( + metrics.metricName("offset-expiration-rate", + "group-coordinator-metrics", + "The rate of expired offsets"), + metrics.metricName("offset-expiration-count", + "group-coordinator-metrics", + "The total number of expired offsets"))) + this.logIdent = s"[GroupMetadataManager brokerId=$brokerId] " private def recreateGauge[T](name: String, gauge: Gauge[T]): Gauge[T] = { @@ -359,6 +380,9 @@ class GroupMetadataManager(brokerId: Int, throw new IllegalStateException("Append status %s should only have one partition %s" .format(responseStatus, offsetTopicPartition)) + // record the number of offsets committed to the log + offsetCommitsSensor.record(records.size) + // construct the commit response status and insert // the offset and metadata to cache if the append status has no error val status = responseStatus(offsetTopicPartition) @@ -742,6 +766,7 @@ class GroupMetadataManager(brokerId: Int, val numOffsetsRemoved = cleanupGroupMetadata(groupMetadataCache.values, group => { group.removeExpiredOffsets(currentTimestamp, config.offsetsRetentionMs) }) + offsetCommitsSensor.record(numOffsetsRemoved) info(s"Removed $numOffsetsRemoved expired offsets in ${time.milliseconds() - currentTimestamp} milliseconds.") } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 370074929ca3..48fc9e9a7d5a 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -341,16 +341,18 @@ class Log(@volatile var dir: File, throw new IllegalArgumentException(s"High watermark $newHighWatermark update exceeds current " + s"log end offset $logEndOffsetMetadata") - val oldHighWatermark = fetchHighWatermarkMetadata - - // Ensure that the high watermark increases monotonically. We also update the high watermark when the new - // offset metadata is on a newer segment, which occurs whenever the log is rolled to a new segment. - if (oldHighWatermark.messageOffset < newHighWatermark.messageOffset || - (oldHighWatermark.messageOffset == newHighWatermark.messageOffset && oldHighWatermark.onOlderSegment(newHighWatermark))) { - updateHighWatermarkMetadata(newHighWatermark) - Some(oldHighWatermark) - } else { - None + lock.synchronized { + val oldHighWatermark = fetchHighWatermarkMetadata + + // Ensure that the high watermark increases monotonically. We also update the high watermark when the new + // offset metadata is on a newer segment, which occurs whenever the log is rolled to a new segment. + if (oldHighWatermark.messageOffset < newHighWatermark.messageOffset || + (oldHighWatermark.messageOffset == newHighWatermark.messageOffset && oldHighWatermark.onOlderSegment(newHighWatermark))) { + updateHighWatermarkMetadata(newHighWatermark) + Some(oldHighWatermark) + } else { + None + } } } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 47c1fa4f001f..e37eacf8c06b 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -571,12 +571,13 @@ private[log] class Cleaner(val id: Int, val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) transactionMetadata.addAbortedTransactions(abortedTransactions) - val retainDeletes = currentSegment.lastModified > deleteHorizonMs - info(s"Cleaning segment $startOffset in log ${log.name} (largest timestamp ${new Date(currentSegment.largestTimestamp)}) " + - s"into ${cleaned.baseOffset}, ${if(retainDeletes) "retaining" else "discarding"} deletes.") + val retainDeletesAndTxnMarkers = currentSegment.lastModified > deleteHorizonMs + info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + + s"with deletion horizon $deleteHorizonMs, " + + s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletes, log.config.maxMessageSize, + cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, stats) } catch { case e: LogSegmentOffsetOverflowException => @@ -618,7 +619,7 @@ private[log] class Cleaner(val id: Int, * @param sourceRecords The dirty log segment * @param dest The cleaned log segment * @param map The key=>offset mapping - * @param retainDeletes Should delete tombstones be retained while cleaning this segment + * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment * @param maxLogMessageSize The maximum message size of the corresponding topic * @param stats Collector for cleaning statistics */ @@ -626,7 +627,7 @@ private[log] class Cleaner(val id: Int, sourceRecords: FileRecords, dest: LogSegment, map: OffsetMap, - retainDeletes: Boolean, + retainDeletesAndTxnMarkers: Boolean, maxLogMessageSize: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], @@ -637,7 +638,7 @@ private[log] class Cleaner(val id: Int, override def checkBatchRetention(batch: RecordBatch): BatchRetention = { // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. // note that we will never delete a marker until all the records from that transaction are removed. - discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletes) + discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) def isBatchLastRecordOfProducer: Boolean = { // We retain the batch in order to preserve the state of active producers. There are three cases: @@ -667,7 +668,7 @@ private[log] class Cleaner(val id: Int, // The batch is only retained to preserve producer sequence information; the records can be removed false else - Cleaner.this.shouldRetainRecord(map, retainDeletes, batch, record, stats) + Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats) } } diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 6d93d7a276e7..4f2671614bb0 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -113,7 +113,7 @@ case class LogConfig(props: java.util.Map[_, _], overriddenConfigs: Set[String] object LogConfig { def main(args: Array[String]): Unit = { - println(configDef.toHtmlTable) + println(configDef.toHtml) } val SegmentBytesProp = TopicConfig.SEGMENT_BYTES_CONFIG diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index f01a74b8c14d..c90ea7aabe4f 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -1036,7 +1036,7 @@ object LogManager { flushRecoveryOffsetCheckpointMs = config.logFlushOffsetCheckpointIntervalMs, flushStartOffsetCheckpointMs = config.logFlushStartOffsetCheckpointIntervalMs, retentionCheckMs = config.logCleanupIntervalMs, - maxPidExpirationMs = config.transactionIdExpirationMs, + maxPidExpirationMs = config.transactionalIdExpirationMs, scheduler = kafkaScheduler, brokerState = brokerState, brokerTopicStats = brokerTopicStats, diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index f5d740be4ffe..38a46e850410 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -409,7 +409,11 @@ class LogSegment private[log] (val log: FileRecords, def collectAbortedTxns(fetchOffset: Long, upperBoundOffset: Long): TxnIndexSearchResult = txnIndex.collectAbortedTxns(fetchOffset, upperBoundOffset) - override def toString = "LogSegment(baseOffset=" + baseOffset + ", size=" + size + ")" + override def toString: String = "LogSegment(baseOffset=" + baseOffset + + ", size=" + size + + ", lastModifiedTime=" + lastModified + + ", largestTime=" + largestTimestamp + ")" /** * Truncate off all index and log entries with offsets >= the given offset. diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index adaaabf946f2..e50a6b039721 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -22,6 +22,7 @@ import java.net._ import java.nio.channels._ import java.nio.channels.{Selector => NSelector} import java.util +import java.util.Optional import java.util.concurrent._ import java.util.concurrent.atomic._ import java.util.function.Supplier @@ -209,11 +210,9 @@ class SocketServer(val config: KafkaConfig, dataPlaneAcceptors.asScala.filterKeys(_ != interBrokerListener).values orderedAcceptors.foreach { acceptor => val endpoint = acceptor.endPoint - debug(s"Wait for authorizer to complete start up on listener ${endpoint.listener}") - authorizerFutures.get(endpoint).foreach { future => - future.join() - } - debug(s"Start processors on listener ${endpoint.listener}") + debug(s"Wait for authorizer to complete start up on listener ${endpoint.listenerName}") + waitForAuthorizerFuture(acceptor, authorizerFutures) + debug(s"Start processors on listener ${endpoint.listenerName}") acceptor.startProcessors(DataPlaneThreadPrefix) } info(s"Started data-plane processors for ${dataPlaneAcceptors.size} acceptors") @@ -226,7 +225,7 @@ class SocketServer(val config: KafkaConfig, */ def startControlPlaneProcessor(authorizerFutures: Map[Endpoint, CompletableFuture[Void]] = Map.empty): Unit = synchronized { controlPlaneAcceptorOpt.foreach { controlPlaneAcceptor => - authorizerFutures.get(controlPlaneAcceptor.endPoint).foreach(_.get) + waitForAuthorizerFuture(controlPlaneAcceptor, authorizerFutures) controlPlaneAcceptor.startProcessors(ControlPlaneThreadPrefix) info(s"Started control-plane processor for the control-plane acceptor") } @@ -380,6 +379,15 @@ class SocketServer(val config: KafkaConfig, } } + private def waitForAuthorizerFuture(acceptor: Acceptor, + authorizerFutures: Map[Endpoint, CompletableFuture[Void]]): Unit = { + //we can't rely on authorizerFutures.get() due to ephemeral ports. Get the future using listener name + authorizerFutures.foreach { case (endpoint, future) => + if (endpoint.listenerName == Optional.of(acceptor.endPoint.listenerName.value)) + future.join() + } + } + // `protected` for test usage protected[network] def newProcessor(id: Int, requestChannel: RequestChannel, connectionQuotas: ConnectionQuotas, listenerName: ListenerName, securityProtocol: SecurityProtocol, memoryPool: MemoryPool): Processor = { diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index 222fbbcf5b80..56935d9b12e6 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -19,6 +19,7 @@ package kafka.security.auth import java.util import kafka.network.RequestChannel.Session +import kafka.security.auth.SimpleAclAuthorizer.BaseAuthorizer import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils} import kafka.utils._ import kafka.zk.ZkVersion @@ -50,6 +51,22 @@ object SimpleAclAuthorizer { def exists: Boolean = zkVersion != ZkVersion.UnknownVersion } val NoAcls = VersionedAcls(Set.empty, ZkVersion.UnknownVersion) + + private[auth] class BaseAuthorizer extends AclAuthorizer { + override def logAuditMessage(requestContext: AuthorizableRequestContext, action: Action, authorized: Boolean): Unit = { + val principal = requestContext.principal + val host = requestContext.clientAddress.getHostAddress + val operation = Operation.fromJava(action.operation) + val resource = AuthorizerUtils.convertToResource(action.resourcePattern) + def logMessage: String = { + val authResult = if (authorized) "Allowed" else "Denied" + s"Principal = $principal is $authResult Operation = $operation from host = $host on resource = $resource" + } + + if (authorized) authorizerLogger.debug(logMessage) + else authorizerLogger.info(logMessage) + } + } } @deprecated("Use kafka.security.authorizer.AclAuthorizer", "Since 2.4") @@ -156,20 +173,4 @@ class SimpleAclAuthorizer extends Authorizer with Logging { else throw e } - - class BaseAuthorizer extends AclAuthorizer { - override def logAuditMessage(requestContext: AuthorizableRequestContext, action: Action, authorized: Boolean): Unit = { - val principal = requestContext.principal - val host = requestContext.clientAddress.getHostAddress - val operation = Operation.fromJava(action.operation) - val resource = AuthorizerUtils.convertToResource(action.resourcePattern) - def logMessage: String = { - val authResult = if (authorized) "Allowed" else "Denied" - s"Principal = $principal is $authResult Operation = $operation from host = $host on resource = $resource" - } - - if (authorized) authorizerLogger.debug(logMessage) - else authorizerLogger.info(logMessage) - } - } } diff --git a/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala b/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala index 84a641d60dd7..8ba060d81492 100644 --- a/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala @@ -61,6 +61,23 @@ object AclAuthorizer { def exists: Boolean = zkVersion != ZkVersion.UnknownVersion } val NoAcls = VersionedAcls(Set.empty, ZkVersion.UnknownVersion) + + // Orders by resource type, then resource pattern type and finally reverse ordering by name. + private object ResourceOrdering extends Ordering[Resource] { + + def compare(a: Resource, b: Resource): Int = { + val rt = a.resourceType compare b.resourceType + if (rt != 0) + rt + else { + val rnt = a.patternType compareTo b.patternType + if (rnt != 0) + rnt + else + (a.name compare b.name) * -1 + } + } + } } class AclAuthorizer extends Authorizer with Logging { @@ -72,7 +89,7 @@ class AclAuthorizer extends Authorizer with Logging { private var extendedAclSupport: Boolean = _ @volatile - private var aclCache = new scala.collection.immutable.TreeMap[Resource, VersionedAcls]()(ResourceOrdering) + private var aclCache = new scala.collection.immutable.TreeMap[Resource, VersionedAcls]()(AclAuthorizer.ResourceOrdering) private val lock = new ReentrantReadWriteLock() // The maximum number of times we should try to update the resource acls in zookeeper before failing; @@ -479,21 +496,4 @@ class AclAuthorizer extends Authorizer with Logging { } } } - - // Orders by resource type, then resource pattern type and finally reverse ordering by name. - private object ResourceOrdering extends Ordering[Resource] { - - def compare(a: Resource, b: Resource): Int = { - val rt = a.resourceType compare b.resourceType - if (rt != 0) - rt - else { - val rnt = a.patternType compareTo b.patternType - if (rnt != 0) - rnt - else - (a.name compare b.name) * -1 - } - } - } } diff --git a/core/src/main/scala/kafka/security/authorizer/AuthorizerUtils.scala b/core/src/main/scala/kafka/security/authorizer/AuthorizerUtils.scala index e85e1063d0a3..909f3a78bbb7 100644 --- a/core/src/main/scala/kafka/security/authorizer/AuthorizerUtils.scala +++ b/core/src/main/scala/kafka/security/authorizer/AuthorizerUtils.scala @@ -84,7 +84,7 @@ object AuthorizerUtils { new AuthorizableRequestContext { override def clientId(): String = "" override def requestType(): Int = -1 - override def listener(): String = "" + override def listenerName(): String = "" override def clientAddress(): InetAddress = session.clientAddress override def principal(): KafkaPrincipal = session.principal override def securityProtocol(): SecurityProtocol = null diff --git a/core/src/main/scala/kafka/server/AdminManager.scala b/core/src/main/scala/kafka/server/AdminManager.scala index 1c4115e20068..5de1bcd97fba 100644 --- a/core/src/main/scala/kafka/server/AdminManager.scala +++ b/core/src/main/scala/kafka/server/AdminManager.scala @@ -32,6 +32,7 @@ import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigExceptio import org.apache.kafka.common.errors.{ApiException, InvalidConfigurationException, InvalidPartitionsException, InvalidReplicaAssignmentException, InvalidRequestException, ReassignmentInProgressException, TopicExistsException, UnknownTopicOrPartitionException} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic +import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicConfigs, CreatableTopicResult} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors @@ -82,6 +83,7 @@ class AdminManager(val config: KafkaConfig, def createTopics(timeout: Int, validateOnly: Boolean, toCreate: Map[String, CreatableTopic], + includeConfigsAndMetatadata: Map[String, CreatableTopicResult], responseCallback: Map[String, ApiError] => Unit): Unit = { // 1. map over topics creating assignment and calling zookeeper @@ -153,6 +155,27 @@ class AdminManager(val config: KafkaConfig, else adminZkClient.createTopicWithAssignment(topic.name, configs, assignments) } + + // For responses with DescribeConfigs permission, populate metadata and configs + includeConfigsAndMetatadata.get(topic.name).foreach { result => + val logConfig = LogConfig.fromProps(KafkaServer.copyKafkaConfigToLog(config), configs) + val createEntry = createTopicConfigEntry(logConfig, configs, includeSynonyms = false)(_, _) + val topicConfigs = logConfig.values.asScala.map { case (k, v) => + val entry = createEntry(k, v) + val source = ConfigSource.values.indices.map(_.toByte) + .find(i => ConfigSource.forId(i.toByte) == entry.source) + .getOrElse(0.toByte) + new CreatableTopicConfigs() + .setName(k) + .setValue(entry.value) + .setIsSensitive(entry.isSensitive) + .setReadOnly(entry.isReadOnly) + .setConfigSource(source) + }.toList.asJava + result.setConfigs(topicConfigs) + result.setNumPartitions(assignments.size) + result.setReplicationFactor(assignments(0).size.toShort) + } CreatePartitionsMetadata(topic.name, assignments, ApiError.NONE) } catch { // Log client errors at a lower level than unexpected exceptions @@ -165,7 +188,7 @@ class AdminManager(val config: KafkaConfig, case e: Throwable => error(s"Error processing create topic request $topic", e) CreatePartitionsMetadata(topic.name, Map(), ApiError.fromThrowable(e)) - }).toIndexedSeq + }).toBuffer // 2. if timeout <= 0, validateOnly or no topics can proceed return immediately if (timeout <= 0 || validateOnly || !metadata.exists(_.error.is(Errors.NONE))) { @@ -181,7 +204,7 @@ class AdminManager(val config: KafkaConfig, } else { // 3. else pass the assignments and errors to the delayed operation and set the keys val delayedCreate = new DelayedCreatePartitions(timeout, metadata, this, responseCallback) - val delayedCreateKeys = toCreate.values.map(topic => new TopicKey(topic.name)).toIndexedSeq + val delayedCreateKeys = toCreate.values.map(topic => new TopicKey(topic.name)).toBuffer // try to complete the request immediately, otherwise put it into the purgatory topicPurgatory.tryCompleteElseWatch(delayedCreate, delayedCreateKeys) } @@ -322,7 +345,7 @@ class AdminManager(val config: KafkaConfig, val filteredConfigPairs = configs.filter { case (configName, _) => /* Always returns true if configNames is None */ configNames.forall(_.contains(configName)) - }.toIndexedSeq + }.toBuffer val configEntries = filteredConfigPairs.map { case (name, value) => createConfigEntry(name, value) } new DescribeConfigsResponse.Config(ApiError.NONE, configEntries.asJava) diff --git a/core/src/main/scala/kafka/server/DelayedCreatePartitions.scala b/core/src/main/scala/kafka/server/DelayedCreatePartitions.scala index cfa017852d40..8542bc2a2ece 100644 --- a/core/src/main/scala/kafka/server/DelayedCreatePartitions.scala +++ b/core/src/main/scala/kafka/server/DelayedCreatePartitions.scala @@ -83,6 +83,6 @@ class DelayedCreatePartitions(delayMs: Long, private def isMissingLeader(topic: String, partition: Int): Boolean = { val partitionInfo = adminManager.metadataCache.getPartitionInfo(topic, partition) - partitionInfo.isEmpty || partitionInfo.get.basePartitionState.leader == LeaderAndIsr.NoLeader + partitionInfo.forall(_.leader == LeaderAndIsr.NoLeader) } } diff --git a/core/src/main/scala/kafka/server/DelayedElectLeader.scala b/core/src/main/scala/kafka/server/DelayedElectLeader.scala index 95b577a26927..d599f98411b3 100644 --- a/core/src/main/scala/kafka/server/DelayedElectLeader.scala +++ b/core/src/main/scala/kafka/server/DelayedElectLeader.scala @@ -50,10 +50,10 @@ class DelayedElectLeader( override def onComplete(): Unit = { // This could be called to force complete, so I need the full list of partitions, so I can time them all out. updateWaiting() - val timedout = waitingPartitions.map { + val timedOut = waitingPartitions.map { case (tp, _) => tp -> new ApiError(Errors.REQUEST_TIMED_OUT, null) }.toMap - responseCallback(timedout ++ fullResults) + responseCallback(timedOut ++ fullResults) } /** @@ -69,17 +69,14 @@ class DelayedElectLeader( waitingPartitions.isEmpty && forceComplete() } - private def updateWaiting() = { - waitingPartitions.foreach { case (tp, leader) => - val ps = replicaManager.metadataCache.getPartitionInfo(tp.topic, tp.partition) - ps match { - case Some(ps) => - if (leader == ps.basePartitionState.leader) { - waitingPartitions -= tp - fullResults += tp -> ApiError.NONE - } - case None => - } + private def updateWaiting(): Unit = { + val metadataCache = replicaManager.metadataCache + val completedPartitions = waitingPartitions.collect { + case (tp, leader) if metadataCache.getPartitionInfo(tp.topic, tp.partition).exists(_.leader == leader) => tp + } + completedPartitions.foreach { tp => + waitingPartitions -= tp + fullResults += tp -> ApiError.NONE } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index de0397cc783e..109530c1b51d 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -48,32 +48,14 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal} import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic -import org.apache.kafka.common.message.CreateTopicsResponseData +import org.apache.kafka.common.message.{AlterPartitionReassignmentsResponseData, CreateTopicsResponseData, DeleteGroupsResponseData, DeleteTopicsResponseData, DescribeGroupsResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} -import org.apache.kafka.common.message.DeleteGroupsResponseData import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} -import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} -import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData -import org.apache.kafka.common.message.DeleteTopicsResponseData import org.apache.kafka.common.message.DeleteTopicsResponseData.{DeletableTopicResult, DeletableTopicResultCollection} -import org.apache.kafka.common.message.DescribeGroupsResponseData import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult -import org.apache.kafka.common.message.ExpireDelegationTokenResponseData -import org.apache.kafka.common.message.FindCoordinatorResponseData -import org.apache.kafka.common.message.HeartbeatResponseData -import org.apache.kafka.common.message.InitProducerIdResponseData -import org.apache.kafka.common.message.JoinGroupResponseData -import org.apache.kafka.common.message.LeaveGroupResponseData import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse -import org.apache.kafka.common.message.ListGroupsResponseData -import org.apache.kafka.common.message.OffsetCommitRequestData -import org.apache.kafka.common.message.OffsetCommitResponseData -import org.apache.kafka.common.message.RenewDelegationTokenResponseData -import org.apache.kafka.common.message.SaslAuthenticateResponseData -import org.apache.kafka.common.message.SaslHandshakeResponseData -import org.apache.kafka.common.message.SyncGroupResponseData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ListenerName, Send} import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -186,6 +168,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.INCREMENTAL_ALTER_CONFIGS => handleIncrementalAlterConfigsRequest(request) case ApiKeys.ALTER_PARTITION_REASSIGNMENTS => handleAlterPartitionReassignmentsRequest(request) case ApiKeys.LIST_PARTITION_REASSIGNMENTS => handleListPartitionReassignmentsRequest(request) + case ApiKeys.OFFSET_DELETE => handleOffsetDeleteRequest(request) } } catch { case e: FatalExitError => throw e @@ -245,7 +228,7 @@ class KafkaApis(val requestChannel: RequestChannel, // for its previous generation so the broker should skip the stale request. info("Received stop replica request with broker epoch " + s"${stopReplicaRequest.brokerEpoch()} smaller than the current broker epoch ${controller.brokerEpoch}") - sendResponseExemptThrottle(request, new StopReplicaResponse(Errors.STALE_BROKER_EPOCH, Map.empty[TopicPartition, Errors].asJava)) + sendResponseExemptThrottle(request, new StopReplicaResponse(new StopReplicaResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code))) } else { val (result, error) = replicaManager.stopReplicas(stopReplicaRequest) // Clearing out the cache for groups that belong to an offsets topic partition for which this broker was the leader, @@ -259,7 +242,16 @@ class KafkaApis(val requestChannel: RequestChannel, groupCoordinator.handleGroupEmigration(topicPartition.partition) } } - sendResponseExemptThrottle(request, new StopReplicaResponse(error, result.asJava)) + + def toStopReplicaPartition(tp: TopicPartition, error: Errors) = + new StopReplicaResponseData.StopReplicaPartitionError() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setErrorCode(error.code) + + sendResponseExemptThrottle(request, new StopReplicaResponse(new StopReplicaResponseData() + .setErrorCode(error.code) + .setPartitionErrors(result.map { case (tp, error) => toStopReplicaPartition(tp, error) }.toBuffer.asJava))) } CoreUtils.swallow(replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads(), this) @@ -270,20 +262,21 @@ class KafkaApis(val requestChannel: RequestChannel, val updateMetadataRequest = request.body[UpdateMetadataRequest] authorizeClusterOperation(request, CLUSTER_ACTION) - if (isBrokerEpochStale(updateMetadataRequest.brokerEpoch())) { + if (isBrokerEpochStale(updateMetadataRequest.brokerEpoch)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info("Received update metadata request with broker epoch " + - s"${updateMetadataRequest.brokerEpoch()} smaller than the current broker epoch ${controller.brokerEpoch}") - sendResponseExemptThrottle(request, new UpdateMetadataResponse(Errors.STALE_BROKER_EPOCH)) + s"${updateMetadataRequest.brokerEpoch} smaller than the current broker epoch ${controller.brokerEpoch}") + sendResponseExemptThrottle(request, + new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code))) } else { val deletedPartitions = replicaManager.maybeUpdateMetadataCache(correlationId, updateMetadataRequest) if (deletedPartitions.nonEmpty) groupCoordinator.handleDeletedPartitions(deletedPartitions) if (adminManager.hasDelayedTopicOperations) { - updateMetadataRequest.partitionStates.keySet.asScala.map(_.topic).foreach { topic => - adminManager.tryCompleteDelayedTopicOperations(topic) + updateMetadataRequest.partitionStates.asScala.foreach { partitionState => + adminManager.tryCompleteDelayedTopicOperations(partitionState.topicName) } } quotas.clientQuotaCallback.foreach { callback => @@ -294,11 +287,13 @@ class KafkaApis(val requestChannel: RequestChannel, } } if (replicaManager.hasDelayedElectionOperations) { - updateMetadataRequest.partitionStates.asScala.foreach { case (tp, _) => + updateMetadataRequest.partitionStates.asScala.foreach { partitionState => + val tp = new TopicPartition(partitionState.topicName, partitionState.partitionIndex) replicaManager.tryCompleteElection(TopicPartitionOperationKey(tp)) } } - sendResponseExemptThrottle(request, new UpdateMetadataResponse(Errors.NONE)) + sendResponseExemptThrottle(request, new UpdateMetadataResponse( + new UpdateMetadataResponseData().setErrorCode(Errors.NONE.code))) } } @@ -1650,6 +1645,8 @@ class KafkaApis(val requestChannel: RequestChannel, val hasClusterAuthorization = authorize(request, CREATE, CLUSTER, CLUSTER_NAME, logIfDenied = false) val topics = createTopicsRequest.data.topics.asScala.map(_.name) val authorizedTopics = if (hasClusterAuthorization) topics.toSet else filterAuthorized(request, CREATE, TOPIC, topics.toSeq) + val authorizedForDescribeConfigs = filterAuthorized(request, DESCRIBE_CONFIGS, TOPIC, topics.toSeq) + .map(name => name -> results.find(name)).toMap results.asScala.foreach(topic => { if (results.findAll(topic.name()).size() > 1) { @@ -1659,6 +1656,9 @@ class KafkaApis(val requestChannel: RequestChannel, topic.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code()) topic.setErrorMessage("Authorization failed.") } + if (!authorizedForDescribeConfigs.contains(topic.name)) { + topic.setTopicConfigErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) + } }) val toCreate = mutable.Map[String, CreatableTopic]() createTopicsRequest.data.topics.asScala.foreach { topic => @@ -1668,15 +1668,23 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleCreateTopicsResults(errors: Map[String, ApiError]): Unit = { errors.foreach { case (topicName, error) => - results.find(topicName). - setErrorCode(error.error.code). - setErrorMessage(error.message) + val result = results.find(topicName) + result.setErrorCode(error.error.code) + .setErrorMessage(error.message) + // Reset any configs in the response if Create failed + if (error != ApiError.NONE) { + result.setConfigs(List.empty.asJava) + .setNumPartitions(-1) + .setReplicationFactor(-1) + .setTopicConfigErrorCode(0.toShort) + } } sendResponseCallback(results) } adminManager.createTopics(createTopicsRequest.data.timeoutMs, createTopicsRequest.data.validateOnly, toCreate, + authorizedForDescribeConfigs, handleCreateTopicsResults) } } @@ -2660,6 +2668,56 @@ class KafkaApis(val requestChannel: RequestChannel, } } + def handleOffsetDeleteRequest(request: RequestChannel.Request): Unit = { + val offsetDeleteRequest = request.body[OffsetDeleteRequest] + val groupId = offsetDeleteRequest.data.groupId + + if (authorize(request, DELETE, GROUP, groupId)) { + val topicPartitions = offsetDeleteRequest.data.topics.asScala.flatMap { topic => + topic.partitions.asScala.map { partition => + new TopicPartition(topic.name, partition.partitionIndex) + } + }.toSeq + + val authorizedTopics = filterAuthorized(request, READ, TOPIC, topicPartitions.map(_.topic)) + val (authorizedTopicPartitions, unauthorizedTopicPartitions) = topicPartitions.partition { topicPartition => + authorizedTopics.contains(topicPartition.topic) + } + + val unauthorizedTopicPartitionsErrors = unauthorizedTopicPartitions.map(_ -> Errors.TOPIC_AUTHORIZATION_FAILED) + val (groupError, authorizedTopicPartitionsErrors) = groupCoordinator.handleDeleteOffsets(groupId, authorizedTopicPartitions) + val topicPartitionsErrors = unauthorizedTopicPartitionsErrors ++ authorizedTopicPartitionsErrors + + sendResponseMaybeThrottle(request, requestThrottleMs => { + if (groupError != Errors.NONE) + offsetDeleteRequest.getErrorResponse(requestThrottleMs, groupError) + else { + val topics = new OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection + topicPartitionsErrors.groupBy(_._1.topic).map { case (topic, topicPartitions) => + val partitions = new OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection + topicPartitions.map { case (topicPartition, error) => + partitions.add( + new OffsetDeleteResponseData.OffsetDeleteResponsePartition() + .setPartitionIndex(topicPartition.partition) + .setErrorCode(error.code) + ) + topics.add(new OffsetDeleteResponseData.OffsetDeleteResponseTopic() + .setName(topic) + .setPartitions(partitions)) + } + } + + new OffsetDeleteResponse(new OffsetDeleteResponseData() + .setTopics(topics) + .setThrottleTimeMs(requestThrottleMs)) + } + }) + } else { + sendResponseMaybeThrottle(request, requestThrottleMs => + offsetDeleteRequest.getErrorResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED)) + } + } + private def authorize(request: RequestChannel.Request, operation: AclOperation, resourceType: ResourceType, diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 4d6a65d79a13..17ebd278f19a 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -258,7 +258,7 @@ object KafkaConfig { private val LogConfigPrefix = "log." def main(args: Array[String]): Unit = { - System.out.println(configDef.toHtmlTable(DynamicBrokerConfig.dynamicConfigUpdateModes)) + System.out.println(configDef.toHtml(DynamicBrokerConfig.dynamicConfigUpdateModes)) } /** ********* Zookeeper Configuration ***********/ @@ -508,7 +508,7 @@ object KafkaConfig { "To avoid conflicts between zookeeper generated broker id's and user configured broker id's, generated broker ids " + "start from " + MaxReservedBrokerIdProp + " + 1." val MessageMaxBytesDoc = TopicConfig.MAX_MESSAGE_BYTES_DOC + - s"

    This can be set per topic with the topic level ${TopicConfig.MAX_MESSAGE_BYTES_CONFIG} config.

    " + s"This can be set per topic with the topic level ${TopicConfig.MAX_MESSAGE_BYTES_CONFIG} config." val NumNetworkThreadsDoc = "The number of threads that the server uses for receiving requests from the network and sending responses to the network" val NumIoThreadsDoc = "The number of threads that the server uses for processing requests, which may include disk I/O" val NumReplicaAlterLogDirsThreadsDoc = "The number of threads that can move replicas between log directories, which may include disk I/O" @@ -731,7 +731,9 @@ object KafkaConfig { "or this timeout is reached. This is similar to the producer request timeout." val OffsetCommitRequiredAcksDoc = "The required acks before the commit can be accepted. In general, the default (-1) should not be overridden" /** ********* Transaction management configuration ***********/ - val TransactionalIdExpirationMsDoc = "The maximum amount of time in ms that the transaction coordinator will wait before proactively expire a producer's transactional id without receiving any transaction status updates from it." + val TransactionalIdExpirationMsDoc = "The time in ms that the transaction coordinator will wait without receiving any transaction status updates " + + "for the current transaction before expiring its transactional id. This setting also influences producer id expiration - producer ids are expired " + + "once this time has elapsed after the last write with the given producer id. Note that producer ids may expire sooner if the last write from the producer id is deleted due to the topic's retention settings." val TransactionsMaxTimeoutMsDoc = "The maximum allowed timeout for transactions. " + "If a client’s requested transaction time exceed this, then the broker will return an error in InitProducerIdRequest. This prevents a client from too large of a timeout, which can stall consumers reading from topics included in the transaction." val TransactionsTopicMinISRDoc = "Overridden " + MinInSyncReplicasProp + " config for the transaction topic." @@ -741,7 +743,7 @@ object KafkaConfig { val TransactionsTopicPartitionsDoc = "The number of partitions for the transaction topic (should not change after deployment)." val TransactionsTopicSegmentBytesDoc = "The transaction topic segment bytes should be kept relatively small in order to facilitate faster log compaction and cache loads" val TransactionsAbortTimedOutTransactionsIntervalMsDoc = "The interval at which to rollback transactions that have timed out" - val TransactionsRemoveExpiredTransactionsIntervalMsDoc = "The interval at which to remove transactions that have expired due to transactional.id.expiration.ms passing" + val TransactionsRemoveExpiredTransactionsIntervalMsDoc = "The interval at which to remove transactions that have expired due to transactional.id.expiration.ms passing" /** ********* Fetch Session Configuration **************/ val MaxIncrementalFetchSessionCacheSlotsDoc = "The maximum number of incremental fetch sessions that we will maintain." @@ -761,11 +763,6 @@ object KafkaConfig { "which is used to determine quota limits applied to client requests. By default, , or " + "quotas stored in ZooKeeper are applied. For any given request, the most specific quota that matches the user principal " + "of the session and the client-id of the request is applied." - /** ********* Transaction Configuration ***********/ - val TransactionIdExpirationMsDoc = "The maximum time of inactivity before a transactional id is expired by the " + - "transaction coordinator. Note that this also influences producer id expiration: Producer ids are guaranteed to expire " + - "after expiration of this timeout from the last write by the producer id (they may expire sooner if the last write " + - "from the producer id is deleted due to the topic's retention settings)." val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + @@ -1361,9 +1358,6 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO val numAlterLogDirsReplicationQuotaSamples = getInt(KafkaConfig.NumAlterLogDirsReplicationQuotaSamplesProp) val alterLogDirsReplicationQuotaWindowSizeSeconds = getInt(KafkaConfig.AlterLogDirsReplicationQuotaWindowSizeSecondsProp) - /** ********* Transaction Configuration **************/ - val transactionIdExpirationMs = getInt(KafkaConfig.TransactionalIdExpirationMsProp) - /** ********* Fetch Session Configuration **************/ val maxIncrementalFetchSessionCacheSlots = getInt(KafkaConfig.MaxIncrementalFetchSessionCacheSlots) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 0028e4a58198..57cb0b6d20ed 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -299,7 +299,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP case Some(authZ) => authZ.start(brokerInfo.broker.toServerInfo(clusterId, config)).asScala.mapValues(_.toCompletableFuture).toMap case None => - brokerInfo.broker.endPoints.map{ ep => (ep.asInstanceOf[Endpoint], CompletableFuture.completedFuture[Void](null)) }.toMap + brokerInfo.broker.endPoints.map { ep => ep.toJava -> CompletableFuture.completedFuture[Void](null) }.toMap } val fetchManager = new FetchManager(Time.SYSTEM, diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index e0203478e077..1ad0e41352e5 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -28,10 +28,12 @@ import kafka.controller.StateChangeLogger import kafka.utils.CoreUtils._ import kafka.utils.Logging import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest} +import org.apache.kafka.common.security.auth.SecurityProtocol /** @@ -75,14 +77,14 @@ class MetadataCache(brokerId: Int) extends Logging { snapshot.partitionStates.get(topic).map { partitions => partitions.map { case (partitionId, partitionState) => val topicPartition = new TopicPartition(topic, partitionId.toInt) - val leaderBrokerId = partitionState.basePartitionState.leader - val leaderEpoch = partitionState.basePartitionState.leaderEpoch + val leaderBrokerId = partitionState.leader + val leaderEpoch = partitionState.leaderEpoch val maybeLeader = getAliveEndpoint(snapshot, leaderBrokerId, listenerName) - val replicas = partitionState.basePartitionState.replicas.asScala + val replicas = partitionState.replicas.asScala val replicaInfo = getEndpoints(snapshot, replicas, listenerName, errorUnavailableEndpoints) val offlineReplicaInfo = getEndpoints(snapshot, partitionState.offlineReplicas.asScala, listenerName, errorUnavailableEndpoints) - val isr = partitionState.basePartitionState.isr.asScala + val isr = partitionState.isr.asScala val isrInfo = getEndpoints(snapshot, isr, listenerName, errorUnavailableEndpoints) maybeLeader match { case None => @@ -148,7 +150,7 @@ class MetadataCache(brokerId: Int) extends Logging { snapshot.partitionStates.keySet } - private def getAllPartitions(snapshot: MetadataSnapshot): Map[TopicPartition, UpdateMetadataRequest.PartitionState] = { + private def getAllPartitions(snapshot: MetadataSnapshot): Map[TopicPartition, UpdateMetadataPartitionState] = { snapshot.partitionStates.flatMap { case (topic, partitionStates) => partitionStates.map { case (partition, state ) => (new TopicPartition(topic, partition.toInt), state) } }.toMap @@ -166,15 +168,15 @@ class MetadataCache(brokerId: Int) extends Logging { metadataSnapshot.aliveBrokers.values.toBuffer } - private def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataRequest.PartitionState]], + private def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], topic: String, partitionId: Int, - stateInfo: UpdateMetadataRequest.PartitionState): Unit = { + stateInfo: UpdateMetadataPartitionState): Unit = { val infos = partitionStates.getOrElseUpdate(topic, mutable.LongMap()) infos(partitionId) = stateInfo } - def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataRequest.PartitionState] = { + def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataPartitionState] = { metadataSnapshot.partitionStates.get(topic).flatMap(_.get(partitionId)) } @@ -184,7 +186,7 @@ class MetadataCache(brokerId: Int) extends Logging { def getPartitionLeaderEndpoint(topic: String, partitionId: Int, listenerName: ListenerName): Option[Node] = { val snapshot = metadataSnapshot snapshot.partitionStates.get(topic).flatMap(_.get(partitionId)) map { partitionInfo => - val leaderId = partitionInfo.basePartitionState.leader + val leaderId = partitionInfo.leader snapshot.aliveNodes.get(leaderId) match { case Some(nodeMap) => @@ -197,8 +199,8 @@ class MetadataCache(brokerId: Int) extends Logging { def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node] = { val snapshot = metadataSnapshot - snapshot.partitionStates.get(tp.topic()).flatMap(_.get(tp.partition())).map { partitionInfo => - val replicaIds = partitionInfo.basePartitionState.replicas + snapshot.partitionStates.get(tp.topic).flatMap(_.get(tp.partition)).map { partitionInfo => + val replicaIds = partitionInfo.replicas replicaIds.asScala .map(replicaId => replicaId.intValue() -> { snapshot.aliveBrokers.get(replicaId.longValue()) match { @@ -220,17 +222,17 @@ class MetadataCache(brokerId: Int) extends Logging { val nodes = snapshot.aliveNodes.map { case (id, nodes) => (id, nodes.get(listenerName).orNull) } def node(id: Integer): Node = nodes.get(id.toLong).orNull val partitions = getAllPartitions(snapshot) - .filter { case (_, state) => state.basePartitionState.leader != LeaderAndIsr.LeaderDuringDelete } + .filter { case (_, state) => state.leader != LeaderAndIsr.LeaderDuringDelete } .map { case (tp, state) => - new PartitionInfo(tp.topic, tp.partition, node(state.basePartitionState.leader), - state.basePartitionState.replicas.asScala.map(node).toArray, - state.basePartitionState.isr.asScala.map(node).toArray, + new PartitionInfo(tp.topic, tp.partition, node(state.leader), + state.replicas.asScala.map(node).toArray, + state.isr.asScala.map(node).toArray, state.offlineReplicas.asScala.map(node).toArray) } val unauthorizedTopics = Collections.emptySet[String] val internalTopics = getAllTopics(snapshot).filter(Topic.isInternal).asJava - new Cluster(clusterId, nodes.values.filter(_ != null).toList.asJava, - partitions.toList.asJava, + new Cluster(clusterId, nodes.values.filter(_ != null).toBuffer.asJava, + partitions.toBuffer.asJava, unauthorizedTopics, internalTopics, snapshot.controllerId.map(id => node(id)).orNull) } @@ -252,9 +254,10 @@ class MetadataCache(brokerId: Int) extends Logging { // move to `AnyRefMap`, which has comparable performance. val nodes = new java.util.HashMap[ListenerName, Node] val endPoints = new mutable.ArrayBuffer[EndPoint] - broker.endPoints.asScala.foreach { ep => - endPoints += EndPoint(ep.host, ep.port, ep.listenerName, ep.securityProtocol) - nodes.put(ep.listenerName, new Node(broker.id, ep.host, ep.port)) + broker.endpoints.asScala.foreach { ep => + val listenerName = new ListenerName(ep.listener) + endPoints += new EndPoint(ep.host, ep.port, listenerName, SecurityProtocol.forId(ep.securityProtocol)) + nodes.put(listenerName, new Node(broker.id, ep.host, ep.port)) } aliveBrokers(broker.id) = Broker(broker.id, endPoints, Option(broker.rack)) aliveNodes(broker.id) = nodes.asScala @@ -266,20 +269,21 @@ class MetadataCache(brokerId: Int) extends Logging { } val deletedPartitions = new mutable.ArrayBuffer[TopicPartition] - if (updateMetadataRequest.partitionStates().isEmpty) { + if (!updateMetadataRequest.partitionStates.iterator.hasNext) { metadataSnapshot = MetadataSnapshot(metadataSnapshot.partitionStates, controllerId, aliveBrokers, aliveNodes) } else { //since kafka may do partial metadata updates, we start by copying the previous state - val partitionStates = new mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataRequest.PartitionState]](metadataSnapshot.partitionStates.size) + val partitionStates = new mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]](metadataSnapshot.partitionStates.size) metadataSnapshot.partitionStates.foreach { case (topic, oldPartitionStates) => - val copy = new mutable.LongMap[UpdateMetadataRequest.PartitionState](oldPartitionStates.size) + val copy = new mutable.LongMap[UpdateMetadataPartitionState](oldPartitionStates.size) copy ++= oldPartitionStates partitionStates += (topic -> copy) } - updateMetadataRequest.partitionStates.asScala.foreach { case (tp, info) => + updateMetadataRequest.partitionStates.asScala.foreach { info => val controllerId = updateMetadataRequest.controllerId val controllerEpoch = updateMetadataRequest.controllerEpoch - if (info.basePartitionState.leader == LeaderAndIsr.LeaderDuringDelete) { + val tp = new TopicPartition(info.topicName, info.partitionIndex) + if (info.leader == LeaderAndIsr.LeaderDuringDelete) { removePartitionInfo(partitionStates, tp.topic, tp.partition) stateChangeLogger.trace(s"Deleted partition $tp from metadata cache in response to UpdateMetadata " + s"request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") @@ -302,7 +306,8 @@ class MetadataCache(brokerId: Int) extends Logging { def contains(tp: TopicPartition): Boolean = getPartitionInfo(tp.topic, tp.partition).isDefined - private def removePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataRequest.PartitionState]], topic: String, partitionId: Int): Boolean = { + private def removePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], + topic: String, partitionId: Int): Boolean = { partitionStates.get(topic).exists { infos => infos.remove(partitionId) if (infos.isEmpty) partitionStates.remove(topic) @@ -310,7 +315,7 @@ class MetadataCache(brokerId: Int) extends Logging { } } - case class MetadataSnapshot(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataRequest.PartitionState]], + case class MetadataSnapshot(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], controllerId: Option[Int], aliveBrokers: mutable.LongMap[Broker], aliveNodes: mutable.LongMap[collection.Map[ListenerName, Node]]) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 12cb507d3500..8a74aa7f8908 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -37,6 +37,9 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.Node import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState +import org.apache.kafka.common.message.LeaderAndIsrResponseData +import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors @@ -1085,8 +1088,8 @@ class ReplicaManager(val config: KafkaConfig, .map(replica => new DefaultReplicaView( replicaEndpoints.getOrElse(replica.brokerId, Node.noNode()), replica.logEndOffset, - currentTimeMs - replica.lastCaughtUpTimeMs - )) + currentTimeMs - replica.lastCaughtUpTimeMs)) + .toSet if (partition.leaderReplicaIdOpt.isDefined) { val leaderReplica: ReplicaView = partition.leaderReplicaIdOpt @@ -1144,10 +1147,12 @@ class ReplicaManager(val config: KafkaConfig, def becomeLeaderOrFollower(correlationId: Int, leaderAndIsrRequest: LeaderAndIsrRequest, onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): LeaderAndIsrResponse = { - leaderAndIsrRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) => - stateChangeLogger.trace(s"Received LeaderAndIsr request $stateInfo " + - s"correlation id $correlationId from controller ${leaderAndIsrRequest.controllerId} " + - s"epoch ${leaderAndIsrRequest.controllerEpoch} for partition $topicPartition") + if (stateChangeLogger.isTraceEnabled) { + leaderAndIsrRequest.partitionStates.asScala.foreach { partitionState => + stateChangeLogger.trace(s"Received LeaderAndIsr request $partitionState " + + s"correlation id $correlationId from controller ${leaderAndIsrRequest.controllerId} " + + s"epoch ${leaderAndIsrRequest.controllerEpoch}") + } } replicaStateChangeLock synchronized { if (leaderAndIsrRequest.controllerEpoch < controllerEpoch) { @@ -1161,10 +1166,11 @@ class ReplicaManager(val config: KafkaConfig, controllerEpoch = leaderAndIsrRequest.controllerEpoch // First check partition's leader epoch - val partitionState = new mutable.HashMap[Partition, LeaderAndIsrRequest.PartitionState]() + val partitionStates = new mutable.HashMap[Partition, LeaderAndIsrPartitionState]() val newPartitions = new mutable.HashSet[Partition] - leaderAndIsrRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) => + leaderAndIsrRequest.partitionStates.asScala.foreach { partitionState => + val topicPartition = new TopicPartition(partitionState.topicName, partitionState.partitionIndex) val partitionOpt = getPartition(topicPartition) match { case HostedPartition.Offline => stateChangeLogger.warn(s"Ignoring LeaderAndIsr request from " + @@ -1185,16 +1191,16 @@ class ReplicaManager(val config: KafkaConfig, partitionOpt.foreach { partition => val currentLeaderEpoch = partition.getLeaderEpoch - val requestLeaderEpoch = stateInfo.basePartitionState.leaderEpoch + val requestLeaderEpoch = partitionState.leaderEpoch if (requestLeaderEpoch > currentLeaderEpoch) { // If the leader epoch is valid record the epoch of the controller that made the leadership decision. // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path - if (stateInfo.basePartitionState.replicas.contains(localBrokerId)) - partitionState.put(partition, stateInfo) + if (partitionState.replicas.contains(localBrokerId)) + partitionStates.put(partition, partitionState) else { stateChangeLogger.warn(s"Ignoring LeaderAndIsr request from controller $controllerId with " + s"correlation id $correlationId epoch $controllerEpoch for partition $topicPartition as itself is not " + - s"in assigned replica list ${stateInfo.basePartitionState.replicas.asScala.mkString(",")}") + s"in assigned replica list ${partitionState.replicas.asScala.mkString(",")}") responseMap.put(topicPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION) } } else if (requestLeaderEpoch < currentLeaderEpoch) { @@ -1214,10 +1220,10 @@ class ReplicaManager(val config: KafkaConfig, } } - val partitionsTobeLeader = partitionState.filter { case (_, stateInfo) => - stateInfo.basePartitionState.leader == localBrokerId + val partitionsTobeLeader = partitionStates.filter { case (_, partitionState) => + partitionState.leader == localBrokerId } - val partitionsToBeFollower = partitionState -- partitionsTobeLeader.keys + val partitionsToBeFollower = partitionStates -- partitionsTobeLeader.keys val highWatermarkCheckpoints = new LazyOffsetCheckpoints(this.highWatermarkCheckpoints) val partitionsBecomeLeader = if (partitionsTobeLeader.nonEmpty) @@ -1244,7 +1250,8 @@ class ReplicaManager(val config: KafkaConfig, // remove metrics for brokers which are not followers of a topic leaderTopicSet.diff(followerTopicSet).foreach(brokerTopicStats.removeOldFollowerMetrics) - leaderAndIsrRequest.partitionStates.asScala.keys.foreach { topicPartition => + leaderAndIsrRequest.partitionStates.asScala.foreach { partitionState => + val topicPartition = new TopicPartition(partitionState.topicName, partitionState.partitionIndex) /* * If there is offline log directory, a Partition object may have been created by getOrCreatePartition() * before getOrCreateReplica() failed to create local replica due to KafkaStorageException. @@ -1284,7 +1291,15 @@ class ReplicaManager(val config: KafkaConfig, replicaFetcherManager.shutdownIdleFetcherThreads() replicaAlterLogDirsManager.shutdownIdleFetcherThreads() onLeadershipChange(partitionsBecomeLeader, partitionsBecomeFollower) - new LeaderAndIsrResponse(Errors.NONE, responseMap.asJava) + val responsePartitions = responseMap.iterator.map { case (tp, error) => + new LeaderAndIsrPartitionError() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setErrorCode(error.code) + }.toBuffer + new LeaderAndIsrResponse(new LeaderAndIsrResponseData() + .setErrorCode(Errors.NONE.code) + .setPartitionErrors(responsePartitions.asJava)) } } } @@ -1304,42 +1319,42 @@ class ReplicaManager(val config: KafkaConfig, */ private def makeLeaders(controllerId: Int, controllerEpoch: Int, - partitionState: Map[Partition, LeaderAndIsrRequest.PartitionState], + partitionStates: Map[Partition, LeaderAndIsrPartitionState], correlationId: Int, responseMap: mutable.Map[TopicPartition, Errors], highWatermarkCheckpoints: OffsetCheckpoints): Set[Partition] = { - partitionState.keys.foreach { partition => + partitionStates.keys.foreach { partition => stateChangeLogger.trace(s"Handling LeaderAndIsr request correlationId $correlationId from " + s"controller $controllerId epoch $controllerEpoch starting the become-leader transition for " + s"partition ${partition.topicPartition}") } - for (partition <- partitionState.keys) + for (partition <- partitionStates.keys) responseMap.put(partition.topicPartition, Errors.NONE) val partitionsToMakeLeaders = mutable.Set[Partition]() try { // First stop fetchers for all the partitions - replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(_.topicPartition)) + replicaFetcherManager.removeFetcherForPartitions(partitionStates.keySet.map(_.topicPartition)) // Update the partition information to be the leader - partitionState.foreach{ case (partition, partitionStateInfo) => + partitionStates.foreach { case (partition, partitionState) => try { - if (partition.makeLeader(controllerId, partitionStateInfo, correlationId, highWatermarkCheckpoints)) { + if (partition.makeLeader(controllerId, partitionState, correlationId, highWatermarkCheckpoints)) { partitionsToMakeLeaders += partition stateChangeLogger.trace(s"Stopped fetchers as part of become-leader request from " + s"controller $controllerId epoch $controllerEpoch with correlation id $correlationId for partition ${partition.topicPartition} " + - s"(last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch})") + s"(last update controller epoch ${partitionState.controllerEpoch})") } else stateChangeLogger.info(s"Skipped the become-leader state change after marking its " + s"partition as leader with correlation id $correlationId from controller $controllerId epoch $controllerEpoch for " + - s"partition ${partition.topicPartition} (last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) " + + s"partition ${partition.topicPartition} (last update controller epoch ${partitionState.controllerEpoch}) " + s"since it is already the leader for the partition.") } catch { case e: KafkaStorageException => stateChangeLogger.error(s"Skipped the become-leader state change with " + s"correlation id $correlationId from controller $controllerId epoch $controllerEpoch for partition ${partition.topicPartition} " + - s"(last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) since " + + s"(last update controller epoch ${partitionState.controllerEpoch}) since " + s"the replica for the partition is offline due to disk error $e") val dirOpt = getLogDir(partition.topicPartition) error(s"Error while making broker the leader for partition $partition in dir $dirOpt", e) @@ -1349,7 +1364,7 @@ class ReplicaManager(val config: KafkaConfig, } catch { case e: Throwable => - partitionState.keys.foreach { partition => + partitionStates.keys.foreach { partition => stateChangeLogger.error(s"Error while processing LeaderAndIsr request correlationId $correlationId received " + s"from controller $controllerId epoch $controllerEpoch for partition ${partition.topicPartition}", e) } @@ -1357,7 +1372,7 @@ class ReplicaManager(val config: KafkaConfig, throw e } - partitionState.keys.foreach { partition => + partitionStates.keys.foreach { partition => stateChangeLogger.trace(s"Completed LeaderAndIsr request correlationId $correlationId from controller $controllerId " + s"epoch $controllerEpoch for the become-leader transition for partition ${partition.topicPartition}") } @@ -1385,14 +1400,14 @@ class ReplicaManager(val config: KafkaConfig, */ private def makeFollowers(controllerId: Int, controllerEpoch: Int, - partitionStates: Map[Partition, LeaderAndIsrRequest.PartitionState], + partitionStates: Map[Partition, LeaderAndIsrPartitionState], correlationId: Int, responseMap: mutable.Map[TopicPartition, Errors], highWatermarkCheckpoints: OffsetCheckpoints) : Set[Partition] = { partitionStates.foreach { case (partition, partitionState) => stateChangeLogger.trace(s"Handling LeaderAndIsr request correlationId $correlationId from controller $controllerId " + s"epoch $controllerEpoch starting the become-follower transition for partition ${partition.topicPartition} with leader " + - s"${partitionState.basePartitionState.leader}") + s"${partitionState.leader}") } for (partition <- partitionStates.keys) @@ -1401,37 +1416,37 @@ class ReplicaManager(val config: KafkaConfig, val partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set() try { // TODO: Delete leaders from LeaderAndIsrRequest - partitionStates.foreach { case (partition, partitionStateInfo) => - val newLeaderBrokerId = partitionStateInfo.basePartitionState.leader + partitionStates.foreach { case (partition, partitionState) => + val newLeaderBrokerId = partitionState.leader try { metadataCache.getAliveBrokers.find(_.id == newLeaderBrokerId) match { // Only change partition state when the leader is available case Some(_) => - if (partition.makeFollower(controllerId, partitionStateInfo, correlationId, highWatermarkCheckpoints)) + if (partition.makeFollower(controllerId, partitionState, correlationId, highWatermarkCheckpoints)) partitionsToMakeFollower += partition else stateChangeLogger.info(s"Skipped the become-follower state change after marking its partition as " + s"follower with correlation id $correlationId from controller $controllerId epoch $controllerEpoch " + s"for partition ${partition.topicPartition} (last update " + - s"controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) " + + s"controller epoch ${partitionState.controllerEpoch}) " + s"since the new leader $newLeaderBrokerId is the same as the old leader") case None => // The leader broker should always be present in the metadata cache. // If not, we should record the error message and abort the transition process for this partition stateChangeLogger.error(s"Received LeaderAndIsrRequest with correlation id $correlationId from " + s"controller $controllerId epoch $controllerEpoch for partition ${partition.topicPartition} " + - s"(last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) " + + s"(last update controller epoch ${partitionState.controllerEpoch}) " + s"but cannot become follower since the new leader $newLeaderBrokerId is unavailable.") // Create the local replica even if the leader is unavailable. This is required to ensure that we include // the partition's high watermark in the checkpoint file (see KAFKA-1647) - partition.createLogIfNotExists(localBrokerId, isNew = partitionStateInfo.isNew, isFutureReplica = false, + partition.createLogIfNotExists(localBrokerId, isNew = partitionState.isNew, isFutureReplica = false, highWatermarkCheckpoints) } } catch { case e: KafkaStorageException => stateChangeLogger.error(s"Skipped the become-follower state change with correlation id $correlationId from " + s"controller $controllerId epoch $controllerEpoch for partition ${partition.topicPartition} " + - s"(last update controller epoch ${partitionStateInfo.basePartitionState.controllerEpoch}) with leader " + + s"(last update controller epoch ${partitionState.controllerEpoch}) with leader " + s"$newLeaderBrokerId since the replica for the partition is offline due to disk error $e") val dirOpt = getLogDir(partition.topicPartition) error(s"Error while making broker the follower for partition $partition with leader " + @@ -1444,7 +1459,7 @@ class ReplicaManager(val config: KafkaConfig, partitionsToMakeFollower.foreach { partition => stateChangeLogger.trace(s"Stopped fetchers as part of become-follower request from controller $controllerId " + s"epoch $controllerEpoch with correlation id $correlationId for partition ${partition.topicPartition} with leader " + - s"${partitionStates(partition).basePartitionState.leader}") + s"${partitionStates(partition).leader}") } partitionsToMakeFollower.foreach { partition => @@ -1456,14 +1471,14 @@ class ReplicaManager(val config: KafkaConfig, partitionsToMakeFollower.foreach { partition => stateChangeLogger.trace(s"Truncated logs and checkpointed recovery boundaries for partition " + s"${partition.topicPartition} as part of become-follower request with correlation id $correlationId from " + - s"controller $controllerId epoch $controllerEpoch with leader ${partitionStates(partition).basePartitionState.leader}") + s"controller $controllerId epoch $controllerEpoch with leader ${partitionStates(partition).leader}") } if (isShuttingDown.get()) { partitionsToMakeFollower.foreach { partition => stateChangeLogger.trace(s"Skipped the adding-fetcher step of the become-follower state " + s"change with correlation id $correlationId from controller $controllerId epoch $controllerEpoch for " + - s"partition ${partition.topicPartition} with leader ${partitionStates(partition).basePartitionState.leader} " + + s"partition ${partition.topicPartition} with leader ${partitionStates(partition).leader} " + "since it is shutting down") } } else { @@ -1493,7 +1508,7 @@ class ReplicaManager(val config: KafkaConfig, partitionStates.keys.foreach { partition => stateChangeLogger.trace(s"Completed LeaderAndIsr request correlationId $correlationId from controller $controllerId " + s"epoch $controllerEpoch for the become-follower transition for partition ${partition.topicPartition} with leader " + - s"${partitionStates(partition).basePartitionState.leader}") + s"${partitionStates(partition).leader}") } partitionsToMakeFollower @@ -1711,7 +1726,7 @@ class ReplicaManager(val config: KafkaConfig, if (expectedLeaders.nonEmpty) { val watchKeys = expectedLeaders.iterator.map { case (tp, _) => TopicPartitionOperationKey(tp) - }.toIndexedSeq + }.toBuffer delayedElectLeaderPurgatory.tryCompleteElseWatch( new DelayedElectLeader( diff --git a/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala b/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala index e6dd0646b661..9d14ae6f2bd0 100644 --- a/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala +++ b/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala @@ -84,7 +84,7 @@ class LeaderEpochFileCache(topicPartition: TopicPartition, debug(s"Appended new epoch entry $entryToAppend. Cache now contains ${epochs.size} entries.") } else if (removedEpochs.size > 1 || removedEpochs.head.startOffset != entryToAppend.startOffset) { // Only log a warning if there were non-trivial removals. If the start offset of the new entry - // matches the start offfset of the removed epoch, then no data has been written and the truncation + // matches the start offset of the removed epoch, then no data has been written and the truncation // is expected. warn(s"New epoch entry $entryToAppend caused truncation of conflicting entries $removedEpochs. " + s"Cache now contains ${epochs.size} entries.") @@ -123,7 +123,7 @@ class LeaderEpochFileCache(topicPartition: TopicPartition, * Offset if the latest epoch was requested. * * During the upgrade phase, where there are existing messages may not have a leader epoch, - * if requestedEpoch is < the first epoch cached, UNSUPPORTED_EPOCH_OFFSET will be returned + * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned * so that the follower falls back to High Water Mark. * * @param requestedEpoch requested leader epoch diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 73f83fe71f0b..864dcaba6395 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -1656,7 +1656,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo private def getTopicConfigs(topics: Set[String]): Seq[GetDataResponse] = { val getDataRequests: Seq[GetDataRequest] = topics.iterator.map { topic => GetDataRequest(ConfigEntityZNode.path(ConfigType.Topic, topic), ctx = Some(topic)) - }.toIndexedSeq + }.toBuffer retryRequestsUntilConnected(getDataRequests) } diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 43769ad7deb6..75d169d0e2f5 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -281,7 +281,7 @@ object TopicZNode { new TopicPartition(topic, partition.toInt) -> PartitionReplicaAssignment( replicas.to[Seq[Int]], getReplicas(addingReplicasJsonOpt, partition), - getReplicas(addingReplicasJsonOpt, partition) + getReplicas(removingReplicasJsonOpt, partition) ) } } diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index 4853424ed35c..fcd1402126ce 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -334,7 +334,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest { @volatile var thrownException: Option[Throwable] = None @volatile var receivedMessages = 0 - @volatile private var partitionAssignment: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]() + private val partitionAssignment = mutable.Set[TopicPartition]() @volatile private var subscriptionChanged = false private var topicsSubscription = topicsToSubscribe @@ -424,7 +424,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest { } // make sure that sum of all partitions to all consumers equals total number of partitions - val totalPartitionsInAssignments = (0 /: assignments) (_ + _.size) + val totalPartitionsInAssignments = assignments.foldLeft(0)(_ + _.size) if (totalPartitionsInAssignments != partitions.size) { // either same partitions got assigned to more than one consumer or some // partitions were not assigned @@ -434,7 +434,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest { // The above checks could miss the case where one or more partitions were assigned to more // than one consumer and the same number of partitions were missing from assignments. // Make sure that all unique assignments are the same as 'partitions' - val uniqueAssignedPartitions = (Set[TopicPartition]() /: assignments) (_ ++ _) + val uniqueAssignedPartitions = assignments.foldLeft(Set.empty[TopicPartition])(_ ++ _) uniqueAssignedPartitions == partitions } diff --git a/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala index 27c435be3be2..2b718584649a 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala @@ -33,6 +33,7 @@ import kafka.utils.TestUtils._ import kafka.utils.{Log4jController, Logging, TestUtils} import kafka.zk.KafkaZkClient import org.apache.kafka.clients.admin._ +import org.apache.kafka.clients.consumer.ConsumerRecords import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.clients.producer.ProducerRecord @@ -43,11 +44,8 @@ import org.apache.kafka.common.TopicPartitionReplica import org.apache.kafka.common.acl._ import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig} import org.apache.kafka.common.errors._ -import org.apache.kafka.common.internals.KafkaFutureImpl -import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity -import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{DeleteRecordsRequest, JoinGroupRequest, MetadataResponse} +import org.apache.kafka.common.requests.{DeleteRecordsRequest, MetadataResponse} import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourceType} import org.apache.kafka.common.utils.{Time, Utils} import org.junit.Assert._ @@ -169,19 +167,37 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { new NewTopic("mytopic2", 3, 3.toShort), new NewTopic("mytopic3", Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava) ) - client.createTopics(newTopics.asJava, new CreateTopicsOptions().validateOnly(true)).all.get() + val validateResult = client.createTopics(newTopics.asJava, new CreateTopicsOptions().validateOnly(true)) + validateResult.all.get() waitForTopics(client, List(), topics) - client.createTopics(newTopics.asJava).all.get() + def validateMetadataAndConfigs(result: CreateTopicsResult): Unit = { + assertEquals(2, result.numPartitions("mytopic").get()) + assertEquals(2, result.replicationFactor("mytopic").get()) + assertEquals(3, result.numPartitions("mytopic2").get()) + assertEquals(3, result.replicationFactor("mytopic2").get()) + assertEquals(configs.head.numPartitions, result.numPartitions("mytopic3").get()) + assertEquals(configs.head.defaultReplicationFactor, result.replicationFactor("mytopic3").get()) + assertFalse(result.config("mytopic").get().entries.isEmpty) + } + validateMetadataAndConfigs(validateResult) + + val createResult = client.createTopics(newTopics.asJava) + createResult.all.get() waitForTopics(client, topics, List()) + validateMetadataAndConfigs(createResult) - val results = client.createTopics(newTopics.asJava).values() + val failedCreateResult = client.createTopics(newTopics.asJava) + val results = failedCreateResult.values() assertTrue(results.containsKey("mytopic")) assertFutureExceptionTypeEquals(results.get("mytopic"), classOf[TopicExistsException]) assertTrue(results.containsKey("mytopic2")) assertFutureExceptionTypeEquals(results.get("mytopic2"), classOf[TopicExistsException]) assertTrue(results.containsKey("mytopic3")) assertFutureExceptionTypeEquals(results.get("mytopic3"), classOf[TopicExistsException]) + assertFutureExceptionTypeEquals(failedCreateResult.numPartitions("mytopic3"), classOf[TopicExistsException]) + assertFutureExceptionTypeEquals(failedCreateResult.replicationFactor("mytopic3"), classOf[TopicExistsException]) + assertFutureExceptionTypeEquals(failedCreateResult.config("mytopic3"), classOf[TopicExistsException]) val topicToDescription = client.describeTopics(topics.asJava).all.get() assertEquals(topics.toSet, topicToDescription.keySet.asScala) @@ -1056,6 +1072,14 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { TestUtils.pollUntilTrue(consumer, () => !consumer.assignment.isEmpty, "Expected non-empty assignment") } + private def subscribeAndWaitForRecords(topic: String, consumer: KafkaConsumer[Array[Byte], Array[Byte]]): Unit = { + consumer.subscribe(Collections.singletonList(topic)) + TestUtils.pollRecordsUntilTrue( + consumer, + (records: ConsumerRecords[Array[Byte], Array[Byte]]) => !records.isEmpty, + "Expected records" ) + } + private def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], numRecords: Int, topicPartition: TopicPartition): Unit = { @@ -1264,7 +1288,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { val part = new TopicPartition(testTopicName, 0) parts.containsKey(part) && (parts.get(part).offset() == 1) }, s"Expected the offset for partition 0 to eventually become 1.") - + // Test delete non-exist consumer instance val invalidInstanceId = "invalid-instance-id" var removeMemberResult = client.removeMemberFromConsumerGroup(testGroupId, new RemoveMemberFromConsumerGroupOptions( @@ -1282,8 +1306,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { } catch { case e: ExecutionException => assertTrue(e.getCause.isInstanceOf[UnknownMemberIdException]) - case _ => - fail("Should have caught exception in getting member future") + case t: Throwable => + fail(s"Should have caught exception in getting member future: $t") } } @@ -1317,8 +1341,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { } catch { case e: ExecutionException => assertTrue(e.getCause.isInstanceOf[UnknownMemberIdException]) - case _ => - fail("Should have caught exception in getting member future") + case t: Throwable => + fail(s"Should have caught exception in getting member future: $t") } } @@ -1351,6 +1375,77 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { } } + @Test + def testDeleteConsumerGroupOffsets(): Unit = { + val config = createConfig() + client = AdminClient.create(config) + try { + val testTopicName = "test_topic" + val testGroupId = "test_group_id" + val testClientId = "test_client_id" + val fakeGroupId = "fake_group_id" + + val tp1 = new TopicPartition(testTopicName, 0) + val tp2 = new TopicPartition("foo", 0) + + client.createTopics(Collections.singleton( + new NewTopic(testTopicName, 1, 1.toShort))).all().get() + waitForTopics(client, List(testTopicName), List()) + + val producer = createProducer() + try { + producer.send(new ProducerRecord(testTopicName, 0, null, null)).get() + } finally { + Utils.closeQuietly(producer, "producer") + } + + val newConsumerConfig = new Properties(consumerConfig) + newConsumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, testGroupId) + newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId) + // Increase timeouts to avoid having a rebalance during the test + newConsumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, Integer.MAX_VALUE.toString) + newConsumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Defaults.GroupMaxSessionTimeoutMs.toString) + val consumer = createConsumer(configOverrides = newConsumerConfig) + + try { + subscribeAndWaitForRecords(testTopicName, consumer) + consumer.commitSync() + + // Test offset deletion while consuming + val offsetDeleteResult = client.deleteConsumerGroupOffsets(testGroupId, Set(tp1, tp2).asJava) + + assertNull(offsetDeleteResult.all().get()) + assertFutureExceptionTypeEquals(offsetDeleteResult.partitionResult(tp1), + classOf[GroupSubscribedToTopicException]) + assertFutureExceptionTypeEquals(offsetDeleteResult.partitionResult(tp2), + classOf[UnknownTopicOrPartitionException]) + + // Test the fake group ID + val fakeDeleteResult = client.deleteConsumerGroupOffsets(fakeGroupId, Set(tp1, tp2).asJava) + + assertFutureExceptionTypeEquals(fakeDeleteResult.all(), classOf[GroupIdNotFoundException]) + assertFutureExceptionTypeEquals(fakeDeleteResult.partitionResult(tp1), + classOf[GroupIdNotFoundException]) + assertFutureExceptionTypeEquals(fakeDeleteResult.partitionResult(tp2), + classOf[GroupIdNotFoundException]) + + } finally { + Utils.closeQuietly(consumer, "consumer") + } + + // Test offset deletion when group is empty + val offsetDeleteResult = client.deleteConsumerGroupOffsets(testGroupId, Set(tp1, tp2).asJava) + + assertNull(offsetDeleteResult.all().get()) + assertNull(offsetDeleteResult.partitionResult(tp1).get()) + assertFutureExceptionTypeEquals(offsetDeleteResult.partitionResult(tp2), + classOf[UnknownTopicOrPartitionException]) + + } finally { + Utils.closeQuietly(client, "adminClient") + } + } + @Test def testElectPreferredLeaders(): Unit = { client = AdminClient.create(createConfig) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 9896f75c6873..38fe16598f70 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -22,7 +22,7 @@ import java.time.Duration import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} import kafka.log.LogConfig import kafka.network.SocketServer -import kafka.security.auth.{ResourceType => AuthResourceType, SimpleAclAuthorizer, Topic} +import kafka.security.auth.{SimpleAclAuthorizer, Topic, ResourceType => AuthResourceType} import kafka.security.authorizer.AuthorizerUtils.WildcardHost import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.utils.TestUtils @@ -37,6 +37,7 @@ import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig} import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME +import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData import org.apache.kafka.common.message.ControlledShutdownRequestData import org.apache.kafka.common.message.CreateTopicsRequestData import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} @@ -47,13 +48,16 @@ import org.apache.kafka.common.message.FindCoordinatorRequestData import org.apache.kafka.common.message.HeartbeatRequestData import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{AlterConfigsResource, AlterableConfig, AlterableConfigCollection} -import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData -import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData import org.apache.kafka.common.message.JoinGroupRequestData import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity +import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData import org.apache.kafka.common.message.OffsetCommitRequestData import org.apache.kafka.common.message.SyncGroupRequestData +import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection +import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity +import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, Records, SimpleRecord} @@ -132,6 +136,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[SimpleAclAuthorizer].getName) properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") + properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1") properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1") properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1") @@ -176,7 +181,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.ELECT_LEADERS -> classOf[ElectLeadersResponse], ApiKeys.INCREMENTAL_ALTER_CONFIGS -> classOf[IncrementalAlterConfigsResponse], ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> classOf[AlterPartitionReassignmentsResponse], - ApiKeys.LIST_PARTITION_REASSIGNMENTS -> classOf[ListPartitionReassignmentsResponse] + ApiKeys.LIST_PARTITION_REASSIGNMENTS -> classOf[ListPartitionReassignmentsResponse], + ApiKeys.OFFSET_DELETE -> classOf[OffsetDeleteResponse] ) val requestKeyToError = Map[ApiKeys, Nothing => Errors]( @@ -198,8 +204,10 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.HEARTBEAT -> ((resp: HeartbeatResponse) => resp.error), ApiKeys.LEAVE_GROUP -> ((resp: LeaveGroupResponse) => resp.error), ApiKeys.DELETE_GROUPS -> ((resp: DeleteGroupsResponse) => resp.get(group)), - ApiKeys.LEADER_AND_ISR -> ((resp: requests.LeaderAndIsrResponse) => resp.responses.asScala.find(_._1 == tp).get._2), - ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => resp.responses.asScala.find(_._1 == tp).get._2), + ApiKeys.LEADER_AND_ISR -> ((resp: requests.LeaderAndIsrResponse) => Errors.forCode( + resp.partitions.asScala.find(p => p.topicName == tp.topic && p.partitionIndex == tp.partition).get.errorCode)), + ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => Errors.forCode( + resp.partitionErrors.asScala.find(pe => pe.topicName == tp.topic && pe.partitionIndex == tp.partition).get.errorCode)), ApiKeys.CONTROLLED_SHUTDOWN -> ((resp: requests.ControlledShutdownResponse) => resp.error), ApiKeys.CREATE_TOPICS -> ((resp: CreateTopicsResponse) => Errors.forCode(resp.data.topics.find(createTopic).errorCode())), ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => Errors.forCode(resp.data.responses.find(deleteTopic).errorCode())), @@ -231,7 +239,15 @@ class AuthorizerIntegrationTest extends BaseRequestTest { topicResourceError.error() }), ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> ((resp: AlterPartitionReassignmentsResponse) => Errors.forCode(resp.data().errorCode())), - ApiKeys.LIST_PARTITION_REASSIGNMENTS -> ((resp: ListPartitionReassignmentsResponse) => Errors.forCode(resp.data().errorCode())) + ApiKeys.LIST_PARTITION_REASSIGNMENTS -> ((resp: ListPartitionReassignmentsResponse) => Errors.forCode(resp.data().errorCode())), + ApiKeys.OFFSET_DELETE -> ((resp: OffsetDeleteResponse) => { + Errors.forCode( + resp.data + .topics().asScala.find(_.name() == topic).get + .partitions().asScala.find(_.partitionIndex() == part).get + .errorCode() + ) + }) ) val requestKeysToAcls = Map[ApiKeys, Map[ResourcePattern, Set[AccessControlEntry]]]( @@ -273,7 +289,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.ELECT_LEADERS -> clusterAlterAcl, ApiKeys.INCREMENTAL_ALTER_CONFIGS -> topicAlterConfigsAcl, ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> clusterAlterAcl, - ApiKeys.LIST_PARTITION_REASSIGNMENTS -> clusterDescribeAcl + ApiKeys.LIST_PARTITION_REASSIGNMENTS -> clusterDescribeAcl, + ApiKeys.OFFSET_DELETE -> groupReadAcl ) @Before @@ -342,14 +359,25 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } private def createUpdateMetadataRequest = { - val partitionState = Map(tp -> new UpdateMetadataRequest.PartitionState( - Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava + val partitionStates = Seq(new UpdateMetadataPartitionState() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setControllerEpoch(Int.MaxValue) + .setLeader(brokerId) + .setLeaderEpoch(Int.MaxValue) + .setIsr(List(brokerId).asJava) + .setZkVersion(2) + .setReplicas(Seq(brokerId).asJava)).asJava val securityProtocol = SecurityProtocol.PLAINTEXT - val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId, - Seq(new requests.UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol, - ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava + val brokers = Seq(new UpdateMetadataBroker() + .setId(brokerId) + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("localhost") + .setPort(0) + .setSecurityProtocol(securityProtocol.id) + .setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava)).asJava val version = ApiKeys.UPDATE_METADATA.latestVersion - new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, Long.MaxValue, partitionState, brokers).build() + new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, Long.MaxValue, partitionStates, brokers).build() } private def createJoinGroupRequest = { @@ -431,7 +459,16 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def leaderAndIsrRequest = { new requests.LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue, Long.MaxValue, - Map(tp -> new LeaderAndIsrRequest.PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, false)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setControllerEpoch(Int.MaxValue) + .setLeader(brokerId) + .setLeaderEpoch(Int.MaxValue) + .setIsr(List(brokerId).asJava) + .setZkVersion(2) + .setReplicas(Seq(brokerId).asJava) + .setIsNew(false)).asJava, Set(new Node(brokerId, "localhost", 0)).asJava).build() } @@ -521,7 +558,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { List(new ListPartitionReassignmentsRequestData.ListPartitionReassignmentsTopics() .setName(topic) .setPartitionIndexes( - List(new Integer(tp.partition)).asJava + List(Integer.valueOf(tp.partition)).asJava )).asJava ) ).build() @@ -558,12 +595,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.CREATE_PARTITIONS -> createPartitionsRequest, ApiKeys.ADD_PARTITIONS_TO_TXN -> addPartitionsToTxnRequest, ApiKeys.ADD_OFFSETS_TO_TXN -> addOffsetsToTxnRequest, - // Check StopReplica last since some APIs depend on replica availability - ApiKeys.STOP_REPLICA -> stopReplicaRequest, ApiKeys.ELECT_LEADERS -> electLeadersRequest, ApiKeys.INCREMENTAL_ALTER_CONFIGS -> incrementalAlterConfigsRequest, ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> alterPartitionReassignmentsRequest, - ApiKeys.LIST_PARTITION_REASSIGNMENTS -> listPartitionReassignmentsRequest + ApiKeys.LIST_PARTITION_REASSIGNMENTS -> listPartitionReassignmentsRequest, + // Check StopReplica last since some APIs depend on replica availability + ApiKeys.STOP_REPLICA -> stopReplicaRequest ) for ((key, request) <- requestKeyToRequest) { @@ -1322,6 +1359,56 @@ class AuthorizerIntegrationTest extends BaseRequestTest { TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } + @Test + def testDeleteGroupOffsetsWithAcl(): Unit = { + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, DELETE, ALLOW)), groupResource) + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, READ, ALLOW)), groupResource) + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, READ, ALLOW)), topicResource) + val consumer = createConsumer() + consumer.assign(List(tp).asJava) + consumer.commitSync(Map(tp -> new OffsetAndMetadata(5, "")).asJava) + consumer.close() + val result = createAdminClient().deleteConsumerGroupOffsets(group, Set(tp).asJava) + assertNull(result.partitionResult(tp).get()) + } + + @Test + def testDeleteGroupOffsetsWithoutDeleteAcl(): Unit = { + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, READ, ALLOW)), groupResource) + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, READ, ALLOW)), topicResource) + val consumer = createConsumer() + consumer.assign(List(tp).asJava) + consumer.commitSync(Map(tp -> new OffsetAndMetadata(5, "")).asJava) + consumer.close() + val result = createAdminClient().deleteConsumerGroupOffsets(group, Set(tp).asJava) + TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException]) + } + + @Test + def testDeleteGroupOffsetsWithDeleteAclWithoutTopicAcl(): Unit = { + // Create the consumer group + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, READ, ALLOW)), groupResource) + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, READ, ALLOW)), topicResource) + val consumer = createConsumer() + consumer.assign(List(tp).asJava) + consumer.commitSync(Map(tp -> new OffsetAndMetadata(5, "")).asJava) + consumer.close() + + // Remove the topic ACL & Check that it does not work without it + removeAllAcls() + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, DELETE, ALLOW)), groupResource) + addAndVerifyAcls(Set(new AccessControlEntry(userPrincipalStr, WildcardHost, READ, ALLOW)), groupResource) + val result = createAdminClient().deleteConsumerGroupOffsets(group, Set(tp).asJava) + assertNull(result.all().get()) + TestUtils.assertFutureExceptionTypeEquals(result.partitionResult(tp), classOf[TopicAuthorizationException]) + } + + @Test + def testDeleteGroupOffsetsWithNoAcl(): Unit = { + val result = createAdminClient().deleteConsumerGroupOffsets(group, Set(tp).asJava) + TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException]) + } + @Test def testUnauthorizedDeleteTopicsWithoutDescribe(): Unit = { val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 209eac0f506b..e355400fccc5 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -108,7 +108,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { if (records.nonEmpty) { consumer.commitSync() - assertEquals(consumer.position(tp), consumer.committed(tp).offset) + assertEquals(consumer.position(tp), consumer.committed(Set(tp).asJava).get(tp).offset) if (consumer.position(tp) == numRecords) { consumer.seekToBeginning(Collections.emptyList()) @@ -153,7 +153,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { } else if (coin == 2) { info("Committing offset.") consumer.commitSync() - assertEquals(consumer.position(tp), consumer.committed(tp).offset) + assertEquals(consumer.position(tp), consumer.committed(Set(tp).asJava).get(tp).offset) } } } @@ -485,7 +485,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { consumer.poll(time.Duration.ofSeconds(3L)) assertTrue("Assignment did not complete on time", assignSemaphore.tryAcquire(1, TimeUnit.SECONDS)) if (committedRecords > 0) - assertEquals(committedRecords, consumer.committed(tp).offset) + assertEquals(committedRecords, consumer.committed(Set(tp).asJava).get(tp).offset) consumer.close() } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index acb0d6b17ba7..62b358e6fa24 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -278,8 +278,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { // now we should see the committed positions from another consumer val anotherConsumer = createConsumer() - assertEquals(300, anotherConsumer.committed(tp).offset) - assertEquals(500, anotherConsumer.committed(tp2).offset) + assertEquals(300, anotherConsumer.committed(Set(tp).asJava).get(tp).offset) + assertEquals(500, anotherConsumer.committed(Set(tp2).asJava).get(tp2).offset) } @Test @@ -305,8 +305,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { // now we should see the committed positions from another consumer val anotherConsumer = createConsumer() - assertEquals(300, anotherConsumer.committed(tp).offset) - assertEquals(500, anotherConsumer.committed(tp2).offset) + assertEquals(300, anotherConsumer.committed(Set(tp).asJava).get(tp).offset) + assertEquals(500, anotherConsumer.committed(Set(tp2).asJava).get(tp2).offset) } @Test @@ -480,17 +480,17 @@ class PlaintextConsumerTest extends BaseConsumerTest { // sync commit val syncMetadata = new OffsetAndMetadata(5, Optional.of(15), "foo") consumer.commitSync(Map((tp, syncMetadata)).asJava) - assertEquals(syncMetadata, consumer.committed(tp)) + assertEquals(syncMetadata, consumer.committed(Set(tp).asJava).get(tp)) // async commit val asyncMetadata = new OffsetAndMetadata(10, "bar") sendAndAwaitAsyncCommit(consumer, Some(Map(tp -> asyncMetadata))) - assertEquals(asyncMetadata, consumer.committed(tp)) + assertEquals(asyncMetadata, consumer.committed(Set(tp).asJava).get(tp)) // handle null metadata val nullMetadata = new OffsetAndMetadata(5, null) consumer.commitSync(Map(tp -> nullMetadata).asJava) - assertEquals(nullMetadata, consumer.committed(tp)) + assertEquals(nullMetadata, consumer.committed(Set(tp).asJava).get(tp)) } @Test @@ -509,7 +509,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(None, callback.lastError) assertEquals(count, callback.successCount) - assertEquals(new OffsetAndMetadata(count), consumer.committed(tp)) + assertEquals(new OffsetAndMetadata(count), consumer.committed(Set(tp).asJava).get(tp)) } @Test @@ -623,7 +623,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { sendRecords(producer, numRecords = 5, tp) val consumer = createConsumer() - assertNull(consumer.committed(new TopicPartition(topic, 15))) + assertTrue(consumer.committed(Set(new TopicPartition(topic, 15)).asJava).isEmpty) // position() on a partition that we aren't subscribed to throws an exception intercept[IllegalStateException] { @@ -634,12 +634,12 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, consumer.position(tp)) consumer.commitSync() - assertEquals(0L, consumer.committed(tp).offset) + assertEquals(0L, consumer.committed(Set(tp).asJava).get(tp).offset) consumeAndVerifyRecords(consumer = consumer, numRecords = 5, startingOffset = 0) assertEquals("After consuming 5 records, position should be 5", 5L, consumer.position(tp)) consumer.commitSync() - assertEquals("Committed offset should be returned", 5L, consumer.committed(tp).offset) + assertEquals("Committed offset should be returned", 5L, consumer.committed(Set(tp).asJava).get(tp).offset) sendRecords(producer, numRecords = 1, tp) @@ -1024,12 +1024,12 @@ class PlaintextConsumerTest extends BaseConsumerTest { // commit sync and verify onCommit is called val commitCountBefore = MockConsumerInterceptor.ON_COMMIT_COUNT.intValue testConsumer.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(2L))).asJava) - assertEquals(2, testConsumer.committed(tp).offset) + assertEquals(2, testConsumer.committed(Set(tp).asJava).get(tp).offset) assertEquals(commitCountBefore + 1, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue) // commit async and verify onCommit is called sendAndAwaitAsyncCommit(testConsumer, Some(Map(tp -> new OffsetAndMetadata(5L)))) - assertEquals(5, testConsumer.committed(tp).offset) + assertEquals(5, testConsumer.committed(Set(tp).asJava).get(tp).offset) assertEquals(commitCountBefore + 2, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue) testConsumer.close() @@ -1076,8 +1076,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { rebalanceListener) // after rebalancing, we should have reset to the committed positions - assertEquals(10, testConsumer.committed(tp).offset) - assertEquals(20, testConsumer.committed(tp2).offset) + assertEquals(10, testConsumer.committed(Set(tp).asJava).get(tp).offset) + assertEquals(20, testConsumer.committed(Set(tp2).asJava).get(tp2).offset) assertTrue(MockConsumerInterceptor.ON_COMMIT_COUNT.intValue() > commitCountBeforeRebalance) // verify commits are intercepted on close @@ -1321,19 +1321,19 @@ class PlaintextConsumerTest extends BaseConsumerTest { val pos1 = consumer.position(tp) val pos2 = consumer.position(tp2) consumer.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(3L))).asJava) - assertEquals(3, consumer.committed(tp).offset) - assertNull(consumer.committed(tp2)) + assertEquals(3, consumer.committed(Set(tp).asJava).get(tp).offset) + assertNull(consumer.committed(Set(tp2).asJava).get(tp2)) // Positions should not change assertEquals(pos1, consumer.position(tp)) assertEquals(pos2, consumer.position(tp2)) consumer.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp2, new OffsetAndMetadata(5L))).asJava) - assertEquals(3, consumer.committed(tp).offset) - assertEquals(5, consumer.committed(tp2).offset) + assertEquals(3, consumer.committed(Set(tp).asJava).get(tp).offset) + assertEquals(5, consumer.committed(Set(tp2).asJava).get(tp2).offset) // Using async should pick up the committed changes after commit completes sendAndAwaitAsyncCommit(consumer, Some(Map(tp2 -> new OffsetAndMetadata(7L)))) - assertEquals(7, consumer.committed(tp2).offset) + assertEquals(7, consumer.committed(Set(tp2).asJava).get(tp2).offset) } @Test @@ -1371,8 +1371,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { awaitAssignment(consumer, newAssignment) // after rebalancing, we should have reset to the committed positions - assertEquals(300, consumer.committed(tp).offset) - assertEquals(500, consumer.committed(tp2).offset) + assertEquals(300, consumer.committed(Set(tp).asJava).get(tp).offset) + assertEquals(500, consumer.committed(Set(tp2).asJava).get(tp2).offset) } @Test @@ -1808,7 +1808,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { } try { - consumer2.committed(tp) + consumer2.committed(Set(tp).asJava) fail("Expected committed offset fetch to fail due to null group id") } catch { case e: InvalidGroupIdException => // OK diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminClientIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminClientIntegrationTest.scala index cad33e07a610..2a8131e560f2 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminClientIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminClientIntegrationTest.scala @@ -15,20 +15,24 @@ package kafka.api import java.io.File import java.util +import kafka.log.LogConfig import kafka.security.auth.{All, Allow, Alter, AlterConfigs, Authorizer, ClusterAction, Create, Delete, Deny, Describe, Group, Operation, PermissionType, SimpleAclAuthorizer, Topic, Acl => AuthAcl, Resource => AuthResource} import kafka.security.authorizer.AuthorizerWrapper -import kafka.server.KafkaConfig +import kafka.server.{Defaults, KafkaConfig} import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import kafka.utils.TestUtils._ import org.apache.kafka.clients.admin._ import org.apache.kafka.common.acl._ -import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidRequestException} +import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidRequestException, TopicAuthorizationException} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} -import org.junit.Assert.assertEquals +import org.junit.Assert.{assertEquals, assertTrue} import org.junit.{After, Assert, Before, Test} import scala.collection.JavaConverters._ +import scala.collection.Seq +import scala.compat.java8.OptionConverters._ import scala.util.{Failure, Success, Try} class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with SaslSetup { @@ -401,6 +405,62 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with testAclCreateGetDelete(expectAuth = false) } + @Test + def testCreateTopicsResponseMetadataAndConfig(): Unit = { + val topic1 = "mytopic1" + val topic2 = "mytopic2" + val denyAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, topic2, PatternType.LITERAL), + new AccessControlEntry("User:*", "*", AclOperation.DESCRIBE_CONFIGS, AclPermissionType.DENY)) + + client = AdminClient.create(createConfig()) + client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get() + + val topics = Seq(topic1, topic2) + val configsOverride = Map(LogConfig.SegmentBytesProp -> "100000").asJava + val newTopics = Seq( + new NewTopic(topic1, 2, 3.toShort).configs(configsOverride), + new NewTopic(topic2, Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava).configs(configsOverride)) + val validateResult = client.createTopics(newTopics.asJava, new CreateTopicsOptions().validateOnly(true)) + validateResult.all.get() + waitForTopics(client, List(), topics) + + def validateMetadataAndConfigs(result: CreateTopicsResult): Unit = { + assertEquals(2, result.numPartitions(topic1).get()) + assertEquals(3, result.replicationFactor(topic1).get()) + val topicConfigs = result.config(topic1).get().entries.asScala + assertTrue(topicConfigs.nonEmpty) + val segmentBytesConfig = topicConfigs.find(_.name == LogConfig.SegmentBytesProp).get + assertEquals(100000, segmentBytesConfig.value.toLong) + assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source) + val compressionConfig = topicConfigs.find(_.name == LogConfig.CompressionTypeProp).get + assertEquals(Defaults.CompressionType, compressionConfig.value) + assertEquals(ConfigEntry.ConfigSource.DEFAULT_CONFIG, compressionConfig.source) + + assertFutureExceptionTypeEquals(result.numPartitions(topic2), classOf[TopicAuthorizationException]) + assertFutureExceptionTypeEquals(result.replicationFactor(topic2), classOf[TopicAuthorizationException]) + assertFutureExceptionTypeEquals(result.config(topic2), classOf[TopicAuthorizationException]) + } + validateMetadataAndConfigs(validateResult) + + val createResult = client.createTopics(newTopics.asJava, new CreateTopicsOptions()) + createResult.all.get() + waitForTopics(client, topics, List()) + validateMetadataAndConfigs(createResult) + val createResponseConfig = createResult.config(topic1).get().entries.asScala + + val topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic1) + val describeResponseConfig = client.describeConfigs(List(topicResource).asJava).values.get(topicResource).get().entries.asScala + assertEquals(describeResponseConfig.size, createResponseConfig.size) + describeResponseConfig.foreach { describeEntry => + val name = describeEntry.name + val createEntry = createResponseConfig.find(_.name == name).get + assertEquals(s"Value mismatch for $name", describeEntry.value, createEntry.value) + assertEquals(s"isReadOnly mismatch for $name", describeEntry.isReadOnly, createEntry.isReadOnly) + assertEquals(s"isSensitive mismatch for $name", describeEntry.isSensitive, createEntry.isSensitive) + assertEquals(s"Source mismatch for $name", describeEntry.source, createEntry.source) + } + } + private def waitForDescribeAcls(client: Admin, filter: AclBindingFilter, acls: Set[AclBinding]): Unit = { var lastResults: util.Collection[AclBinding] = null TestUtils.waitUntilTrue(() => { diff --git a/core/src/test/scala/integration/kafka/api/SslAdminClientIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SslAdminClientIntegrationTest.scala index b3e412bd96fc..689cb0aef9f8 100644 --- a/core/src/test/scala/integration/kafka/api/SslAdminClientIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslAdminClientIntegrationTest.scala @@ -127,7 +127,7 @@ class SslAdminClientIntegrationTest extends SaslSslAdminClientIntegrationTest { def validateRequestContext(context: AuthorizableRequestContext, apiKey: ApiKeys): Unit = { assertEquals(SecurityProtocol.SSL, context.securityProtocol) - assertEquals("SSL", context.listener) + assertEquals("SSL", context.listenerName) assertEquals(KafkaPrincipal.ANONYMOUS, context.principal) assertEquals(apiKey.id.toInt, context.requestType) assertEquals(apiKey.latestVersion.toInt, context.requestVersion) diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 254ccadc8909..e1bf3db4ff9e 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -389,7 +389,7 @@ class TransactionsTest extends KafkaServerTestHarness { val producer2 = transactionalProducers(1) producer2.initTransactions() - assertEquals(offsetAndMetadata, consumer.committed(tp)) + assertEquals(offsetAndMetadata, consumer.committed(Set(tp).asJava).get(tp)) } @Test diff --git a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala index 871953addd3d..16e3cbc9260c 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala @@ -65,10 +65,6 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { } } - override protected def brokerPropertyOverrides(properties: Properties): Unit = { - super.brokerPropertyOverrides(properties) - } - @Test def testDynamicConnectionQuota(): Unit = { val maxConnectionsPerIP = 5 diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index f58046ad4891..acab29bd161b 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -260,7 +260,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet val SslKeystorePasswordVal = "${file:ssl.keystore.password:password}" val configPrefix = listenerPrefix(SecureExternal) - var brokerConfigs = describeConfig(adminClients.head, servers).entries.asScala + val brokerConfigs = describeConfig(adminClients.head, servers).entries.asScala // the following are values before updated assertTrue("Initial value of polling interval", brokerConfigs.find(_.name == TestMetricsReporter.PollingIntervalProp) == None) assertTrue("Initial value of ssl truststore type", brokerConfigs.find(_.name == configPrefix+KafkaConfig.SslTruststoreTypeProp) == None) diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala index 3da7a35352c5..4fd110aab6a1 100644 --- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -278,7 +278,6 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { @Test(expected = classOf[IllegalArgumentException]) def testEntityDefaultOptionWithDescribeBrokerLoggerIsNotAllowed(): Unit = { - val node = new Node(1, "localhost", 9092) val optsList = List("--bootstrap-server", "localhost:9092", "--entity-type", ConfigCommand.BrokerLoggerConfigType, "--entity-default", @@ -290,7 +289,6 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { @Test(expected = classOf[IllegalArgumentException]) def testEntityDefaultOptionWithAlterBrokerLoggerIsNotAllowed(): Unit = { - val node = new Node(1, "localhost", 9092) val optsList = List("--bootstrap-server", "localhost:9092", "--entity-type", ConfigCommand.BrokerLoggerConfigType, "--entity-default", diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala index fdadec41e372..3ee83e25eeab 100644 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala @@ -27,7 +27,7 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AdminClientConfig import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import org.apache.kafka.common.errors.WakeupException import org.apache.kafka.common.serialization.StringDeserializer import org.junit.{After, Before} @@ -70,14 +70,10 @@ class ConsumerGroupCommandTest extends KafkaServerTestHarness { props.put("group.id", group) val consumer = new KafkaConsumer(props, new StringDeserializer, new StringDeserializer) try { - consumer.partitionsFor(topic).asScala.flatMap { partitionInfo => - val tp = new TopicPartition(partitionInfo.topic, partitionInfo.partition) - val committed = consumer.committed(tp) - if (committed == null) - None - else - Some(tp -> committed.offset) - }.toMap + val partitions: Set[TopicPartition] = consumer.partitionsFor(topic) + .asScala.toSet.map {partitionInfo : PartitionInfo => new TopicPartition(partitionInfo.topic, partitionInfo.partition)} + + consumer.committed(partitions.asJava).asScala.mapValues(_.offset()).toMap } finally { consumer.close() } diff --git a/core/src/test/scala/unit/kafka/admin/PreferredReplicaLeaderElectionCommandTest.scala b/core/src/test/scala/unit/kafka/admin/PreferredReplicaLeaderElectionCommandTest.scala index 295fc3bd87fd..37032f17501a 100644 --- a/core/src/test/scala/unit/kafka/admin/PreferredReplicaLeaderElectionCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/PreferredReplicaLeaderElectionCommandTest.scala @@ -93,8 +93,8 @@ class PreferredReplicaLeaderElectionCommandTest extends ZooKeeperTestHarness wit debug(s"Starting server $targetServer now that a non-preferred replica is leader") servers(targetServer).startup() TestUtils.waitUntilTrue(() => servers.forall { server => - server.metadataCache.getPartitionInfo(partition.topic(), partition.partition()).exists { partitionState => - partitionState.basePartitionState.isr.contains(targetServer) + server.metadataCache.getPartitionInfo(partition.topic, partition.partition).exists { partitionState => + partitionState.isr.contains(targetServer) } }, s"Replicas for partition $partition not created") @@ -106,8 +106,7 @@ class PreferredReplicaLeaderElectionCommandTest extends ZooKeeperTestHarness wit private def awaitLeader(topicPartition: TopicPartition, timeoutMs: Long = test.TestUtils.DEFAULT_MAX_WAIT_MS): Int = { TestUtils.awaitValue(() => { - servers.head.metadataCache.getPartitionInfo(topicPartition.topic, topicPartition.partition) - .map(_.basePartitionState.leader) + servers.head.metadataCache.getPartitionInfo(topicPartition.topic, topicPartition.partition).map(_.leader) }, s"Timed out waiting to find current leader of $topicPartition", timeoutMs) } diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala index 0c2bdd1b7a7c..10ed6bb98ad4 100644 --- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala @@ -684,6 +684,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { assertEquals(Seq(2, 1), zkClient.getReplicasForPartition(new TopicPartition("orders", 0))) assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1))) assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(sameMoveTp)) + assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 3))) assertEquals(Seq.empty, zkClient.getReplicasForPartition(new TopicPartition("customers", 0))) } diff --git a/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala b/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala index 571a0775bd3a..dadef1d00d5d 100644 --- a/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala +++ b/core/src/test/scala/unit/kafka/api/ApiVersionTest.scala @@ -88,6 +88,13 @@ class ApiVersionTest { assertEquals(KAFKA_2_2_IV1, ApiVersion("2.2")) assertEquals(KAFKA_2_2_IV0, ApiVersion("2.2-IV0")) assertEquals(KAFKA_2_2_IV1, ApiVersion("2.2-IV1")) + + assertEquals(KAFKA_2_3_IV1, ApiVersion("2.3")) + assertEquals(KAFKA_2_3_IV0, ApiVersion("2.3-IV0")) + assertEquals(KAFKA_2_3_IV1, ApiVersion("2.3-IV1")) + + assertEquals(KAFKA_2_4_IV0, ApiVersion("2.4")) + assertEquals(KAFKA_2_4_IV0, ApiVersion("2.4-IV0")) } @Test @@ -116,7 +123,22 @@ class ApiVersionTest { def testShortVersion(): Unit = { assertEquals("0.8.0", KAFKA_0_8_0.shortVersion) assertEquals("0.10.0", KAFKA_0_10_0_IV0.shortVersion) + assertEquals("0.10.0", KAFKA_0_10_0_IV1.shortVersion) assertEquals("0.11.0", KAFKA_0_11_0_IV0.shortVersion) + assertEquals("0.11.0", KAFKA_0_11_0_IV1.shortVersion) + assertEquals("0.11.0", KAFKA_0_11_0_IV2.shortVersion) + assertEquals("1.0", KAFKA_1_0_IV0.shortVersion) + assertEquals("1.1", KAFKA_1_1_IV0.shortVersion) + assertEquals("2.0", KAFKA_2_0_IV0.shortVersion) + assertEquals("2.0", KAFKA_2_0_IV1.shortVersion) + assertEquals("2.1", KAFKA_2_1_IV0.shortVersion) + assertEquals("2.1", KAFKA_2_1_IV1.shortVersion) + assertEquals("2.1", KAFKA_2_1_IV2.shortVersion) + assertEquals("2.2", KAFKA_2_2_IV0.shortVersion) + assertEquals("2.2", KAFKA_2_2_IV1.shortVersion) + assertEquals("2.3", KAFKA_2_3_IV0.shortVersion) + assertEquals("2.3", KAFKA_2_3_IV1.shortVersion) + assertEquals("2.4", KAFKA_2_4_IV0.shortVersion) } @Test diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index aeb9768fe735..8e812ebd0031 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -32,11 +32,12 @@ import kafka.server.checkpoints.OffsetCheckpoints import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{ApiException, OffsetNotAvailableException, ReplicaNotAvailableException} +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.record._ -import org.apache.kafka.common.requests.{EpochEndOffset, IsolationLevel, LeaderAndIsrRequest, ListOffsetRequest} +import org.apache.kafka.common.requests.{EpochEndOffset, IsolationLevel, ListOffsetRequest} import org.junit.{After, Before, Test} import org.junit.Assert._ import org.mockito.Mockito.{doNothing, mock, when} @@ -482,9 +483,14 @@ class PartitionTest { new SimpleRecord(20,"k4".getBytes, "v2".getBytes), new SimpleRecord(21,"k5".getBytes, "v3".getBytes))) - val leaderState = new LeaderAndIsrRequest.PartitionState( - controllerEpoch, leader, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true - ) + val leaderState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(true) assertTrue("Expected first makeLeader() to return 'leader changed'", partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints)) @@ -552,15 +558,27 @@ class PartitionTest { assertEquals(Right(None), fetchOffsetsForTimestamp(30, Some(IsolationLevel.READ_UNCOMMITTED))) // Make into a follower - val followerState = new LeaderAndIsrRequest.PartitionState( - controllerEpoch, follower2, leaderEpoch + 1, isr, 4, replicas.map(Int.box).asJava, false - ) + val followerState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(follower2) + .setLeaderEpoch(leaderEpoch + 1) + .setIsr(isr) + .setZkVersion(4) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(false) + assertTrue(partition.makeFollower(controllerId, followerState, 1, offsetCheckpoints)) // Back to leader, this resets the startLogOffset for this epoch (to 2), we're now in the fault condition - val newLeaderState = new LeaderAndIsrRequest.PartitionState( - controllerEpoch, leader, leaderEpoch + 2, isr, 5, replicas.map(Int.box).asJava, false - ) + val newLeaderState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch + 2) + .setIsr(isr) + .setZkVersion(5) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(false) + assertTrue(partition.makeLeader(controllerId, newLeaderState, 2, offsetCheckpoints)) // Try to get offsets as a client @@ -633,13 +651,25 @@ class PartitionTest { if (isLeader) { assertTrue("Expected become leader transition to succeed", - partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId, - leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints)) + partition.makeLeader(controllerId, new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true), 0, offsetCheckpoints)) assertEquals(leaderEpoch, partition.getLeaderEpoch) } else { assertTrue("Expected become follower transition to succeed", - partition.makeFollower(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId + 1, - leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints)) + partition.makeFollower(controllerId, new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId + 1) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true), 0, offsetCheckpoints)) assertEquals(leaderEpoch, partition.getLeaderEpoch) assertEquals(None, partition.leaderLogIfLocal) } @@ -710,8 +740,14 @@ class PartitionTest { partition.createLogIfNotExists(brokerId, isNew = false, isFutureReplica = false, offsetCheckpoints) assertTrue("Expected become leader transition to succeed", - partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId, - leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints)) + partition.makeLeader(controllerId, new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true), 0, offsetCheckpoints)) assertEquals(leaderEpoch, partition.getLeaderEpoch) val records = createTransactionalRecords(List( @@ -773,19 +809,36 @@ class PartitionTest { @Test def testMakeFollowerWithNoLeaderIdChange(): Unit = { // Start off as follower - var partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 1, - List[Integer](0, 1, 2, brokerId).asJava, 1, List[Integer](0, 1, 2, brokerId).asJava, false) - partition.makeFollower(0, partitionStateInfo, 0, offsetCheckpoints) + var partitionState = new LeaderAndIsrPartitionState() + .setControllerEpoch(0) + .setLeader(1) + .setLeaderEpoch(1) + .setIsr(List[Integer](0, 1, 2, brokerId).asJava) + .setZkVersion(1) + .setReplicas(List[Integer](0, 1, 2, brokerId).asJava) + .setIsNew(false) + partition.makeFollower(0, partitionState, 0, offsetCheckpoints) // Request with same leader and epoch increases by only 1, do become-follower steps - partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4, - List[Integer](0, 1, 2, brokerId).asJava, 1, List[Integer](0, 1, 2, brokerId).asJava, false) - assertTrue(partition.makeFollower(0, partitionStateInfo, 2, offsetCheckpoints)) + partitionState = new LeaderAndIsrPartitionState() + .setControllerEpoch(0) + .setLeader(1) + .setLeaderEpoch(4) + .setIsr(List[Integer](0, 1, 2, brokerId).asJava) + .setZkVersion(1) + .setReplicas(List[Integer](0, 1, 2, brokerId).asJava) + .setIsNew(false) + assertTrue(partition.makeFollower(0, partitionState, 2, offsetCheckpoints)) // Request with same leader and same epoch, skip become-follower steps - partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4, - List[Integer](0, 1, 2, brokerId).asJava, 1, List[Integer](0, 1, 2, brokerId).asJava, false) - assertFalse(partition.makeFollower(0, partitionStateInfo, 2, offsetCheckpoints)) + partitionState = new LeaderAndIsrPartitionState() + .setControllerEpoch(0) + .setLeader(1) + .setLeaderEpoch(4) + .setIsr(List[Integer](0, 1, 2, brokerId).asJava) + .setZkVersion(1) + .setReplicas(List[Integer](0, 1, 2, brokerId).asJava) + assertFalse(partition.makeFollower(0, partitionState, 2, offsetCheckpoints)) } @Test @@ -806,7 +859,14 @@ class PartitionTest { val batch3 = TestUtils.records(records = List(new SimpleRecord("k6".getBytes, "v1".getBytes), new SimpleRecord("k7".getBytes, "v2".getBytes))) - val leaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 1, replicas, true) + val leaderState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true) assertTrue("Expected first makeLeader() to return 'leader changed'", partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints)) assertEquals("Current leader epoch", leaderEpoch, partition.getLeaderEpoch) @@ -834,12 +894,24 @@ class PartitionTest { assertEquals("Expected leader's HW", lastOffsetOfFirstBatch, partition.log.get.highWatermark) // current leader becomes follower and then leader again (without any new records appended) - val followerState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, follower2, leaderEpoch + 1, isr, 1, - replicas, false) + val followerState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(follower2) + .setLeaderEpoch(leaderEpoch + 1) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(false) partition.makeFollower(controllerId, followerState, 1, offsetCheckpoints) - val newLeaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch + 2, isr, 1, - replicas, false) + val newLeaderState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch + 2) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(false) assertTrue("Expected makeLeader() to return 'leader changed' after makeFollower()", partition.makeLeader(controllerEpoch, newLeaderState, 2, offsetCheckpoints)) val currentLeaderEpochStartOffset = partition.localLogOrException.logEndOffset @@ -904,8 +976,14 @@ class PartitionTest { }) partition.setLog(log, isFutureLog = false) - val leaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId, - leaderEpoch, isr, 1, replicaIds, true) + val leaderState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicaIds) + .setIsNew(true) partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints) partitions += partition } @@ -989,7 +1067,14 @@ class PartitionTest { assertFalse(partition.isAtMinIsr) // Make isr set to only have leader to trigger AtMinIsr (default min isr config is 1) - val leaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 1, replicas, true) + val leaderState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true) partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints) assertTrue(partition.isAtMinIsr) } @@ -1012,8 +1097,18 @@ class PartitionTest { val initializeTimeMs = time.milliseconds() assertTrue("Expected become leader transition to succeed", - partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId, - leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints)) + partition.makeLeader( + controllerId, + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true), + 0, + offsetCheckpoints)) val remoteReplica = partition.getReplica(remoteBrokerId).get assertEquals(initializeTimeMs, remoteReplica.lastCaughtUpTimeMs) @@ -1065,11 +1160,16 @@ class PartitionTest { "Expected become leader transition to succeed", partition.makeLeader( controllerId, - new LeaderAndIsrRequest.PartitionState( - controllerEpoch, brokerId, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true), + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(true), 0, - offsetCheckpoints - ) + offsetCheckpoints) ) assertEquals(Set(brokerId), partition.inSyncReplicaIds) @@ -1122,8 +1222,18 @@ class PartitionTest { partition.createLogIfNotExists(brokerId, isNew = false, isFutureReplica = false, offsetCheckpoints) assertTrue("Expected become leader transition to succeed", - partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId, - leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints)) + partition.makeLeader( + controllerId, + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true), + 0, + offsetCheckpoints)) assertEquals(Set(brokerId), partition.inSyncReplicaIds) val remoteReplica = partition.getReplica(remoteBrokerId).get @@ -1170,11 +1280,16 @@ class PartitionTest { "Expected become leader transition to succeed", partition.makeLeader( controllerId, - new LeaderAndIsrRequest.PartitionState( - controllerEpoch, brokerId, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true), + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(true), 0, - offsetCheckpoints - ) + offsetCheckpoints) ) assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds) assertEquals(0L, partition.localLogOrException.highWatermark) @@ -1222,11 +1337,16 @@ class PartitionTest { "Expected become leader transition to succeed", partition.makeLeader( controllerId, - new LeaderAndIsrRequest.PartitionState( - controllerEpoch, brokerId, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true), + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(true), 0, - offsetCheckpoints - ) + offsetCheckpoints) ) assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds) assertEquals(0L, partition.localLogOrException.highWatermark) @@ -1289,11 +1409,16 @@ class PartitionTest { "Expected become leader transition to succeed", partition.makeLeader( controllerId, - new LeaderAndIsrRequest.PartitionState( - controllerEpoch, brokerId, leaderEpoch, isr, 1, replicas.map(Int.box).asJava, true), + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(true), 0, - offsetCheckpoints - ) + offsetCheckpoints) ) assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds) assertEquals(0L, partition.localLogOrException.highWatermark) @@ -1339,8 +1464,18 @@ class PartitionTest { val initializeTimeMs = time.milliseconds() partition.createLogIfNotExists(brokerId, isNew = false, isFutureReplica = false, offsetCheckpoints) assertTrue("Expected become leader transition to succeed", - partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId, - leaderEpoch, isr, 1, replicas, true), 0, offsetCheckpoints)) + partition.makeLeader( + controllerId, + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true), + 0, + offsetCheckpoints)) assertEquals(Set(brokerId, remoteBrokerId), partition.inSyncReplicaIds) assertEquals(0L, partition.localLogOrException.highWatermark) @@ -1375,8 +1510,14 @@ class PartitionTest { val controllerId = 0 val controllerEpoch = 3 val replicas = List[Integer](brokerId, brokerId + 1).asJava - val leaderState = new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId, - 6, replicas, 1, replicas, false) + val leaderState = new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(6) + .setIsr(replicas) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(false) partition.makeLeader(controllerId, leaderState, 0, offsetCheckpoints) assertEquals(4, partition.localLogOrException.highWatermark) } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala index 9c2941ae2fd3..e603b25cfa3f 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala @@ -18,11 +18,14 @@ package kafka.controller import java.util.Properties -import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_0_10_2_IV0, KAFKA_0_9_0, KAFKA_1_0_IV0, KAFKA_2_2_IV0, LeaderAndIsr} +import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_0_10_2_IV0, KAFKA_0_9_0, KAFKA_1_0_IV0, KAFKA_2_2_IV0, KAFKA_2_4_IV0, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.message.{LeaderAndIsrResponseData, StopReplicaResponseData} +import org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError +import org.apache.kafka.common.message.StopReplicaResponseData.StopReplicaPartitionError import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{AbstractControlRequest, AbstractResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, StopReplicaRequest, StopReplicaResponse, UpdateMetadataRequest} @@ -70,18 +73,21 @@ class ControllerChannelManagerTest { val leaderAndIsrRequest = leaderAndIsrRequests.head assertEquals(controllerId, leaderAndIsrRequest.controllerId) assertEquals(controllerEpoch, leaderAndIsrRequest.controllerEpoch) - assertEquals(partitions.keySet, leaderAndIsrRequest.partitionStates.keySet.asScala) + assertEquals(partitions.keySet, + leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex)).toSet) assertEquals(partitions.map { case (k, v) => (k, v.leader) }, - leaderAndIsrRequest.partitionStates.asScala.mapValues(_.basePartitionState.leader).toMap) + leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex) -> p.leader).toMap) assertEquals(partitions.map { case (k, v) => (k, v.isr) }, - leaderAndIsrRequest.partitionStates.asScala.mapValues(_.basePartitionState.isr.asScala).toMap) + leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex) -> p.isr.asScala).toMap) applyLeaderAndIsrResponseCallbacks(Errors.NONE, batch.sentRequests(2).toList) assertEquals(1, batch.sentEvents.size) val LeaderAndIsrResponseReceived(response, brokerId) = batch.sentEvents.head + val leaderAndIsrResponse = response.asInstanceOf[LeaderAndIsrResponse] assertEquals(2, brokerId) - assertEquals(partitions.keySet, response.asInstanceOf[LeaderAndIsrResponse].responses.keySet.asScala) + assertEquals(partitions.keySet, + leaderAndIsrResponse.partitions.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex)).toSet) } @Test @@ -106,8 +112,10 @@ class ControllerChannelManagerTest { assertEquals(1, updateMetadataRequests.size) val leaderAndIsrRequest = leaderAndIsrRequests.head - assertEquals(Set(partition), leaderAndIsrRequest.partitionStates.keySet.asScala) - assertTrue(leaderAndIsrRequest.partitionStates.get(partition).isNew) + val partitionStates = leaderAndIsrRequest.partitionStates.asScala + assertEquals(Seq(partition), partitionStates.map(p => new TopicPartition(p.topicName, p.partitionIndex))) + val partitionState = partitionStates.find(p => p.topicName == partition.topic && p.partitionIndex == partition.partition) + assertEquals(Some(true), partitionState.map(_.isNew)) } @Test @@ -139,7 +147,7 @@ class ControllerChannelManagerTest { assertEquals(1, leaderAndIsrRequests.size) assertEquals(1, updateMetadataRequests.size) val leaderAndIsrRequest = leaderAndIsrRequests.head - assertEquals(Set(partition), leaderAndIsrRequest.partitionStates.keySet.asScala) + assertEquals(Seq(partition), leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex))) } } @@ -149,8 +157,9 @@ class ControllerChannelManagerTest { for (apiVersion <- ApiVersion.allVersions) { val leaderAndIsrRequestVersion: Short = - if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 2 - else if (config.interBrokerProtocolVersion >= KAFKA_1_0_IV0) 1 + if (apiVersion >= KAFKA_2_4_IV0) 3 + else if (apiVersion >= KAFKA_2_2_IV0) 2 + else if (apiVersion >= KAFKA_1_0_IV0) 1 else 0 testLeaderAndIsrRequestFollowsInterBrokerProtocolVersion(apiVersion, leaderAndIsrRequestVersion) @@ -173,9 +182,10 @@ class ControllerChannelManagerTest { batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(1, 2, 3)), isNew = false) batch.sendRequestsToBrokers(controllerEpoch) - val leaderAndIsrRequests = batch.collectLeaderAndIsrRequestsFor(2, expectedLeaderAndIsrVersion) + val leaderAndIsrRequests = batch.collectLeaderAndIsrRequestsFor(2) assertEquals(1, leaderAndIsrRequests.size) - assertEquals(expectedLeaderAndIsrVersion, leaderAndIsrRequests.head.version) + assertEquals(s"IBP $interBrokerProtocolVersion should use version $expectedLeaderAndIsrVersion", + expectedLeaderAndIsrVersion, leaderAndIsrRequests.head.version) } @Test @@ -201,11 +211,12 @@ class ControllerChannelManagerTest { assertEquals(1, updateMetadataRequests.size) val updateMetadataRequest = updateMetadataRequests.head - assertEquals(3, updateMetadataRequest.partitionStates.size) + val partitionStates = updateMetadataRequest.partitionStates.asScala.toBuffer + assertEquals(3, partitionStates.size) assertEquals(partitions.map { case (k, v) => (k, v.leader) }, - updateMetadataRequest.partitionStates.asScala.map { case (k, v) => (k, v.basePartitionState.leader) }) + partitionStates.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.leader)).toMap) assertEquals(partitions.map { case (k, v) => (k, v.isr) }, - updateMetadataRequest.partitionStates.asScala.map { case (k, v) => (k, v.basePartitionState.isr.asScala) }) + partitionStates.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.isr.asScala)).toMap) assertEquals(controllerId, updateMetadataRequest.controllerId) assertEquals(controllerEpoch, updateMetadataRequest.controllerEpoch) @@ -236,7 +247,7 @@ class ControllerChannelManagerTest { assertEquals(1, updateMetadataRequests.size) val updateMetadataRequest = updateMetadataRequests.head - assertEquals(0, updateMetadataRequest.partitionStates.size) + assertEquals(0, updateMetadataRequest.partitionStates.asScala.size) assertEquals(3, updateMetadataRequest.liveBrokers.size) assertEquals(Set(1, 2, 3), updateMetadataRequest.liveBrokers.asScala.map(_.id).toSet) } @@ -266,19 +277,18 @@ class ControllerChannelManagerTest { assertEquals(1, updateMetadataRequests.size) val updateMetadataRequest = updateMetadataRequests.head - assertEquals(3, updateMetadataRequest.partitionStates.size) + assertEquals(3, updateMetadataRequest.partitionStates.asScala.size) assertTrue(updateMetadataRequest.partitionStates.asScala - .filterKeys(_.topic == "foo") - .values - .map(_.basePartitionState.leader) + .filter(_.topicName == "foo") + .map(_.leader) .forall(leaderId => leaderId == LeaderAndIsr.LeaderDuringDelete)) assertEquals(partitions.filter { case (k, _) => k.topic == "bar" }.map { case (k, v) => (k, v.leader) }, - updateMetadataRequest.partitionStates.asScala.filter { case (k, _) => k.topic == "bar" }.map { case (k, v) => - (k, v.basePartitionState.leader) }) + updateMetadataRequest.partitionStates.asScala.filter(ps => ps.topicName == "bar").map { ps => + (new TopicPartition(ps.topicName, ps.partitionIndex), ps.leader) }.toMap) assertEquals(partitions.map { case (k, v) => (k, v.isr) }, - updateMetadataRequest.partitionStates.asScala.map { case (k, v) => (k, v.basePartitionState.isr.asScala) }) + updateMetadataRequest.partitionStates.asScala.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.isr.asScala)).toMap) assertEquals(3, updateMetadataRequest.liveBrokers.size) assertEquals(Set(1, 2, 3), updateMetadataRequest.liveBrokers.asScala.map(_.id).toSet) @@ -304,7 +314,7 @@ class ControllerChannelManagerTest { assertEquals(1, updateMetadataRequests.size) val updateMetadataRequest = updateMetadataRequests.head - assertEquals(0, updateMetadataRequest.partitionStates.size) + assertEquals(0, updateMetadataRequest.partitionStates.asScala.size) assertEquals(2, updateMetadataRequest.liveBrokers.size) assertEquals(Set(1, 2), updateMetadataRequest.liveBrokers.asScala.map(_.id).toSet) } @@ -316,11 +326,11 @@ class ControllerChannelManagerTest { for (apiVersion <- ApiVersion.allVersions) { val updateMetadataRequestVersion: Short = - if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 5 - else if (config.interBrokerProtocolVersion >= KAFKA_1_0_IV0) 4 - else if (config.interBrokerProtocolVersion >= KAFKA_0_10_2_IV0) 3 - else if (config.interBrokerProtocolVersion >= KAFKA_0_10_0_IV1) 2 - else if (config.interBrokerProtocolVersion >= KAFKA_0_9_0) 1 + if (apiVersion >= KAFKA_2_2_IV0) 5 + else if (apiVersion >= KAFKA_1_0_IV0) 4 + else if (apiVersion >= KAFKA_0_10_2_IV0) 3 + else if (apiVersion >= KAFKA_0_10_0_IV1) 2 + else if (apiVersion >= KAFKA_0_9_0) 1 else 0 testUpdateMetadataFollowsInterBrokerProtocolVersion(apiVersion, updateMetadataRequestVersion) @@ -341,8 +351,11 @@ class ControllerChannelManagerTest { assertEquals(1, batch.sentRequests.size) assertTrue(batch.sentRequests.contains(2)) - val requests = batch.collectUpdateMetadataRequestsFor(2, expectedUpdateMetadataVersion) - assertTrue(requests.forall(_.version == expectedUpdateMetadataVersion)) + val requests = batch.collectUpdateMetadataRequestsFor(2) + val allVersions = requests.map(_.version) + assertTrue(s"IBP $interBrokerProtocolVersion should use version $expectedUpdateMetadataVersion, " + + s"but found versions $allVersions", + allVersions.forall(_ == expectedUpdateMetadataVersion)) } @Test @@ -369,7 +382,7 @@ class ControllerChannelManagerTest { val sentRequests = batch.sentRequests(2) assertEquals(1, sentRequests.size) - val sentStopReplicaRequests = batch.collectStopReplicRequestsFor(2) + val sentStopReplicaRequests = batch.collectStopReplicaRequestsFor(2) assertEquals(1, sentStopReplicaRequests.size) val stopReplicaRequest = sentStopReplicaRequests.head @@ -407,10 +420,10 @@ class ControllerChannelManagerTest { val sentRequests = batch.sentRequests(2) assertEquals(1, sentRequests.size) - val sentStopReplicaRequests = batch.collectStopReplicRequestsFor(2) + val sentStopReplicaRequests = batch.collectStopReplicaRequestsFor(2) assertEquals(1, sentStopReplicaRequests.size) assertEquals(partitions, sentStopReplicaRequests.flatMap(_.partitions.asScala).toSet) - assertTrue(sentStopReplicaRequests.forall(_.deletePartitions())) + assertTrue(sentStopReplicaRequests.forall(_.deletePartitions)) // No events will be sent after the response returns applyStopReplicaResponseCallbacks(Errors.NONE, batch.sentRequests(2).toList) @@ -444,7 +457,7 @@ class ControllerChannelManagerTest { val sentRequests = batch.sentRequests(2) assertEquals(1, sentRequests.size) - val sentStopReplicaRequests = batch.collectStopReplicRequestsFor(2) + val sentStopReplicaRequests = batch.collectStopReplicaRequestsFor(2) assertEquals(1, sentStopReplicaRequests.size) assertEquals(partitions, sentStopReplicaRequests.flatMap(_.partitions.asScala).toSet) assertTrue(sentStopReplicaRequests.forall(_.deletePartitions())) @@ -493,7 +506,7 @@ class ControllerChannelManagerTest { val sentRequests = batch.sentRequests(2) assertEquals(2, sentRequests.size) - val sentStopReplicaRequests = batch.collectStopReplicRequestsFor(2) + val sentStopReplicaRequests = batch.collectStopReplicaRequestsFor(2) assertEquals(2, sentStopReplicaRequests.size) val (deleteRequests, nonDeleteRequests) = sentStopReplicaRequests.partition(_.deletePartitions()) @@ -529,7 +542,7 @@ class ControllerChannelManagerTest { assertEquals(1, sentRequests.size) for (brokerId <- Set(2, 3)) { - val sentStopReplicaRequests = batch.collectStopReplicRequestsFor(brokerId) + val sentStopReplicaRequests = batch.collectStopReplicaRequestsFor(brokerId) assertEquals(1, sentStopReplicaRequests.size) val stopReplicaRequest = sentStopReplicaRequests.head @@ -569,7 +582,7 @@ class ControllerChannelManagerTest { val sentRequests = batch.sentRequests(2) assertEquals(1, sentRequests.size) - val sentStopReplicaRequests = batch.collectStopReplicRequestsFor(2) + val sentStopReplicaRequests = batch.collectStopReplicaRequestsFor(2) assertEquals(1, sentStopReplicaRequests.size) val stopReplicaRequest = sentStopReplicaRequests.head @@ -583,14 +596,14 @@ class ControllerChannelManagerTest { for (apiVersion <- ApiVersion.allVersions) { if (apiVersion < KAFKA_2_2_IV0) - testStopReplicaFollowsInterBrokerProtocolVersion(ApiVersion.latestVersion, 0.toShort) + testStopReplicaFollowsInterBrokerProtocolVersion(apiVersion, 0.toShort) else - testStopReplicaFollowsInterBrokerProtocolVersion(ApiVersion.latestVersion, 1.toShort) + testStopReplicaFollowsInterBrokerProtocolVersion(apiVersion, 1.toShort) } } private def testStopReplicaFollowsInterBrokerProtocolVersion(interBrokerProtocolVersion: ApiVersion, - expectedStopReplicaRequestVersion: Short): Unit = { + expectedStopReplicaRequestVersion: Short): Unit = { val context = initContext(Seq(1, 2, 3), Set("foo"), 2, 3) val config = createConfig(interBrokerProtocolVersion) val batch = new MockControllerBrokerRequestBatch(context, config) @@ -605,15 +618,28 @@ class ControllerChannelManagerTest { assertEquals(1, batch.sentRequests.size) assertTrue(batch.sentRequests.contains(2)) - val requests = batch.collectStopReplicRequestsFor(2, expectedStopReplicaRequestVersion) - assertTrue(requests.forall(_.version() == expectedStopReplicaRequestVersion)) + val requests = batch.collectStopReplicaRequestsFor(2) + val allVersions = requests.map(_.version) + assertTrue(s"IBP $interBrokerProtocolVersion should use version $expectedStopReplicaRequestVersion, " + + s"but found versions $allVersions", + allVersions.forall(_ == expectedStopReplicaRequestVersion)) } private def applyStopReplicaResponseCallbacks(error: Errors, sentRequests: List[SentRequest]): Unit = { sentRequests.filter(_.responseCallback != null).foreach { sentRequest => val stopReplicaRequest = sentRequest.request.build().asInstanceOf[StopReplicaRequest] - val partitionErrorMap = stopReplicaRequest.partitions.asScala.map(_ -> error).toMap.asJava - val stopReplicaResponse = new StopReplicaResponse(error, partitionErrorMap) + val stopReplicaResponse = + if (error == Errors.NONE) { + val partitionErrors = stopReplicaRequest.partitions.asScala.map { tp => + new StopReplicaPartitionError() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setErrorCode(error.code) + }.toBuffer.asJava + new StopReplicaResponse(new StopReplicaResponseData().setPartitionErrors(partitionErrors)) + } else { + stopReplicaRequest.getErrorResponse(error.exception) + } sentRequest.responseCallback.apply(stopReplicaResponse) } } @@ -621,9 +647,16 @@ class ControllerChannelManagerTest { private def applyLeaderAndIsrResponseCallbacks(error: Errors, sentRequests: List[SentRequest]): Unit = { sentRequests.filter(_.request.apiKey == ApiKeys.LEADER_AND_ISR).filter(_.responseCallback != null).foreach { sentRequest => val leaderAndIsrRequest = sentRequest.request.build().asInstanceOf[LeaderAndIsrRequest] - val partitionErrorMap = leaderAndIsrRequest.partitionStates.asScala.keySet.map(_ -> error).toMap.asJava - val leaderAndIsrResponse = new LeaderAndIsrResponse(error, partitionErrorMap) - sentRequest.responseCallback.apply(leaderAndIsrResponse) + val partitionErrors = leaderAndIsrRequest.partitionStates.asScala.map(p => + new LeaderAndIsrPartitionError() + .setTopicName(p.topicName) + .setPartitionIndex(p.partitionIndex) + .setErrorCode(error.code)) + val leaderAndIsrResponse = new LeaderAndIsrResponse( + new LeaderAndIsrResponseData() + .setErrorCode(error.code) + .setPartitionErrors(partitionErrors.toBuffer.asJava)) + sentRequest.responseCallback(leaderAndIsrResponse) } } @@ -631,7 +664,8 @@ class ControllerChannelManagerTest { val props = new Properties() props.put(KafkaConfig.BrokerIdProp, controllerId.toString) props.put(KafkaConfig.ZkConnectProp, "zkConnect") - props.put(KafkaConfig.InterBrokerProtocolVersionProp, ApiVersion.latestVersion.version) + props.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerVersion.version) + props.put(KafkaConfig.LogMessageFormatVersionProp, interBrokerVersion.version) KafkaConfig.fromProps(props) } @@ -680,32 +714,29 @@ class ControllerChannelManagerTest { sentRequests(brokerId).append(SentRequest(request, callback)) } - def collectStopReplicRequestsFor(brokerId: Int, - version: Short = ApiKeys.STOP_REPLICA.latestVersion): List[StopReplicaRequest] = { + def collectStopReplicaRequestsFor(brokerId: Int): List[StopReplicaRequest] = { sentRequests.get(brokerId) match { case Some(requests) => requests .filter(_.request.apiKey == ApiKeys.STOP_REPLICA) - .map(_.request.build(version).asInstanceOf[StopReplicaRequest]).toList + .map(_.request.build().asInstanceOf[StopReplicaRequest]).toList case None => List.empty[StopReplicaRequest] } } - def collectUpdateMetadataRequestsFor(brokerId: Int, - version: Short = ApiKeys.UPDATE_METADATA.latestVersion): List[UpdateMetadataRequest] = { + def collectUpdateMetadataRequestsFor(brokerId: Int): List[UpdateMetadataRequest] = { sentRequests.get(brokerId) match { case Some(requests) => requests .filter(_.request.apiKey == ApiKeys.UPDATE_METADATA) - .map(_.request.build(version).asInstanceOf[UpdateMetadataRequest]).toList + .map(_.request.build().asInstanceOf[UpdateMetadataRequest]).toList case None => List.empty[UpdateMetadataRequest] } } - def collectLeaderAndIsrRequestsFor(brokerId: Int, - version: Short = ApiKeys.LEADER_AND_ISR.latestVersion): List[LeaderAndIsrRequest] = { + def collectLeaderAndIsrRequestsFor(brokerId: Int): List[LeaderAndIsrRequest] = { sentRequests.get(brokerId) match { case Some(requests) => requests .filter(_.request.apiKey == ApiKeys.LEADER_AND_ISR) - .map(_.request.build(version).asInstanceOf[LeaderAndIsrRequest]).toList + .map(_.request.build().asInstanceOf[LeaderAndIsrRequest]).toList case None => List.empty[LeaderAndIsrRequest] } } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 867f4c6fa151..dc09d7300abb 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -143,10 +143,10 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { val offlineReplicaPartitionInfo = server.metadataCache.getPartitionInfo(topic, 0).get assertEquals(1, offlineReplicaPartitionInfo.offlineReplicas.size()) assertEquals(testBroker.config.brokerId, offlineReplicaPartitionInfo.offlineReplicas.get(0)) - assertEquals(assignment(0).asJava, offlineReplicaPartitionInfo.basePartitionState.replicas) - assertEquals(Seq(remainingBrokers.head.config.brokerId).asJava, offlineReplicaPartitionInfo.basePartitionState.isr) + assertEquals(assignment(0).asJava, offlineReplicaPartitionInfo.replicas) + assertEquals(Seq(remainingBrokers.head.config.brokerId).asJava, offlineReplicaPartitionInfo.isr) val onlinePartitionInfo = server.metadataCache.getPartitionInfo(topic, 1).get - assertEquals(assignment(1).asJava, onlinePartitionInfo.basePartitionState.replicas) + assertEquals(assignment(1).asJava, onlinePartitionInfo.replicas) assertTrue(onlinePartitionInfo.offlineReplicas.isEmpty) } @@ -158,7 +158,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { val partitionInfoOpt = server.metadataCache.getPartitionInfo(topic, partitionId) if (partitionInfoOpt.isDefined) { val partitionInfo = partitionInfoOpt.get - !partitionInfo.offlineReplicas.isEmpty || !partitionInfo.basePartitionState.replicas.asScala.equals(replicas) + !partitionInfo.offlineReplicas.isEmpty || !partitionInfo.replicas.asScala.equals(replicas) } else { true } @@ -204,8 +204,8 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { val partitionInfoOpt = broker.metadataCache.getPartitionInfo(topic, 0) if (partitionInfoOpt.isDefined) { val partitionInfo = partitionInfoOpt.get - (!partitionInfo.offlineReplicas.isEmpty && partitionInfo.basePartitionState.leader == -1 - && !partitionInfo.basePartitionState.replicas.isEmpty && !partitionInfo.basePartitionState.isr.isEmpty) + (!partitionInfo.offlineReplicas.isEmpty && partitionInfo.leader == -1 + && !partitionInfo.replicas.isEmpty && !partitionInfo.isr.isEmpty) } else { false } @@ -463,14 +463,14 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { var activeServers = servers.filter(s => s.config.brokerId != 2) // wait for the update metadata request to trickle to the brokers TestUtils.waitUntilTrue(() => - activeServers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.basePartitionState.isr.size != 3), + activeServers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.isr.size != 3), "Topic test not created after timeout") assertEquals(0, partitionsRemaining.size) var partitionStateInfo = activeServers.head.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get - var leaderAfterShutdown = partitionStateInfo.basePartitionState.leader + var leaderAfterShutdown = partitionStateInfo.leader assertEquals(0, leaderAfterShutdown) - assertEquals(2, partitionStateInfo.basePartitionState.isr.size) - assertEquals(List(0,1), partitionStateInfo.basePartitionState.isr.asScala) + assertEquals(2, partitionStateInfo.isr.size) + assertEquals(List(0,1), partitionStateInfo.isr.asScala) controller.controlledShutdown(1, servers.find(_.config.brokerId == 1).get.kafkaController.brokerEpoch, controlledShutdownCallback) partitionsRemaining = resultQueue.take() match { case Success(partitions) => partitions @@ -479,15 +479,15 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { assertEquals(0, partitionsRemaining.size) activeServers = servers.filter(s => s.config.brokerId == 0) partitionStateInfo = activeServers.head.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get - leaderAfterShutdown = partitionStateInfo.basePartitionState.leader + leaderAfterShutdown = partitionStateInfo.leader assertEquals(0, leaderAfterShutdown) - assertTrue(servers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.basePartitionState.leader == 0)) + assertTrue(servers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.leader == 0)) controller.controlledShutdown(0, servers.find(_.config.brokerId == 0).get.kafkaController.brokerEpoch, controlledShutdownCallback) partitionsRemaining = resultQueue.take().get assertEquals(1, partitionsRemaining.size) // leader doesn't change since all the replicas are shut down - assertTrue(servers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.basePartitionState.leader == 0)) + assertTrue(servers.forall(_.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic,partition).get.leader == 0)) } @Test diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index 1b7b68f528d3..9cc0d3a18ab2 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -34,6 +34,8 @@ import java.util.concurrent.locks.ReentrantLock import kafka.cluster.Partition import kafka.zk.KafkaZkClient +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity @@ -41,7 +43,8 @@ import org.junit.Assert._ import org.junit.{After, Assert, Before, Test} import org.scalatest.Assertions.intercept -import scala.collection.mutable +import scala.collection.JavaConverters._ +import scala.collection.{Seq, mutable} import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future, Promise, TimeoutException} @@ -2818,6 +2821,216 @@ class GroupCoordinatorTest { assertEquals(Dead.toString, groupCoordinator.handleDescribeGroup(groupId)._2.state) } + @Test + def testDeleteOffsetOfNonExistingGroup(): Unit = { + val tp = new TopicPartition("foo", 0) + val (groupError, topics) = groupCoordinator.handleDeleteOffsets(groupId, Seq(tp)) + + assertEquals(Errors.GROUP_ID_NOT_FOUND, groupError) + assert(topics.isEmpty) + } + + @Test + def testDeleteOffsetOfNonEmptyNonConsumerGroup(): Unit = { + val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID + dynamicJoinGroup(groupId, memberId, "My Protocol", protocols) + val tp = new TopicPartition("foo", 0) + val (groupError, topics) = groupCoordinator.handleDeleteOffsets(groupId, Seq(tp)) + + assertEquals(Errors.NON_EMPTY_GROUP, groupError) + assert(topics.isEmpty) + } + + @Test + def testDeleteOffsetOfEmptyNonConsumerGroup(): Unit = { + // join the group + val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID + + val joinGroupResult = dynamicJoinGroup(groupId, memberId, "My Protocol", protocols) + assertEquals(Errors.NONE, joinGroupResult.error) + + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.leaderId, Map.empty) + assertEquals(Errors.NONE, syncGroupResult._2) + + val t1p0 = new TopicPartition("foo", 0) + val t2p0 = new TopicPartition("bar", 0) + val offset = offsetAndMetadata(37) + + EasyMock.reset(replicaManager) + val validOffsetCommitResult = commitOffsets(groupId, joinGroupResult.memberId, joinGroupResult.generationId, + Map(t1p0 -> offset, t2p0 -> offset)) + assertEquals(Errors.NONE, validOffsetCommitResult(t1p0)) + assertEquals(Errors.NONE, validOffsetCommitResult(t2p0)) + + // and leaves. + EasyMock.reset(replicaManager) + val leaveGroupResults = singleLeaveGroup(groupId, joinGroupResult.memberId) + verifyLeaveGroupResult(leaveGroupResults) + + assert(groupCoordinator.groupManager.getGroup(groupId).exists(_.is(Empty))) + + val groupTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) + val partition: Partition = EasyMock.niceMock(classOf[Partition]) + + EasyMock.reset(replicaManager) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) + EasyMock.expect(replicaManager.getPartition(groupTopicPartition)).andStubReturn(HostedPartition.Online(partition)) + EasyMock.expect(replicaManager.nonOfflinePartition(groupTopicPartition)).andStubReturn(Some(partition)) + EasyMock.replay(replicaManager, partition) + + val (groupError, topics) = groupCoordinator.handleDeleteOffsets(groupId, Seq(t1p0)) + + assertEquals(Errors.NONE, groupError) + assert(topics.size == 1) + assertEquals(Some(Errors.NONE), topics.get(t1p0)) + + val cachedOffsets = groupCoordinator.groupManager.getOffsets(groupId, Some(Seq(t1p0, t2p0))) + + assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(t1p0).map(_.offset)) + assertEquals(Some(offset.offset), cachedOffsets.get(t2p0).map(_.offset)) + } + + @Test + def testDeleteOffsetOfConsumerGroupWithUnparsableProtocol(): Unit = { + val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID + val joinGroupResult = dynamicJoinGroup(groupId, memberId, protocolType, protocols) + + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.leaderId, Map.empty) + assertEquals(Errors.NONE, syncGroupResult._2) + + val t1p0 = new TopicPartition("foo", 0) + val t2p0 = new TopicPartition("bar", 0) + val offset = offsetAndMetadata(37) + + EasyMock.reset(replicaManager) + val validOffsetCommitResult = commitOffsets(groupId, joinGroupResult.memberId, joinGroupResult.generationId, + Map(t1p0 -> offset)) + assertEquals(Errors.NONE, validOffsetCommitResult(t1p0)) + + val (groupError, topics) = groupCoordinator.handleDeleteOffsets(groupId, Seq(t1p0, t2p0)) + + assertEquals(Errors.NONE, groupError) + assert(topics.size == 2) + assertEquals(Some(Errors.GROUP_SUBSCRIBED_TO_TOPIC), topics.get(t1p0)) + assertEquals(Some(Errors.UNKNOWN_TOPIC_OR_PARTITION), topics.get(t2p0)) + } + + @Test + def testDeleteOffsetOfDeadConsumerGroup(): Unit = { + val group = new GroupMetadata(groupId, Dead, new MockTime()) + group.protocolType = Some(protocolType) + groupCoordinator.groupManager.addGroup(group) + + val tp = new TopicPartition("foo", 0) + val (groupError, topics) = groupCoordinator.handleDeleteOffsets(groupId, Seq(tp)) + + assertEquals(Errors.GROUP_ID_NOT_FOUND, groupError) + assert(topics.size == 0) + } + + @Test + def testDeleteOffsetOfEmptyConsumerGroup(): Unit = { + // join the group + val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID + val joinGroupResult = dynamicJoinGroup(groupId, memberId, protocolType, protocols) + assertEquals(Errors.NONE, joinGroupResult.error) + + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.leaderId, Map.empty) + assertEquals(Errors.NONE, syncGroupResult._2) + + val t1p0 = new TopicPartition("foo", 0) + val t2p0 = new TopicPartition("bar", 0) + val t3p0 = new TopicPartition("unknown", 0) + val offset = offsetAndMetadata(37) + + EasyMock.reset(replicaManager) + val validOffsetCommitResult = commitOffsets(groupId, joinGroupResult.memberId, joinGroupResult.generationId, + Map(t1p0 -> offset, t2p0 -> offset)) + assertEquals(Errors.NONE, validOffsetCommitResult(t1p0)) + assertEquals(Errors.NONE, validOffsetCommitResult(t2p0)) + + // and leaves. + EasyMock.reset(replicaManager) + val leaveGroupResults = singleLeaveGroup(groupId, joinGroupResult.memberId) + verifyLeaveGroupResult(leaveGroupResults) + + assert(groupCoordinator.groupManager.getGroup(groupId).exists(_.is(Empty))) + + val groupTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) + val partition: Partition = EasyMock.niceMock(classOf[Partition]) + + EasyMock.reset(replicaManager) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) + EasyMock.expect(replicaManager.getPartition(groupTopicPartition)).andStubReturn(HostedPartition.Online(partition)) + EasyMock.expect(replicaManager.nonOfflinePartition(groupTopicPartition)).andStubReturn(Some(partition)) + EasyMock.replay(replicaManager, partition) + + val (groupError, topics) = groupCoordinator.handleDeleteOffsets(groupId, Seq(t1p0, t3p0)) + + assertEquals(Errors.NONE, groupError) + assert(topics.size == 2) + assertEquals(Some(Errors.NONE), topics.get(t1p0)) + assertEquals(Some(Errors.UNKNOWN_TOPIC_OR_PARTITION), topics.get(t3p0)) + + val cachedOffsets = groupCoordinator.groupManager.getOffsets(groupId, Some(Seq(t1p0, t2p0))) + + assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(t1p0).map(_.offset)) + assertEquals(Some(offset.offset), cachedOffsets.get(t2p0).map(_.offset)) + } + + @Test + def testDeleteOffsetOfStableConsumerGroup(): Unit = { + // join the group + val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID + val subscription = new Subscription(List("bar").asJava) + + val joinGroupResult = dynamicJoinGroup(groupId, memberId, protocolType, + List(("protocol", ConsumerProtocol.serializeSubscription(subscription).array()))) + assertEquals(Errors.NONE, joinGroupResult.error) + + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.leaderId, Map.empty) + assertEquals(Errors.NONE, syncGroupResult._2) + + val t1p0 = new TopicPartition("foo", 0) + val t2p0 = new TopicPartition("bar", 0) + val t3p0 = new TopicPartition("unknown", 0) + val offset = offsetAndMetadata(37) + + EasyMock.reset(replicaManager) + val validOffsetCommitResult = commitOffsets(groupId, joinGroupResult.memberId, joinGroupResult.generationId, + Map(t1p0 -> offset, t2p0 -> offset)) + assertEquals(Errors.NONE, validOffsetCommitResult(t1p0)) + assertEquals(Errors.NONE, validOffsetCommitResult(t2p0)) + + assert(groupCoordinator.groupManager.getGroup(groupId).exists(_.is(Stable))) + + val groupTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) + val partition: Partition = EasyMock.niceMock(classOf[Partition]) + + EasyMock.reset(replicaManager) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) + EasyMock.expect(replicaManager.getPartition(groupTopicPartition)).andStubReturn(HostedPartition.Online(partition)) + EasyMock.expect(replicaManager.nonOfflinePartition(groupTopicPartition)).andStubReturn(Some(partition)) + EasyMock.replay(replicaManager, partition) + + val (groupError, topics) = groupCoordinator.handleDeleteOffsets(groupId, Seq(t1p0, t2p0, t3p0)) + + assertEquals(Errors.NONE, groupError) + assert(topics.size == 3) + assertEquals(Some(Errors.NONE), topics.get(t1p0)) + assertEquals(Some(Errors.GROUP_SUBSCRIBED_TO_TOPIC), topics.get(t2p0)) + assertEquals(Some(Errors.UNKNOWN_TOPIC_OR_PARTITION), topics.get(t3p0)) + + val cachedOffsets = groupCoordinator.groupManager.getOffsets(groupId, Some(Seq(t1p0, t2p0))) + + assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(t1p0).map(_.offset)) + assertEquals(Some(offset.offset), cachedOffsets.get(t2p0).map(_.offset)) + } + @Test def shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup(): Unit = { val firstJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index f961d8e18d53..516180d5c733 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -804,6 +804,74 @@ class GroupMetadataManagerTest { assertEquals(staticMemberId, group.getStaticMemberId(groupInstanceId)) } + @Test + def testLoadConsumerGroup(): Unit = { + val generation = 27 + val protocolType = "consumer" + val protocol = "protocol" + val memberId = "member1" + val topic = "foo" + + val subscriptions = List( + ("protocol", ConsumerProtocol.serializeSubscription(new Subscription(List(topic).asJava)).array()) + ) + + val member = new MemberMetadata(memberId, groupId, groupInstanceId, "", "", rebalanceTimeout, + sessionTimeout, protocolType, subscriptions) + + val members = Seq(member) + + val group = GroupMetadata.loadGroup(groupId, Stable, generation, protocolType, protocol, null, None, + members, time) + + assertTrue(group.is(Stable)) + assertEquals(generation, group.generationId) + assertEquals(Some(protocolType), group.protocolType) + assertEquals(protocol, group.protocolOrNull) + assertEquals(Some(Set(topic)), group.getSubscribedTopics) + assertTrue(group.has(memberId)) + } + + @Test + def testLoadEmptyConsumerGroup(): Unit = { + val generation = 27 + val protocolType = "consumer" + + val group = GroupMetadata.loadGroup(groupId, Empty, generation, protocolType, null, null, None, + Seq(), time) + + assertTrue(group.is(Empty)) + assertEquals(generation, group.generationId) + assertEquals(Some(protocolType), group.protocolType) + assertNull(group.protocolOrNull) + assertEquals(Some(Set.empty), group.getSubscribedTopics) + } + + @Test + def testLoadConsumerGroupWithFaultyConsumerProtocol(): Unit = { + val generation = 27 + val protocolType = "consumer" + val protocol = "protocol" + val memberId = "member1" + + val subscriptions = List(("protocol", Array[Byte]())) + + val member = new MemberMetadata(memberId, groupId, groupInstanceId, "", "", rebalanceTimeout, + sessionTimeout, protocolType, subscriptions) + + val members = Seq(member) + + val group = GroupMetadata.loadGroup(groupId, Stable, generation, protocolType, protocol, null, None, + members, time) + + assertTrue(group.is(Stable)) + assertEquals(generation, group.generationId) + assertEquals(Some(protocolType), group.protocolType) + assertEquals(protocol, group.protocolOrNull) + assertEquals(None, group.getSubscribedTopics) + assertTrue(group.has(memberId)) + } + @Test def testReadFromOldGroupMetadata(): Unit = { val generation = 1 @@ -1673,6 +1741,144 @@ class GroupMetadataManagerTest { assert(group.is(Dead)) } + @Test + def testOffsetExpirationOfActiveGroupSemantics(): Unit = { + val memberId = "memberId" + val clientId = "clientId" + val clientHost = "localhost" + + val topic1 = "foo" + val topic1Partition0 = new TopicPartition(topic1, 0) + val topic1Partition1 = new TopicPartition(topic1, 1) + + val topic2 = "bar" + val topic2Partition0 = new TopicPartition(topic2, 0) + val topic2Partition1 = new TopicPartition(topic2, 1) + + val offset = 37 + + groupMetadataManager.addPartitionOwnership(groupPartitionId) + + val group = new GroupMetadata(groupId, Empty, time) + groupMetadataManager.addGroup(group) + + // Subscribe to topic1 and topic2 + val subscriptionTopic1AndTopic2 = new Subscription(List(topic1, topic2).asJava) + + val member = new MemberMetadata( + memberId, + groupId, + groupInstanceId, + clientId, + clientHost, + rebalanceTimeout, + sessionTimeout, + ConsumerProtocol.PROTOCOL_TYPE, + List(("protocol", ConsumerProtocol.serializeSubscription(subscriptionTopic1AndTopic2).array())) + ) + + group.add(member, _ => ()) + group.transitionTo(PreparingRebalance) + group.initNextGeneration() + group.transitionTo(Stable) + + val startMs = time.milliseconds + + val t1p0OffsetAndMetadata = OffsetAndMetadata(offset, "", startMs) + val t1p1OffsetAndMetadata = OffsetAndMetadata(offset, "", startMs) + + val t2p0OffsetAndMetadata = OffsetAndMetadata(offset, "", startMs) + val t2p1OffsetAndMetadata = OffsetAndMetadata(offset, "", startMs) + + val offsets = immutable.Map( + topic1Partition0 -> t1p0OffsetAndMetadata, + topic1Partition1 -> t1p1OffsetAndMetadata, + topic2Partition0 -> t2p0OffsetAndMetadata, + topic2Partition1 -> t2p1OffsetAndMetadata) + + mockGetPartition() + expectAppendMessage(Errors.NONE) + EasyMock.replay(replicaManager) + + var commitErrors: Option[immutable.Map[TopicPartition, Errors]] = None + def callback(errors: immutable.Map[TopicPartition, Errors]): Unit = { + commitErrors = Some(errors) + } + + groupMetadataManager.storeOffsets(group, memberId, offsets, callback) + assertTrue(group.hasOffsets) + + assertFalse(commitErrors.isEmpty) + assertEquals(Some(Errors.NONE), commitErrors.get.get(topic1Partition0)) + + // advance time to just after the offset of last partition is to be expired + time.sleep(defaultOffsetRetentionMs + 2) + + // no offset should expire because all topics are actively consumed + groupMetadataManager.cleanupGroupMetadata() + + assertEquals(Some(group), groupMetadataManager.getGroup(groupId)) + assert(group.is(Stable)) + + assertEquals(Some(t1p0OffsetAndMetadata), group.offset(topic1Partition0)) + assertEquals(Some(t1p1OffsetAndMetadata), group.offset(topic1Partition1)) + assertEquals(Some(t2p0OffsetAndMetadata), group.offset(topic2Partition0)) + assertEquals(Some(t2p1OffsetAndMetadata), group.offset(topic2Partition1)) + + var cachedOffsets = groupMetadataManager.getOffsets(groupId, + Some(Seq(topic1Partition0, topic1Partition1, topic2Partition0, topic2Partition1))) + + assertEquals(Some(offset), cachedOffsets.get(topic1Partition0).map(_.offset)) + assertEquals(Some(offset), cachedOffsets.get(topic1Partition1).map(_.offset)) + assertEquals(Some(offset), cachedOffsets.get(topic2Partition0).map(_.offset)) + assertEquals(Some(offset), cachedOffsets.get(topic2Partition1).map(_.offset)) + + EasyMock.verify(replicaManager) + + group.transitionTo(PreparingRebalance) + + // Subscribe to topic1, offsets of topic2 should be removed + val subscriptionTopic1 = new Subscription(List(topic1).asJava) + + group.updateMember( + member, + List(("protocol", ConsumerProtocol.serializeSubscription(subscriptionTopic1).array())), + null + ) + + group.initNextGeneration() + group.transitionTo(Stable) + + // expect the offset tombstone + EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), + isFromClient = EasyMock.eq(false), requiredAcks = EasyMock.anyInt())) + .andReturn(LogAppendInfo.UnknownLogAppendInfo) + EasyMock.expectLastCall().times(1) + + EasyMock.replay(partition) + + groupMetadataManager.cleanupGroupMetadata() + + EasyMock.verify(partition) + EasyMock.verify(replicaManager) + + assertEquals(Some(group), groupMetadataManager.getGroup(groupId)) + assert(group.is(Stable)) + + assertEquals(Some(t1p0OffsetAndMetadata), group.offset(topic1Partition0)) + assertEquals(Some(t1p1OffsetAndMetadata), group.offset(topic1Partition1)) + assertEquals(None, group.offset(topic2Partition0)) + assertEquals(None, group.offset(topic2Partition1)) + + cachedOffsets = groupMetadataManager.getOffsets(groupId, + Some(Seq(topic1Partition0, topic1Partition1, topic2Partition0, topic2Partition1))) + + assertEquals(Some(offset), cachedOffsets.get(topic1Partition0).map(_.offset)) + assertEquals(Some(offset), cachedOffsets.get(topic1Partition1).map(_.offset)) + assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(topic2Partition0).map(_.offset)) + assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(topic2Partition1).map(_.offset)) + } + @Test def testLoadOffsetFromOldCommit() = { val groupMetadataTopicPartition = groupTopicPartition diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala index 6436e959bc98..82f151c36173 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala @@ -18,12 +18,16 @@ package kafka.coordinator.group import kafka.common.OffsetAndMetadata +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.utils.Time import org.junit.Assert._ import org.junit.{Before, Test} +import scala.collection.JavaConverters._ + /** * Test group state transitions and other GroupMetadata functionality */ @@ -262,6 +266,57 @@ class GroupMetadataTest { assertFalse(group.supportsProtocols(protocolType, Set("range", "foo"))) } + @Test + def testSubscribedTopics(): Unit = { + // not able to compute it for a newly created group + assertEquals(None, group.getSubscribedTopics) + + val memberId = "memberId" + val member = new MemberMetadata(memberId, groupId, groupInstanceId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, protocolType, List(("range", ConsumerProtocol.serializeSubscription(new Subscription(List("foo").asJava)).array()))) + + group.transitionTo(PreparingRebalance) + group.add(member) + + group.initNextGeneration() + + assertEquals(Some(Set("foo")), group.getSubscribedTopics) + + group.transitionTo(PreparingRebalance) + group.remove(memberId) + + group.initNextGeneration() + + assertEquals(Some(Set.empty), group.getSubscribedTopics) + + val memberWithFaultyProtocol = new MemberMetadata(memberId, groupId, groupInstanceId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, protocolType, List(("range", Array.empty[Byte]))) + + group.transitionTo(PreparingRebalance) + group.add(memberWithFaultyProtocol) + + group.initNextGeneration() + + assertEquals(None, group.getSubscribedTopics) + } + + @Test + def testSubscribedTopicsNonConsumerGroup(): Unit = { + // not able to compute it for a newly created group + assertEquals(None, group.getSubscribedTopics) + + val memberId = "memberId" + val member = new MemberMetadata(memberId, groupId, groupInstanceId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, "My Protocol", List(("range", Array.empty[Byte]))) + + group.transitionTo(PreparingRebalance) + group.add(member) + + group.initNextGeneration() + + assertEquals(None, group.getSubscribedTopics) + } + @Test def testInitNextGeneration(): Unit = { member.supportedProtocols = List(("roundrobin", Array.empty[Byte])) diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index 4996eec52322..ac8b787a8331 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -255,7 +255,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { //make sure follower server joins the ISR TestUtils.waitUntilTrue(() => { val partitionInfoOpt = followerServer.metadataCache.getPartitionInfo(topic, partitionId) - partitionInfoOpt.isDefined && partitionInfoOpt.get.basePartitionState.isr.contains(followerId) + partitionInfoOpt.isDefined && partitionInfoOpt.get.isr.contains(followerId) }, "Inconsistent metadata after first server startup") servers.filter(server => server.config.brokerId == leaderId).foreach(server => shutdownServer(server)) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index e39f697efcc1..556810eff6fb 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -115,14 +115,22 @@ class LogConfigTest { assertFalse(isValid("100:0,10 : ")) } - /* Sanity check that toHtml produces one of the expected configs */ + /* Sanity check that toHtmlTable produces one of the expected configs */ @Test - def testToHtml(): Unit = { + def testToHtmlTable(): Unit = { val html = LogConfig.configDefCopy.toHtmlTable val expectedConfig = "
    " assertTrue(s"Could not find `$expectedConfig` in:\n $html", html.contains(expectedConfig)) } + /* Sanity check that toHtml produces one of the expected configs */ + @Test + def testToHtml(): Unit = { + val html = LogConfig.configDefCopy.toHtml + val expectedConfig = "
  • file.delete.delay.ms" + assertTrue(s"Could not find `$expectedConfig` in:\n $html", html.contains(expectedConfig)) + } + /* Sanity check that toEnrichedRst produces one of the expected configs */ @Test def testToEnrichedRst(): Unit = { diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index 875e15e805f3..62040d5e262f 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -30,10 +30,8 @@ import kafka.utils._ import scala.collection._ import scala.collection.JavaConverters._ -import scala.util.matching.Regex import kafka.log.LogConfig import org.apache.kafka.common.TopicPartition -import org.scalatest.Assertions class MetricsTest extends KafkaServerTestHarness with Logging { val numNodes = 2 diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 7072c94f7653..2cf637a26360 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -32,7 +32,7 @@ import kafka.security.CredentialProvider import kafka.server.{KafkaConfig, ThrottledChannel} import kafka.utils.Implicits._ import kafka.utils.{CoreUtils, TestUtils} -import org.apache.kafka.common.{Endpoint, TopicPartition} +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.KafkaChannel.ChannelMuteState @@ -202,6 +202,10 @@ class SocketServerTest { val config = KafkaConfig.fromProps(testProps) val testableServer = new TestableSocketServer(config) testableServer.startup(startupProcessors = false) + val updatedEndPoints = config.advertisedListeners.map { endpoint => + endpoint.copy(port = testableServer.boundPort(endpoint.listenerName)) + }.map(_.toJava) + val externalReadyFuture = new CompletableFuture[Void]() val executor = Executors.newSingleThreadExecutor() @@ -221,7 +225,7 @@ class SocketServerTest { sendAndReceiveControllerRequest(socket1, testableServer) val externalListener = new ListenerName("EXTERNAL") - val externalEndpoint = new Endpoint(externalListener.value, SecurityProtocol.PLAINTEXT, "localhost", 0) + val externalEndpoint = updatedEndPoints.find(e => e.listenerName.get == externalListener.value).get val futures = Map(externalEndpoint -> externalReadyFuture) val startFuture = executor.submit(CoreUtils.runnable(testableServer.startDataPlaneProcessors(futures))) TestUtils.waitUntilTrue(() => listenerStarted(config.interBrokerListenerName), "Inter-broker listener not started") diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index faeb34df132a..2062fb762028 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -127,8 +127,8 @@ abstract class BaseRequestTest extends IntegrationTestHarness { /** * Serializes and sends the request to the given api. */ - def send(request: AbstractRequest, apiKey: ApiKeys, socket: Socket, apiVersion: Option[Short] = None): Unit = { - val header = nextRequestHeader(apiKey, apiVersion.getOrElse(request.version)) + def send(request: AbstractRequest, apiKey: ApiKeys, socket: Socket, apiVersion: Short): Unit = { + val header = nextRequestHeader(apiKey, apiVersion) val serializedBytes = request.serialize(header).array sendRequest(socket, serializedBytes) } @@ -136,9 +136,9 @@ abstract class BaseRequestTest extends IntegrationTestHarness { /** * Receive response and return a ByteBuffer containing response without the header */ - def receive(socket: Socket): ByteBuffer = { + def receive(socket: Socket, headerVersion: Short): ByteBuffer = { val response = receiveResponse(socket) - skipResponseHeader(response) + skipResponseHeader(response, headerVersion) } /** @@ -146,9 +146,10 @@ abstract class BaseRequestTest extends IntegrationTestHarness { * A ByteBuffer containing the response is returned. */ def sendAndReceive(request: AbstractRequest, apiKey: ApiKeys, socket: Socket, apiVersion: Option[Short] = None): ByteBuffer = { - send(request, apiKey, socket, apiVersion) + val version = apiVersion.getOrElse(request.version) + send(request, apiKey, socket, version) val response = receiveResponse(socket) - skipResponseHeader(response) + skipResponseHeader(response, apiKey.headerVersion(version)) } /** @@ -159,7 +160,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness { val request = requestBuilder.build() val header = new RequestHeader(apiKey, request.version, clientId, correlationId) val response = requestAndReceive(socket, request.serialize(header).array) - val responseBuffer = skipResponseHeader(response) + val responseBuffer = skipResponseHeader(response, header.headerVersion()) apiKey.parseResponse(request.version, responseBuffer) } @@ -171,13 +172,13 @@ abstract class BaseRequestTest extends IntegrationTestHarness { val header = nextRequestHeader(apiKey, apiVersion) val serializedBytes = AbstractRequestResponse.serialize(header.toStruct, requestStruct).array val response = requestAndReceive(socket, serializedBytes) - skipResponseHeader(response) + skipResponseHeader(response, header.headerVersion()) } - protected def skipResponseHeader(response: Array[Byte]): ByteBuffer = { + protected def skipResponseHeader(response: Array[Byte], headerVersion: Short): ByteBuffer = { val responseBuffer = ByteBuffer.wrap(response) // Parse the header to ensure its valid and move the buffer forward - ResponseHeader.parse(responseBuffer) + ResponseHeader.parse(responseBuffer, headerVersion) responseBuffer } diff --git a/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala index d80b5268ac5c..23836dd66d8c 100755 --- a/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala @@ -24,10 +24,11 @@ import kafka.utils.TestUtils import kafka.utils.TestUtils.createTopic import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState +import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests.UpdateMetadataRequest.EndPoint import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Time @@ -132,10 +133,17 @@ class BrokerEpochIntegrationTest extends ZooKeeperTestHarness { try { // Send LeaderAndIsr request with correct broker epoch { - val partitionStates = Map( - tp -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, brokerId2, LeaderAndIsr.initialLeaderEpoch + 1, - Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion, - Seq(0, 1).map(Integer.valueOf).asJava, false) + val partitionStates = Seq( + new LeaderAndIsrPartitionState() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId2) + .setLeaderEpoch(LeaderAndIsr.initialLeaderEpoch + 1) + .setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava) + .setZkVersion(LeaderAndIsr.initialZKVersion) + .setReplicas(Seq(0, 1).map(Integer.valueOf).asJava) + .setIsNew(false) ) val requestBuilder = new LeaderAndIsrRequest.Builder( ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, controllerEpoch, @@ -153,32 +161,43 @@ class BrokerEpochIntegrationTest extends ZooKeeperTestHarness { // Send UpdateMetadata request with correct broker epoch { - val partitionStates = Map( - tp -> new UpdateMetadataRequest.PartitionState(controllerEpoch, brokerId2, LeaderAndIsr.initialLeaderEpoch + 1, - Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion, - Seq(0, 1).map(Integer.valueOf).asJava, Seq.empty.asJava) - ) - val liverBrokers = brokerAndEpochs.map { brokerAndEpoch => - val broker = brokerAndEpoch._1 + val partitionStates = Seq( + new UpdateMetadataPartitionState() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId2) + .setLeaderEpoch(LeaderAndIsr.initialLeaderEpoch + 1) + .setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava) + .setZkVersion(LeaderAndIsr.initialZKVersion) + .setReplicas(Seq(0, 1).map(Integer.valueOf).asJava)) + val liveBrokers = brokerAndEpochs.map { case (broker, _) => val securityProtocol = SecurityProtocol.PLAINTEXT val listenerName = ListenerName.forSecurityProtocol(securityProtocol) val node = broker.node(listenerName) - val endPoints = Seq(new EndPoint(node.host, node.port, securityProtocol, listenerName)) - new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull) - } + val endpoints = Seq(new UpdateMetadataEndpoint() + .setHost(node.host) + .setPort(node.port) + .setSecurityProtocol(securityProtocol.id) + .setListener(listenerName.value)) + new UpdateMetadataBroker() + .setId(broker.id) + .setEndpoints(endpoints.asJava) + .setRack(broker.rack.orNull) + }.toBuffer val requestBuilder = new UpdateMetadataRequest.Builder( ApiKeys.UPDATE_METADATA.latestVersion, controllerId, controllerEpoch, epochInRequest, - partitionStates.asJava, liverBrokers.toSet.asJava) + partitionStates.asJava, liveBrokers.asJava) if (isEpochInRequestStale) { sendAndVerifyStaleBrokerEpochInResponse(controllerChannelManager, requestBuilder) } else { sendAndVerifySuccessfulResponse(controllerChannelManager, requestBuilder) - TestUtils.waitUntilMetadataIsPropagated(Seq(broker2), tp.topic(), tp.partition(), 10000) + TestUtils.waitUntilMetadataIsPropagated(Seq(broker2), tp.topic, tp.partition, 10000) assertEquals(brokerId2, - broker2.metadataCache.getPartitionInfo(tp.topic(), tp.partition()).get.basePartitionState.leader) + broker2.metadataCache.getPartitionInfo(tp.topic, tp.partition).get.leader) } } diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index c267f04e8e11..42409db72c24 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -116,7 +116,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { createTopic(topic, numPartitions = 1, replicationFactor = 1) val version = ApiKeys.PRODUCE.latestVersion: Short - val serializedBytes = { + val (serializedBytes, headerVersion) = { val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, version, null, correlationId) val records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("message".getBytes)) @@ -124,13 +124,13 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val byteBuffer = ByteBuffer.allocate(headerBytes.length + request.toStruct.sizeOf) byteBuffer.put(headerBytes) request.toStruct.writeTo(byteBuffer) - byteBuffer.array() + (byteBuffer.array(), request.api.headerVersion(version)) } val response = requestAndReceive(serializedBytes) val responseBuffer = ByteBuffer.wrap(response) - val responseHeader = ResponseHeader.parse(responseBuffer) + val responseHeader = ResponseHeader.parse(responseBuffer, headerVersion) val produceResponse = ProduceResponse.parse(responseBuffer, version) assertEquals("The response should parse completely", 0, responseBuffer.remaining) diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index 0363963de7f5..daea62424043 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -34,6 +34,7 @@ import org.junit.Assert._ import org.junit.Test import scala.collection.JavaConverters._ +import scala.collection.Seq import scala.util.Random /** @@ -279,7 +280,7 @@ class FetchRequestTest extends BaseRequestTest { val socket = connect(brokerSocketServer(leaderId)) try { - send(fetchRequest, ApiKeys.FETCH, socket) + send(fetchRequest, ApiKeys.FETCH, socket, fetchRequest.api.headerVersion(fetchRequest.version())) if (closeAfterPartialResponse) { // read some data to ensure broker has muted this channel and then close socket val size = new DataInputStream(socket.getInputStream).readInt() @@ -290,7 +291,7 @@ class FetchRequestTest extends BaseRequestTest { size > maxPartitionBytes - batchSize) None } else { - Some(FetchResponse.parse(receive(socket), version)) + Some(FetchResponse.parse(receive(socket, ApiKeys.FETCH.headerVersion(version)), version)) } } finally { socket.close() @@ -556,7 +557,7 @@ class FetchRequestTest extends BaseRequestTest { } private def records(partitionData: FetchResponse.PartitionData[MemoryRecords]): Seq[Record] = { - partitionData.records.records.asScala.toIndexedSeq + partitionData.records.records.asScala.toBuffer } private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse[MemoryRecords], @@ -574,7 +575,7 @@ class FetchRequestTest extends BaseRequestTest { val records = partitionData.records responseBufferSize += records.sizeInBytes - val batches = records.batches.asScala.toIndexedSeq + val batches = records.batches.asScala.toBuffer assertTrue(batches.size < numMessagesPerPartition) val batchesSize = batches.map(_.sizeInBytes).sum responseSize += batchesSize diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 06fc534d2b78..48d21e29f26a 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -41,7 +41,6 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse -import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint} import org.apache.kafka.common.requests.WriteTxnMarkersRequest.TxnMarkerEntry import org.apache.kafka.common.requests.{FetchMetadata => JFetchMetadata, _} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} @@ -50,6 +49,7 @@ import EasyMock._ import org.apache.kafka.common.message.{HeartbeatRequestData, JoinGroupRequestData, OffsetCommitRequestData, OffsetCommitResponseData, SyncGroupRequestData, TxnOffsetCommitRequestData} import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity +import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.replica.ClientMetadata import org.apache.kafka.server.authorizer.Authorizer import org.junit.Assert.{assertEquals, assertNull, assertTrue} @@ -714,14 +714,34 @@ class KafkaApisTest { private def updateMetadataCacheWithInconsistentListeners(): (ListenerName, ListenerName) = { val plaintextListener = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) val anotherListener = new ListenerName("LISTENER2") - val brokers = Set( - new Broker(0, Seq(new EndPoint("broker0", 9092, SecurityProtocol.PLAINTEXT, plaintextListener), - new EndPoint("broker0", 9093, SecurityProtocol.PLAINTEXT, anotherListener)).asJava, "rack"), - new Broker(1, Seq(new EndPoint("broker1", 9092, SecurityProtocol.PLAINTEXT, plaintextListener)).asJava, - "rack") + val brokers = Seq( + new UpdateMetadataBroker() + .setId(0) + .setRack("rack") + .setEndpoints(Seq( + new UpdateMetadataEndpoint() + .setHost("broker0") + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(plaintextListener.value), + new UpdateMetadataEndpoint() + .setHost("broker0") + .setPort(9093) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(anotherListener.value) + ).asJava), + new UpdateMetadataBroker() + .setId(1) + .setRack("rack") + .setEndpoints(Seq( + new UpdateMetadataEndpoint() + .setHost("broker1") + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(plaintextListener.value)).asJava) ) val updateMetadataRequest = new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, 0, - 0, 0, Map.empty[TopicPartition, UpdateMetadataRequest.PartitionState].asJava, brokers.asJava).build() + 0, 0, Seq.empty[UpdateMetadataPartitionState].asJava, brokers.asJava).build() metadataCache.updateMetadata(correlationId = 0, updateMetadataRequest) (plaintextListener, anotherListener) } @@ -797,7 +817,7 @@ class KafkaApisTest { send.writeTo(channel) channel.close() channel.buffer.getInt() // read the size - ResponseHeader.parse(channel.buffer) + ResponseHeader.parse(channel.buffer, sendResponse.request.header.headerVersion()) val struct = api.responseSchema(request.version).read(channel.buffer) AbstractResponse.parseResponse(api, struct, request.version) } @@ -815,12 +835,29 @@ class KafkaApisTest { private def setupBasicMetadataCache(topic: String, numPartitions: Int): Unit = { val replicas = List(0.asInstanceOf[Integer]).asJava - val partitionState = new UpdateMetadataRequest.PartitionState(1, 0, 1, replicas, 0, replicas, Collections.emptyList()) + + def createPartitionState(partition: Int) = new UpdateMetadataPartitionState() + .setTopicName(topic) + .setPartitionIndex(partition) + .setControllerEpoch(1) + .setLeader(0) + .setLeaderEpoch(1) + .setReplicas(replicas) + .setZkVersion(0) + .setReplicas(replicas) + val plaintextListener = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) - val broker = new Broker(0, Seq(new EndPoint("broker0", 9092, SecurityProtocol.PLAINTEXT, plaintextListener)).asJava, "rack") - val partitions = (0 until numPartitions).map(new TopicPartition(topic, _) -> partitionState).toMap + val broker = new UpdateMetadataBroker() + .setId(0) + .setRack("rack") + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("broker0") + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(plaintextListener.value)).asJava) + val partitionStates = (0 until numPartitions).map(createPartitionState) val updateMetadataRequest = new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, 0, - 0, 0, partitions.asJava, Set(broker).asJava).build() + 0, 0, partitionStates.asJava, Seq(broker).asJava).build() metadataCache.updateMetadata(correlationId = 0, updateMetadataRequest) } } diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 6aeb355832cf..01423a39f61f 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -28,6 +28,7 @@ import kafka.cluster.Broker import kafka.controller.{ControllerChannelManager, ControllerContext, StateChangeLogger} import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -93,12 +94,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { servers.head.startup() //make sure second server joins the ISR TestUtils.waitUntilTrue(() => { - val partitionInfoOpt = servers.last.metadataCache.getPartitionInfo(topic, partitionId) - if (partitionInfoOpt.isDefined) { - partitionInfoOpt.get.basePartitionState.isr.size() == 2 - } else { - false - } + servers.last.metadataCache.getPartitionInfo(topic, partitionId).exists(_.isr.size == 2) }, "Inconsistent metadata after second broker startup") servers.last.shutdown() @@ -145,13 +141,21 @@ class LeaderElectionTest extends ZooKeeperTestHarness { controllerChannelManager.startup() try { val staleControllerEpoch = 0 - val partitionStates = Map( - new TopicPartition(topic, partitionId) -> new LeaderAndIsrRequest.PartitionState(2, brokerId2, LeaderAndIsr.initialLeaderEpoch, - Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion, - Seq(0, 1).map(Integer.valueOf).asJava, false) + val partitionStates = Seq( + new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(partitionId) + .setControllerEpoch(2) + .setLeader(brokerId2) + .setLeaderEpoch(LeaderAndIsr.initialLeaderEpoch) + .setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava) + .setZkVersion(LeaderAndIsr.initialZKVersion) + .setReplicas(Seq(0, 1).map(Integer.valueOf).asJava) + .setIsNew(false) ) val requestBuilder = new LeaderAndIsrRequest.Builder( - ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, staleControllerEpoch, servers(brokerId2).kafkaController.brokerEpoch ,partitionStates.asJava, nodes.toSet.asJava) + ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, staleControllerEpoch, + servers(brokerId2).kafkaController.brokerEpoch, partitionStates.asJava, nodes.toSet.asJava) controllerChannelManager.sendRequest(brokerId2, requestBuilder, staleControllerEpochCallback) TestUtils.waitUntilTrue(() => staleControllerEpochDetected, "Controller epoch should be stale") diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index 8d7b27a25d37..a42a86a5d03e 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -20,14 +20,14 @@ import java.util import java.util.Optional import util.Arrays.asList -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.UpdateMetadataRequest -import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint} import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.Test import org.junit.Assert._ +import org.scalatest.Assertions import scala.collection.JavaConverters._ @@ -53,22 +53,57 @@ class MetadataCacheTest { val controllerId = 2 val controllerEpoch = 1 - def endPoints(brokerId: Int): Seq[EndPoint] = { + def endpoints(brokerId: Int): Seq[UpdateMetadataEndpoint] = { val host = s"foo-$brokerId" Seq( - new EndPoint(host, 9092, SecurityProtocol.PLAINTEXT, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)), - new EndPoint(host, 9093, SecurityProtocol.SSL, ListenerName.forSecurityProtocol(SecurityProtocol.SSL)) + new UpdateMetadataEndpoint() + .setHost(host) + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT).value), + new UpdateMetadataEndpoint() + .setHost(host) + .setPort(9093) + .setSecurityProtocol(SecurityProtocol.SSL.id) + .setListener(ListenerName.forSecurityProtocol(SecurityProtocol.SSL).value) ) } val brokers = (0 to 4).map { brokerId => - new Broker(brokerId, endPoints(brokerId).asJava, "rack1") - }.toSet + new UpdateMetadataBroker() + .setId(brokerId) + .setEndpoints(endpoints(brokerId).asJava) + .setRack("rack1") + } - val partitionStates = Map( - new TopicPartition(topic0, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 0, 0, asList(0, 1, 3), zkVersion, asList(0, 1, 3), asList()), - new TopicPartition(topic0, 1) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 1, 1, asList(1, 0), zkVersion, asList(1, 2, 0, 4), asList()), - new TopicPartition(topic1, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 2, 2, asList(2, 1), zkVersion, asList(2, 1, 3), asList())) + val partitionStates = Seq( + new UpdateMetadataPartitionState() + .setTopicName(topic0) + .setPartitionIndex(0) + .setControllerEpoch(controllerEpoch) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(asList(0, 1, 3)) + .setZkVersion(zkVersion) + .setReplicas(asList(0, 1, 3)), + new UpdateMetadataPartitionState() + .setTopicName(topic0) + .setPartitionIndex(1) + .setControllerEpoch(controllerEpoch) + .setLeader(1) + .setLeaderEpoch(1) + .setIsr(asList(1, 0)) + .setZkVersion(zkVersion) + .setReplicas(asList(1, 2, 0, 4)), + new UpdateMetadataPartitionState() + .setTopicName(topic1) + .setPartitionIndex(0) + .setControllerEpoch(controllerEpoch) + .setLeader(2) + .setLeaderEpoch(2) + .setIsr(asList(2, 1)) + .setZkVersion(zkVersion) + .setReplicas(asList(2, 1, 3))) val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch, @@ -86,7 +121,7 @@ class MetadataCacheTest { assertEquals(Errors.NONE, topicMetadata.error) assertEquals(topic, topicMetadata.topic) - val topicPartitionStates = partitionStates.filter { case (tp, _) => tp.topic == topic } + val topicPartitionStates = partitionStates.filter { ps => ps.topicName == topic } val partitionMetadatas = topicMetadata.partitionMetadata.asScala.sortBy(_.partition) assertEquals(s"Unexpected partition count for topic $topic", topicPartitionStates.size, partitionMetadatas.size) @@ -94,14 +129,15 @@ class MetadataCacheTest { assertEquals(Errors.NONE, partitionMetadata.error) assertEquals(partitionId, partitionMetadata.partition) val leader = partitionMetadata.leader - val partitionState = topicPartitionStates(new TopicPartition(topic, partitionId)) - assertEquals(partitionState.basePartitionState.leader, leader.id) - assertEquals(Optional.of(partitionState.basePartitionState.leaderEpoch), partitionMetadata.leaderEpoch) - assertEquals(partitionState.basePartitionState.isr, partitionMetadata.isr.asScala.map(_.id).asJava) - assertEquals(partitionState.basePartitionState.replicas, partitionMetadata.replicas.asScala.map(_.id).asJava) - val endPoint = endPoints(partitionMetadata.leader.id).find(_.listenerName == listenerName).get - assertEquals(endPoint.host, leader.host) - assertEquals(endPoint.port, leader.port) + val partitionState = topicPartitionStates.find(_.partitionIndex == partitionId).getOrElse( + Assertions.fail(s"Unable to find partition state for partition $partitionId")) + assertEquals(partitionState.leader, leader.id) + assertEquals(Optional.of(partitionState.leaderEpoch), partitionMetadata.leaderEpoch) + assertEquals(partitionState.isr, partitionMetadata.isr.asScala.map(_.id).asJava) + assertEquals(partitionState.replicas, partitionMetadata.replicas.asScala.map(_.id).asJava) + val endpoint = endpoints(partitionMetadata.leader.id).find(_.listener == listenerName.value).get + assertEquals(endpoint.host, leader.host) + assertEquals(endpoint.port, leader.port) } } @@ -115,7 +151,13 @@ class MetadataCacheTest { def getTopicMetadataPartitionLeaderNotAvailable(): Unit = { val securityProtocol = SecurityProtocol.PLAINTEXT val listenerName = ListenerName.forSecurityProtocol(securityProtocol) - val brokers = Set(new Broker(0, Seq(new EndPoint("foo", 9092, securityProtocol, listenerName)).asJava, null)) + val brokers = Seq(new UpdateMetadataBroker() + .setId(0) + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("foo") + .setPort(9092) + .setSecurityProtocol(securityProtocol.id) + .setListener(listenerName.value)).asJava)) verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, listenerName, leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = false) verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, listenerName, @@ -127,10 +169,28 @@ class MetadataCacheTest { val plaintextListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) val sslListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.SSL) val broker0Endpoints = Seq( - new EndPoint("host0", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName), - new EndPoint("host0", 9093, SecurityProtocol.SSL, sslListenerName)) - val broker1Endpoints = Seq(new EndPoint("host1", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName)) - val brokers = Set(new Broker(0, broker0Endpoints.asJava, null), new Broker(1, broker1Endpoints.asJava, null)) + new UpdateMetadataEndpoint() + .setHost("host0") + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(plaintextListenerName.value), + new UpdateMetadataEndpoint() + .setHost("host0") + .setPort(9093) + .setSecurityProtocol(SecurityProtocol.SSL.id) + .setListener(sslListenerName.value)) + val broker1Endpoints = Seq(new UpdateMetadataEndpoint() + .setHost("host1") + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(plaintextListenerName.value)) + val brokers = Seq( + new UpdateMetadataBroker() + .setId(0) + .setEndpoints(broker0Endpoints.asJava), + new UpdateMetadataBroker() + .setId(1) + .setEndpoints(broker1Endpoints.asJava)) verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, sslListenerName, leader = 1, Errors.LISTENER_NOT_FOUND, errorUnavailableListeners = true) } @@ -140,15 +200,33 @@ class MetadataCacheTest { val plaintextListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) val sslListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.SSL) val broker0Endpoints = Seq( - new EndPoint("host0", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName), - new EndPoint("host0", 9093, SecurityProtocol.SSL, sslListenerName)) - val broker1Endpoints = Seq(new EndPoint("host1", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName)) - val brokers = Set(new Broker(0, broker0Endpoints.asJava, null), new Broker(1, broker1Endpoints.asJava, null)) + new UpdateMetadataEndpoint() + .setHost("host0") + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(plaintextListenerName.value), + new UpdateMetadataEndpoint() + .setHost("host0") + .setPort(9093) + .setSecurityProtocol(SecurityProtocol.SSL.id) + .setListener(sslListenerName.value)) + val broker1Endpoints = Seq(new UpdateMetadataEndpoint() + .setHost("host1") + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(plaintextListenerName.value)) + val brokers = Seq( + new UpdateMetadataBroker() + .setId(0) + .setEndpoints(broker0Endpoints.asJava), + new UpdateMetadataBroker() + .setId(1) + .setEndpoints(broker1Endpoints.asJava)) verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, sslListenerName, leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = false) } - private def verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers: Set[Broker], + private def verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers: Seq[UpdateMetadataBroker], listenerName: ListenerName, leader: Int, expectedError: Errors, @@ -162,8 +240,15 @@ class MetadataCacheTest { val controllerEpoch = 1 val leaderEpoch = 1 - val partitionStates = Map( - new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asList(0), asList())) + val partitionStates = Seq(new UpdateMetadataPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(asList(0)) + .setZkVersion(zkVersion) + .setReplicas(asList(0))) val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch, @@ -198,7 +283,13 @@ class MetadataCacheTest { val controllerEpoch = 1 val securityProtocol = SecurityProtocol.PLAINTEXT val listenerName = ListenerName.forSecurityProtocol(securityProtocol) - val brokers = Set(new Broker(0, Seq(new EndPoint("foo", 9092, securityProtocol, listenerName)).asJava, null)) + val brokers = Seq(new UpdateMetadataBroker() + .setId(0) + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("foo") + .setPort(9092) + .setSecurityProtocol(securityProtocol.id) + .setListener(listenerName.value)).asJava)) // replica 1 is not available val leader = 0 @@ -206,8 +297,16 @@ class MetadataCacheTest { val replicas = asList[Integer](0, 1) val isr = asList[Integer](0) - val partitionStates = Map( - new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList())) + val partitionStates = Seq( + new UpdateMetadataPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(zkVersion) + .setReplicas(replicas)) val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch, @@ -258,7 +357,14 @@ class MetadataCacheTest { val controllerEpoch = 1 val securityProtocol = SecurityProtocol.PLAINTEXT val listenerName = ListenerName.forSecurityProtocol(securityProtocol) - val brokers = Set(new Broker(0, Seq(new EndPoint("foo", 9092, securityProtocol, listenerName)).asJava, "rack1")) + val brokers = Seq(new UpdateMetadataBroker() + .setId(0) + .setRack("rack1") + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("foo") + .setPort(9092) + .setSecurityProtocol(securityProtocol.id) + .setListener(listenerName.value)).asJava)) // replica 1 is not available val leader = 0 @@ -266,8 +372,15 @@ class MetadataCacheTest { val replicas = asList[Integer](0) val isr = asList[Integer](0, 1) - val partitionStates = Map( - new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList())) + val partitionStates = Seq(new UpdateMetadataPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(zkVersion) + .setReplicas(replicas)) val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch, @@ -312,15 +425,28 @@ class MetadataCacheTest { val topic = "topic" val cache = new MetadataCache(1) val securityProtocol = SecurityProtocol.PLAINTEXT - val brokers = Set(new Broker(0, - Seq(new EndPoint("foo", 9092, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol))).asJava, "")) + val brokers = Seq(new UpdateMetadataBroker() + .setId(0) + .setRack("") + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("foo") + .setPort(9092) + .setSecurityProtocol(securityProtocol.id) + .setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava)) val controllerEpoch = 1 val leader = 0 val leaderEpoch = 0 val replicas = asList[Integer](0) val isr = asList[Integer](0, 1) - val partitionStates = Map( - new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList())) + val partitionStates = Seq(new UpdateMetadataPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(3) + .setReplicas(replicas)) val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava, brokers.asJava).build() @@ -337,31 +463,44 @@ class MetadataCacheTest { val topic = "topic" val cache = new MetadataCache(1) - def updateCache(brokerIds: Set[Int]): Unit = { + def updateCache(brokerIds: Seq[Int]): Unit = { val brokers = brokerIds.map { brokerId => val securityProtocol = SecurityProtocol.PLAINTEXT - new Broker(brokerId, Seq( - new EndPoint("foo", 9092, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol))).asJava, "") + new UpdateMetadataBroker() + .setId(brokerId) + .setRack("") + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("foo") + .setPort(9092) + .setSecurityProtocol(securityProtocol.id) + .setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava) } val controllerEpoch = 1 val leader = 0 val leaderEpoch = 0 val replicas = asList[Integer](0) val isr = asList[Integer](0, 1) - val partitionStates = Map( - new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList())) + val partitionStates = Seq(new UpdateMetadataPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(3) + .setReplicas(replicas)) val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava, brokers.asJava).build() cache.updateMetadata(15, updateMetadataRequest) } - val initialBrokerIds = (0 to 2).toSet + val initialBrokerIds = (0 to 2) updateCache(initialBrokerIds) val aliveBrokersFromCache = cache.getAliveBrokers // This should not change `aliveBrokersFromCache` - updateCache((0 to 3).toSet) - assertEquals(initialBrokerIds, aliveBrokersFromCache.map(_.id).toSet) + updateCache((0 to 3)) + assertEquals(initialBrokerIds.toSet, aliveBrokersFromCache.map(_.id).toSet) } } diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index c51444cbe9d0..2b3f32181908 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -36,6 +36,7 @@ import scala.collection.Seq class MetadataRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { + properties.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") properties.setProperty(KafkaConfig.DefaultReplicationFactorProp, "2") properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 857d06ecc50c..438dedf5b283 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -36,6 +36,7 @@ import kafka.utils.TestUtils.createBroker import kafka.utils.timer.MockTimer import kafka.utils.{MockScheduler, MockTime, TestUtils} import kafka.zk.KafkaZkClient +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record._ @@ -170,8 +171,16 @@ class ReplicaManagerTest { new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints)) // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, 0) -> - new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) rm.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true) @@ -184,8 +193,16 @@ class ReplicaManagerTest { // Make this replica the follower val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, 0) -> - new LeaderAndIsrRequest.PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(1) + .setLeaderEpoch(1) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ()) @@ -209,8 +226,16 @@ class ReplicaManagerTest { // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, 0) -> - new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(true)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true) @@ -261,8 +286,16 @@ class ReplicaManagerTest { // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, 0) -> - new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(true)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true) @@ -358,8 +391,16 @@ class ReplicaManagerTest { // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, 0) -> - new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(true)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) replicaManager.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true) @@ -425,8 +466,16 @@ class ReplicaManagerTest { // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, 0) -> - new LeaderAndIsrRequest.PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1), new Node(2, "host2", 2)).asJava).build() rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) rm.getPartitionOrException(new TopicPartition(topic, 0), expectLeader = true) @@ -469,10 +518,18 @@ class ReplicaManagerTest { val replicas = aliveBrokersIds.toList.map(Int.box).asJava // Broker 0 becomes leader of the partition - val leaderAndIsrPartitionState = new LeaderAndIsrRequest.PartitionState(0, 0, leaderEpoch, - replicas, 0, replicas, true) + val leaderAndIsrPartitionState = new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(leaderEpoch) + .setIsr(replicas) + .setZkVersion(0) + .setReplicas(replicas) + .setIsNew(true) val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - Map(tp -> leaderAndIsrPartitionState).asJava, + Seq(leaderAndIsrPartitionState).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() val leaderAndIsrResponse = replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) assertEquals(Errors.NONE, leaderAndIsrResponse.error) @@ -565,9 +622,27 @@ class ReplicaManagerTest { val partition0Replicas = Seq[Integer](0, 1).asJava val partition1Replicas = Seq[Integer](0, 2).asJava val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map( - tp0 -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, partition0Replicas, 0, partition0Replicas, true), - tp1 -> new LeaderAndIsrRequest.PartitionState(0, 0, 0, partition1Replicas, 0, partition1Replicas, true) + Seq( + new LeaderAndIsrPartitionState() + .setTopicName(tp0.topic) + .setPartitionIndex(tp0.partition) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(partition0Replicas) + .setZkVersion(0) + .setReplicas(partition0Replicas) + .setIsNew(true), + new LeaderAndIsrPartitionState() + .setTopicName(tp1.topic) + .setPartitionIndex(tp1.partition) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(partition1Replicas) + .setZkVersion(0) + .setReplicas(partition1Replicas) + .setIsNew(true) ).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) @@ -653,11 +728,12 @@ class ReplicaManagerTest { topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, expectTruncation = true) // Initialize partition state to follower, with leader = 1, leaderEpoch = 1 - val partition = replicaManager.createPartition(new TopicPartition(topic, topicPartition)) + val tp = new TopicPartition(topic, topicPartition) + val partition = replicaManager.createPartition(tp) val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints) partition.createLogIfNotExists(followerBrokerId, isNew = false, isFutureReplica = false, offsetCheckpoints) partition.makeFollower(controllerId, - leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds), + leaderAndIsrPartitionState(tp, leaderEpoch, leaderBrokerId, aliveBrokerIds), correlationId, offsetCheckpoints) // Make local partition a follower - because epoch increased by more than 1, truncation should @@ -665,8 +741,7 @@ class ReplicaManagerTest { leaderEpoch += leaderEpochIncrement val leaderAndIsrRequest0 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, controllerEpoch, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, topicPartition) -> - leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava, + Seq(leaderAndIsrPartitionState(tp, leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava, Set(new Node(followerBrokerId, "host1", 0), new Node(leaderBrokerId, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest0, @@ -693,13 +768,14 @@ class ReplicaManagerTest { topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, expectTruncation = true) - val partition = replicaManager.createPartition(new TopicPartition(topic, topicPartition)) + val tp = new TopicPartition(topic, topicPartition) + val partition = replicaManager.createPartition(tp) val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints) partition.createLogIfNotExists(leaderBrokerId, isNew = false, isFutureReplica = false, offsetCheckpoints) partition.makeLeader( controllerId, - leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds), + leaderAndIsrPartitionState(tp, leaderEpoch, leaderBrokerId, aliveBrokerIds), correlationId, offsetCheckpoints ) @@ -737,8 +813,16 @@ class ReplicaManagerTest { // Make this replica the follower val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, 0) -> - new LeaderAndIsrRequest.PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(1) + .setLeaderEpoch(1) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ()) @@ -778,8 +862,16 @@ class ReplicaManagerTest { // Make this replica the follower val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - collection.immutable.Map(new TopicPartition(topic, 0) -> - new LeaderAndIsrRequest.PartitionState(0, 0, 1, brokerList, 0, brokerList, false)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(1) + .setIsr(brokerList) + .setZkVersion(0) + .setReplicas(brokerList) + .setIsNew(false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ()) @@ -960,11 +1052,20 @@ class ReplicaManagerTest { (replicaManager, mockLogMgr) } - private def leaderAndIsrPartitionState(leaderEpoch: Int, + private def leaderAndIsrPartitionState(topicPartition: TopicPartition, + leaderEpoch: Int, leaderBrokerId: Int, - aliveBrokerIds: Seq[Integer]) : LeaderAndIsrRequest.PartitionState = { - new LeaderAndIsrRequest.PartitionState(controllerEpoch, leaderBrokerId, leaderEpoch, aliveBrokerIds.asJava, - zkVersion, aliveBrokerIds.asJava, false) + aliveBrokerIds: Seq[Integer]): LeaderAndIsrPartitionState = { + new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(topicPartition.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(leaderBrokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(aliveBrokerIds.asJava) + .setZkVersion(zkVersion) + .setReplicas(aliveBrokerIds.asJava) + .setIsNew(false) } private class CallbackResult[T] { @@ -1117,11 +1218,27 @@ class ReplicaManagerTest { val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, 0, brokerEpoch, - collection.immutable.Map( - tp0 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch, - partition0Replicas, 0, partition0Replicas, true), - tp1 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 1, leaderEpoch, - partition1Replicas, 0, partition1Replicas, true) + Seq( + new LeaderAndIsrPartitionState() + .setTopicName(tp0.topic) + .setPartitionIndex(tp0.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(0) + .setLeaderEpoch(leaderEpoch) + .setIsr(partition0Replicas) + .setZkVersion(0) + .setReplicas(partition0Replicas) + .setIsNew(true), + new LeaderAndIsrPartitionState() + .setTopicName(tp1.topic) + .setPartitionIndex(tp1.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(1) + .setLeaderEpoch(leaderEpoch) + .setIsr(partition1Replicas) + .setZkVersion(0) + .setReplicas(partition1Replicas) + .setIsNew(true) ).asJava, Set(new Node(0, "host0", 0), new Node(1, "host1", 1)).asJava).build() @@ -1131,11 +1248,27 @@ class ReplicaManagerTest { // make broker 0 the leader of partition 1 so broker 1 loses its leadership position val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, controllerEpoch, brokerEpoch, - collection.immutable.Map( - tp0 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch + leaderEpochIncrement, - partition0Replicas, 0, partition0Replicas, true), - tp1 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch + leaderEpochIncrement, - partition1Replicas, 0, partition1Replicas, true) + Seq( + new LeaderAndIsrPartitionState() + .setTopicName(tp0.topic) + .setPartitionIndex(tp0.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(0) + .setLeaderEpoch(leaderEpoch + leaderEpochIncrement) + .setIsr(partition0Replicas) + .setZkVersion(0) + .setReplicas(partition0Replicas) + .setIsNew(true), + new LeaderAndIsrPartitionState() + .setTopicName(tp1.topic) + .setPartitionIndex(tp1.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(0) + .setLeaderEpoch(leaderEpoch + leaderEpochIncrement) + .setIsr(partition1Replicas) + .setZkVersion(0) + .setReplicas(partition1Replicas) + .setIsNew(true) ).asJava, Set(new Node(0, "host0", 0), new Node(1, "host1", 1)).asJava).build() @@ -1173,11 +1306,27 @@ class ReplicaManagerTest { val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, 0, brokerEpoch, - collection.immutable.Map( - tp0 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 1, leaderEpoch, - partition0Replicas, 0, partition0Replicas, true), - tp1 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 1, leaderEpoch, - partition1Replicas, 0, partition1Replicas, true) + Seq( + new LeaderAndIsrPartitionState() + .setTopicName(tp0.topic) + .setPartitionIndex(tp0.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(1) + .setLeaderEpoch(leaderEpoch) + .setIsr(partition0Replicas) + .setZkVersion(0) + .setReplicas(partition0Replicas) + .setIsNew(true), + new LeaderAndIsrPartitionState() + .setTopicName(tp1.topic) + .setPartitionIndex(tp1.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(1) + .setLeaderEpoch(leaderEpoch) + .setIsr(partition1Replicas) + .setZkVersion(0) + .setReplicas(partition1Replicas) + .setIsNew(true) ).asJava, Set(new Node(0, "host0", 0), new Node(1, "host1", 1)).asJava).build() @@ -1187,11 +1336,27 @@ class ReplicaManagerTest { // make broker 0 the leader of partition 1 so broker 1 loses its leadership position val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, controllerEpoch, brokerEpoch, - collection.immutable.Map( - tp0 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch + leaderEpochIncrement, - partition0Replicas, 0, partition0Replicas, true), - tp1 -> new LeaderAndIsrRequest.PartitionState(controllerEpoch, 0, leaderEpoch + leaderEpochIncrement, - partition1Replicas, 0, partition1Replicas, true) + Seq( + new LeaderAndIsrPartitionState() + .setTopicName(tp0.topic) + .setPartitionIndex(tp0.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(0) + .setLeaderEpoch(leaderEpoch + leaderEpochIncrement) + .setIsr(partition0Replicas) + .setZkVersion(0) + .setReplicas(partition0Replicas) + .setIsNew(true), + new LeaderAndIsrPartitionState() + .setTopicName(tp1.topic) + .setPartitionIndex(tp1.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(0) + .setLeaderEpoch(leaderEpoch + leaderEpochIncrement) + .setIsr(partition1Replicas) + .setZkVersion(0) + .setReplicas(partition1Replicas) + .setIsNew(true) ).asJava, Set(new Node(0, "host0", 0), new Node(1, "host1", 1)).asJava).build() diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala index da9c5340cbd3..16c1ab7e2c70 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala @@ -32,7 +32,7 @@ class ReplicationQuotaManagerTest { private val metrics = new Metrics(new MetricConfig(), Collections.emptyList(), time) @After - def tearDown: Unit = { + def tearDown(): Unit = { metrics.close() } diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 4a93ef9f67a0..4e0540b4f058 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -27,7 +27,9 @@ import org.apache.kafka.common.acl._ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection +import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity +import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.message._ import org.apache.kafka.common.metrics.{KafkaMetric, Quota, Sensor} import org.apache.kafka.common.network.ListenerName @@ -227,20 +229,39 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.LEADER_AND_ISR => new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue, Long.MaxValue, - Map(tp -> new LeaderAndIsrRequest.PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, - 2, Seq(brokerId).asJava, true)).asJava, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setControllerEpoch(Int.MaxValue) + .setLeader(brokerId) + .setLeaderEpoch(Int.MaxValue) + .setIsr(List(brokerId).asJava) + .setZkVersion(2) + .setReplicas(Seq(brokerId).asJava) + .setIsNew(true)).asJava, Set(new Node(brokerId, "localhost", 0)).asJava) case ApiKeys.STOP_REPLICA => new StopReplicaRequest.Builder(ApiKeys.STOP_REPLICA.latestVersion, brokerId, Int.MaxValue, Long.MaxValue, true, Set(tp).asJava) case ApiKeys.UPDATE_METADATA => - val partitionState = Map(tp -> new UpdateMetadataRequest.PartitionState( - Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava + val partitionState = Seq(new UpdateMetadataPartitionState() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setControllerEpoch(Int.MaxValue) + .setLeader(brokerId) + .setLeaderEpoch(Int.MaxValue) + .setIsr(List(brokerId).asJava) + .setZkVersion(2) + .setReplicas(Seq(brokerId).asJava)).asJava val securityProtocol = SecurityProtocol.PLAINTEXT - val brokers = Set(new UpdateMetadataRequest.Broker(brokerId, - Seq(new UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol, - ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava + val brokers = Seq(new UpdateMetadataBroker() + .setId(brokerId) + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("localhost") + .setPort(0) + .setSecurityProtocol(securityProtocol.id) + .setListener(ListenerName.forSecurityProtocol(securityProtocol).value)).asJava)).asJava new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, brokerId, Int.MaxValue, Long.MaxValue, partitionState, brokers) case ApiKeys.CONTROLLED_SHUTDOWN => @@ -473,6 +494,17 @@ class RequestQuotaTest extends BaseRequestTest { new ListPartitionReassignmentsRequestData() ) + case ApiKeys.OFFSET_DELETE => + new OffsetDeleteRequest.Builder( + new OffsetDeleteRequestData() + .setGroupId("test-group") + .setTopics(new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection( + Collections.singletonList(new OffsetDeleteRequestData.OffsetDeleteRequestTopic() + .setName("test-topic") + .setPartitions(Collections.singletonList( + new OffsetDeleteRequestData.OffsetDeleteRequestPartition() + .setPartitionIndex(0)))).iterator()))) + case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) } @@ -578,6 +610,7 @@ class RequestQuotaTest extends BaseRequestTest { new IncrementalAlterConfigsResponse(response, ApiKeys.INCREMENTAL_ALTER_CONFIGS.latestVersion()).throttleTimeMs case ApiKeys.ALTER_PARTITION_REASSIGNMENTS => new AlterPartitionReassignmentsResponse(response).throttleTimeMs case ApiKeys.LIST_PARTITION_REASSIGNMENTS => new ListPartitionReassignmentsResponse(response).throttleTimeMs + case ApiKeys.OFFSET_DELETE => new OffsetDeleteResponse(response).throttleTimeMs() case requestId => throw new IllegalArgumentException(s"No throttle time for $requestId") } } diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index a5883ca0a102..33e456680038 100755 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -233,7 +233,7 @@ class ServerShutdownTest extends ZooKeeperTestHarness { // Initiate a sendRequest and wait until connection is established and one byte is received by the peer val requestBuilder = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, - controllerId, 1, 0L, Map.empty.asJava, brokerAndEpochs.keys.map(_.node(listenerName)).toSet.asJava) + controllerId, 1, 0L, Seq.empty.asJava, brokerAndEpochs.keys.map(_.node(listenerName)).toSet.asJava) controllerChannelManager.sendRequest(1, requestBuilder) receiveFuture.get(10, TimeUnit.SECONDS) diff --git a/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala b/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala index f246b253a256..b1b43d069c0d 100644 --- a/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala @@ -25,7 +25,6 @@ import org.junit.Assert._ import org.junit.Test import collection.JavaConverters._ - class StopReplicaRequestTest extends BaseRequestTest { override val logDirCount = 2 override val brokerCount: Int = 1 @@ -44,14 +43,16 @@ class StopReplicaRequestTest extends BaseRequestTest { val offlineDir = server.logManager.getLog(tp1).get.dir.getParent server.replicaManager.handleLogDirFailure(offlineDir, sendZkNotification = false) - for (i <- 1 to 2) { + for (_ <- 1 to 2) { val request1 = new StopReplicaRequest.Builder(1, server.config.brokerId, server.replicaManager.controllerEpoch, server.kafkaController.brokerEpoch, true, Set(tp0, tp1).asJava).build() val response1 = connectAndSend(request1, ApiKeys.STOP_REPLICA, controllerSocketServer) - val partitionErrors1 = StopReplicaResponse.parse(response1, request1.version).responses() - assertEquals(Errors.NONE, partitionErrors1.get(tp0)) - assertEquals(Errors.KAFKA_STORAGE_ERROR, partitionErrors1.get(tp1)) + val partitionErrors1 = StopReplicaResponse.parse(response1, request1.version).partitionErrors.asScala + assertEquals(Some(Errors.NONE.code), + partitionErrors1.find(pe => pe.topicName == tp0.topic && pe.partitionIndex == tp0.partition).map(_.errorCode)) + assertEquals(Some(Errors.KAFKA_STORAGE_ERROR.code), + partitionErrors1.find(pe => pe.topicName == tp1.topic && pe.partitionIndex == tp1.partition).map(_.errorCode)) } } diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala index 4bf8fcca8e10..26de772a62fd 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala @@ -231,10 +231,7 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { private def waitForEpochChangeTo(topic: String, partition: Int, epoch: Int): Unit = { TestUtils.waitUntilTrue(() => { - brokers(0).metadataCache.getPartitionInfo(topic, partition) match { - case Some(m) => m.basePartitionState.leaderEpoch == epoch - case None => false - } + brokers(0).metadataCache.getPartitionInfo(topic, partition).exists(_.leaderEpoch == epoch) }, "Epoch didn't change") } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index d28bd99c5538..f3ccd16afdec 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -910,16 +910,14 @@ object TestUtils extends Logging { def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = { var leader: Int = -1 - waitUntilTrue(() => - servers.foldLeft(true) { - (result, server) => - val partitionStateOpt = server.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic, partition) - partitionStateOpt match { - case None => false - case Some(partitionState) => - leader = partitionState.basePartitionState.leader - result && Request.isValidBrokerId(leader) - } + waitUntilTrue( + () => servers.forall { server => + server.dataPlaneRequestProcessor.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) if Request.isValidBrokerId(partitionState.leader) => + leader = partitionState.leader + true + case _ => false + } }, "Partition [%s,%d] metadata not propagated after %d ms".format(topic, partition, timeout), waitTimeMs = timeout) @@ -1432,11 +1430,12 @@ object TestUtils extends Logging { offsetsToCommit.toMap } - def resetToCommittedPositions(consumer: KafkaConsumer[Array[Byte], Array[Byte]]) = { + def resetToCommittedPositions(consumer: KafkaConsumer[Array[Byte], Array[Byte]]) { + val committed = consumer.committed(consumer.assignment).asScala.mapValues(_.offset) + consumer.assignment.asScala.foreach { topicPartition => - val offset = consumer.committed(topicPartition) - if (offset != null) - consumer.seek(topicPartition, offset.offset) + if (committed.contains(topicPartition)) + consumer.seek(topicPartition, committed(topicPartition)) else consumer.seekToBeginning(Collections.singletonList(topicPartition)) } diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index f9961d886bc2..135092dd8fa0 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -811,6 +811,27 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { expectedSuccessfulPartitions, successfulPartitions) } + @Test + def testTopicAssignments(): Unit = { + assertEquals(0, zkClient.getPartitionAssignmentForTopics(Set(topicPartition.topic())).size) + zkClient.createTopicAssignment(topicPartition.topic(), + Map(topicPartition -> Seq())) + + val expectedAssignment = PartitionReplicaAssignment(Seq(1,2,3), Seq(1), Seq(3)) + val response = zkClient.setTopicAssignmentRaw(topicPartition.topic(), + Map(topicPartition -> expectedAssignment), controllerEpochZkVersion) + assertEquals(Code.OK, response.resultCode) + + val topicPartitionAssignments = zkClient.getPartitionAssignmentForTopics(Set(topicPartition.topic())) + assertEquals(1, topicPartitionAssignments.size) + assertTrue(topicPartitionAssignments.contains(topicPartition.topic())) + val partitionAssignments = topicPartitionAssignments(topicPartition.topic()) + assertEquals(1, partitionAssignments.size) + assertTrue(partitionAssignments.contains(topicPartition.partition())) + val assignment = partitionAssignments(topicPartition.partition()) + assertEquals(expectedAssignment, assignment) + } + @Test def testUpdateLeaderAndIsr(): Unit = { zkClient.createRecursive(TopicZNode.path(topic1)) diff --git a/docs/security.html b/docs/security.html index b4c86546f080..8467bd658fd7 100644 --- a/docs/security.html +++ b/docs/security.html @@ -1052,13 +1052,15 @@
    Customizing SSL User N
    Customizing SASL User Name
    By default, the SASL user name will be the primary part of the Kerberos principal. One can change that by setting sasl.kerberos.principal.to.local.rules to a customized rule in server.properties. - The format of sasl.kerberos.principal.to.local.rules is a list where each rule works in the same way as the auth_to_local in Kerberos configuration file (krb5.conf). This also support additional lowercase rule, to force the translated result to be all lower case. This is done by adding a "/L" to the end of the rule. check below formats for syntax. + The format of sasl.kerberos.principal.to.local.rules is a list where each rule works in the same way as the auth_to_local in Kerberos configuration file (krb5.conf). This also support additional lowercase/uppercase rule, to force the translated result to be all lowercase/uppercase. This is done by adding a "/L" or "/U" to the end of the rule. check below formats for syntax. Each rules starts with RULE: and contains an expression as the following formats. See the kerberos documentation for more details.
             RULE:[n:string](regexp)s/pattern/replacement/
             RULE:[n:string](regexp)s/pattern/replacement/g
             RULE:[n:string](regexp)s/pattern/replacement//L
             RULE:[n:string](regexp)s/pattern/replacement/g/L
    +        RULE:[n:string](regexp)s/pattern/replacement//U
    +        RULE:[n:string](regexp)s/pattern/replacement/g/U
         
    An example of adding a rule to properly translate user@MYDOMAIN.COM to user while also keeping the default rule in place is: diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html index 474cac9bb021..4ee23a249417 100644 --- a/docs/streams/core-concepts.html +++ b/docs/streams/core-concepts.html @@ -49,7 +49,7 @@

    Core Concepts

  • Has no external dependencies on systems other than Apache Kafka itself as the internal messaging layer; notably, it uses Kafka's partitioning model to horizontally scale processing while maintaining strong ordering guarantees.
  • Supports fault-tolerant local state, which enables very fast and efficient stateful operations like windowed joins and aggregations.
  • Supports exactly-once processing semantics to guarantee that each record will be processed once and only once even when there is a failure on either Streams clients or Kafka brokers in the middle of processing.
  • -
  • Employs one-record-at-a-time processing to achieve millisecond processing latency, and supports event-time based windowing operations with late arrival of records.
  • +
  • Employs one-record-at-a-time processing to achieve millisecond processing latency, and supports event-time based windowing operations with out-of-order arrival of records.
  • Offers necessary stream processing primitives, along with a high-level Streams DSL and a low-level Processor API.
  • @@ -124,7 +124,7 @@

    Time

    • When new output records are generated via processing some input record, for example, context.forward() triggered in the process() function call, output record timestamps are inherited from input record timestamps directly.
    • When new output records are generated via periodic functions such as Punctuator#punctuate(), the output record timestamp is defined as the current internal time (obtained through context.timestamp()) of the stream task.
    • -
    • For aggregations, the timestamp of a resulting aggregate update record will be that of the latest arrived input record that triggered the update.
    • +
    • For aggregations, the timestamp of a result update record will be the maximum timestamp of all input records contributing to the result.

    @@ -137,7 +137,7 @@

    - In the Kafka Streams DSL, an input stream of an aggregation can be a KStream or a KTable, but the output stream will always be a KTable. This allows Kafka Streams to update an aggregate value upon the late arrival of further records after the value was produced and emitted. When such late arrival happens, the aggregating KStream or KTable emits a new aggregate value. Because the output is a KTable, the new value is considered to overwrite the old value with the same key in subsequent processing steps. + In the Kafka Streams DSL, an input stream of an aggregation can be a KStream or a KTable, but the output stream will always be a KTable. This allows Kafka Streams to update an aggregate value upon the out-of-order arrival of further records after the value was produced and emitted. When such out-of-order arrival happens, the aggregating KStream or KTable emits a new aggregate value. Because the output is a KTable, the new value is considered to overwrite the old value with the same key in subsequent processing steps.

    Windowing

    @@ -145,10 +145,10 @@

    Windo Windowing lets you control how to group records that have the same key for stateful operations such as aggregations or joins into so-called windows. Windows are tracked per record key.

    - Windowing operations are available in the Kafka Streams DSL. When working with windows, you can specify a retention period for the window. This retention period controls how long Kafka Streams will wait for out-of-order or late-arriving data records for a given window. If a record arrives after the retention period of a window has passed, the record is discarded and will not be processed in that window. + Windowing operations are available in the Kafka Streams DSL. When working with windows, you can specify a grace period for the window. This grace period controls how long Kafka Streams will wait for out-of-order data records for a given window. If a record arrives after the grace period of a window has passed, the record is discarded and will not be processed in that window. Specifically, a record is discarded if its timestamp dictates it belongs to a window, but the current stream time is greater than the end of the window plus the grace period.

    - Late-arriving records are always possible in the real world and should be properly accounted for in your applications. It depends on the effective time semantics how late records are handled. In the case of processing-time, the semantics are "when the record is being processed", which means that the notion of late records is not applicable as, by definition, no record can be late. Hence, late-arriving records can only be considered as such (i.e. as arriving "late") for event-time or ingestion-time semantics. In both cases, Kafka Streams is able to properly handle late-arriving records. + Out-of-order records are always possible in the real world and should be properly accounted for in your applications. It depends on the effective time semantics how out-of-order records are handled. In the case of processing-time, the semantics are "when the record is being processed", which means that the notion of out-of-order records is not applicable as, by definition, no record can be out-of-order. Hence, out-of-order records can only be considered as such for event-time. In both cases, Kafka Streams is able to properly handle out-of-order records.

    Duality of Streams and Tables

    diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html index 21f488b54919..221e3c0f89cb 100644 --- a/docs/streams/developer-guide/config-streams.html +++ b/docs/streams/developer-guide/config-streams.html @@ -474,7 +474,9 @@

    state.dir
    The state directory. Kafka Streams persists local states under the state directory. Each application has a subdirectory on its hosting machine that is located under the state directory. The name of the subdirectory is the application ID. The state stores associated - with the application are created under this subdirectory.
    + with the application are created under this subdirectory. When running multiple instances of the same application on a single machine, + this path must be unique for each such instance. +
    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/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index f5c3df95eff1..c99b03e6c504 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -335,7 +335,7 @@

    Some KStream transformations may generate one or more KStream objects, for example: - filter and map on a KStream will generate another KStream - branch on KStream can generate multiple KStreams

    -

    Some others may generate a KTable object, for example an aggregation of a KStream also yields a KTable. This allows Kafka Streams to continuously update the computed value upon arrivals of late records after it +

    Some others may generate a KTable object, for example an aggregation of a KStream also yields a KTable. This allows Kafka Streams to continuously update the computed value upon arrivals of out-of-order records after it has already been produced to the downstream transformation operators.

    All KTable transformation operations can only generate another KTable. However, the Kafka Streams DSL does provide a special function that converts a KTable representation into a KStream. All of these transformation methods can be chained together to compose @@ -2961,14 +2961,14 @@

    of time in Kafka Streams is milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to milliseconds (e.g. t=5 would become t=300,000).

    -

    If we then receive three additional records (including two late-arriving records), what would happen is that the two +

    If we then receive three additional records (including two out-of-order records), what would happen is that the two existing sessions for the green record key will be merged into a single session starting at time 0 and ending at time 6, consisting of a total of three records. The existing session for the blue record key will be extended to end at time 5, consisting of a total of two records. And, finally, there will be a new session for the blue key starting and ending at time 11.

    -

    Detected sessions after having received six input records. Note the two late-arriving data records at t=4 (green) and +

    Detected sessions after having received six input records. Note the two out-of-order data records at t=4 (green) and t=5 (blue), which lead to a merge of sessions and an extension of a session, respectively.

    @@ -3007,7 +3007,7 @@

    grace(ofMinutes(10))
    This allows us to bound the lateness of events the window will accept. - For example, the 09:00 to 10:00 window will accept late-arriving records until 10:10, at which point, the window is closed. + For example, the 09:00 to 10:00 window will accept out-of-order records until 10:10, at which point, the window is closed.
    .suppress(Suppressed.untilWindowCloses(...))
    This configures the suppression operator to emit nothing for a window until it closes, and then emit the final result. diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 0d10442230e0..bad37c70541f 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -53,8 +53,10 @@

    Upgrade Guide and API Changes

    - Note, that a brokers must be on version 0.10.1 or higher to run a Kafka Streams application version 0.10.1 or higher; - On-disk message format must be 0.10 or higher to run a Kafka Streams application version 1.0 or higher. + To run a Kafka Streams application version 2.2.1, 2.3.0, or higher a broker version 0.11.0 or higher is required + and the on-disk message format must be 0.11 or higher. + Brokers must be on version 0.10.1 or higher to run a Kafka Streams application version 0.10.1 to 2.2.0. + Additionally, on-disk message format must be 0.10 or higher to run a Kafka Streams application version 1.0 to 2.2.0. For Kafka Streams 0.10.0, broker version 0.10.0 or higher is required.

    @@ -139,6 +141,12 @@

    Streams API For more details please read KAFKA-8215.

    +

    Notable changes in Kafka Streams 2.2.1

    +

    + As of Kafka Streams 2.2.1 a message format 0.11 or higher is required; + this implies that brokers must be on version 0.11.0 or higher. +

    +

    Streams API changes in 2.2.0

    We've simplified the KafkaStreams#state transition diagram during the starting up phase a bit in 2.2.0: in older versions the state will transit from CREATED to RUNNING, and then to REBALANCING to get the first diff --git a/docs/upgrade.html b/docs/upgrade.html index f95adfcf00fd..0c69336127ca 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -40,6 +40,11 @@

    Notable changes in 2 it may result in uneven distribution of records across partitions in edge cases. Generally users will not be impacted, but this difference may be noticeable in tests and other situations producing records for a very short amount of time. +
  • The blocking KafkaConsumer#committed methods have been extended to allow a list of partitions as input parameters rather than a single partition. + It enables fewer request/response iterations between clients and brokers fetching for the committed offsets for the consumer group. + The old overloaded functions are deprecated and we would recommend users to make their code changes to leverage the new methods (details + can be found in KIP-520). +
  • Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x to 2.3.0

    @@ -99,6 +104,7 @@
    Notable changes in 2
  • Kafka Streams DSL switches its used store types. While this change is mainly transparent to users, there are some corner cases that may require code changes. See the Kafka Streams upgrade section for more details.
  • +
  • Kafka Streams 2.3.0 requires 0.11 message format or higher and does not work with older message format.
  • Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x to 2.2.0

    @@ -143,6 +149,11 @@

    Upgrading from 0.8.x, 0.9.x, 0.1 +
    Notable changes in 2.2.1
    +
      +
    • Kafka Streams 2.2.1 requires 0.11 message format or higher and does not work with older message format.
    • +
    +

    Notable changes in 2.2.0
    • The default consumer group id has been changed from the empty string ("") to null. Consumers who use the new default group id will not be able to subscribe to topics, diff --git a/generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java b/generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java index f5d00a697008..5a2cf117fea5 100644 --- a/generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java @@ -67,12 +67,16 @@ String responseSchema() { } } - public ApiMessageTypeGenerator() { - this.headerGenerator = new HeaderGenerator(); + public ApiMessageTypeGenerator(String packageName) { + this.headerGenerator = new HeaderGenerator(packageName); this.apis = new TreeMap<>(); this.buffer = new CodeBuffer(); } + public boolean hasRegisteredTypes() { + return !apis.isEmpty(); + } + public void registerMessageType(MessageSpec spec) { switch (spec.type()) { case REQUEST: { diff --git a/generator/src/main/java/org/apache/kafka/message/ClauseGenerator.java b/generator/src/main/java/org/apache/kafka/message/ClauseGenerator.java new file mode 100644 index 000000000000..5ef55f26e383 --- /dev/null +++ b/generator/src/main/java/org/apache/kafka/message/ClauseGenerator.java @@ -0,0 +1,25 @@ +/* + * 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.message; + +/** + * Generates a clause. + */ +public interface ClauseGenerator { + void generate(Versions versions); +} diff --git a/generator/src/main/java/org/apache/kafka/message/FieldType.java b/generator/src/main/java/org/apache/kafka/message/FieldType.java index d98b8f39c050..b14449ba9d97 100644 --- a/generator/src/main/java/org/apache/kafka/message/FieldType.java +++ b/generator/src/main/java/org/apache/kafka/message/FieldType.java @@ -97,10 +97,30 @@ public String toString() { } } + final class UUIDFieldType implements FieldType { + static final UUIDFieldType INSTANCE = new UUIDFieldType(); + private static final String NAME = "uuid"; + + @Override + public Optional fixedLength() { + return Optional.of(16); + } + + @Override + public String toString() { + return NAME; + } + } + final class StringFieldType implements FieldType { static final StringFieldType INSTANCE = new StringFieldType(); private static final String NAME = "string"; + @Override + public boolean serializationIsDifferentInFlexibleVersions() { + return true; + } + @Override public boolean isString() { return true; @@ -121,6 +141,11 @@ final class BytesFieldType implements FieldType { static final BytesFieldType INSTANCE = new BytesFieldType(); private static final String NAME = "bytes"; + @Override + public boolean serializationIsDifferentInFlexibleVersions() { + return true; + } + @Override public boolean isBytes() { return true; @@ -144,6 +169,11 @@ final class StructType implements FieldType { this.type = type; } + @Override + public boolean serializationIsDifferentInFlexibleVersions() { + return true; + } + @Override public boolean isStruct() { return true; @@ -162,6 +192,11 @@ final class ArrayType implements FieldType { this.elementType = elementType; } + @Override + public boolean serializationIsDifferentInFlexibleVersions() { + return true; + } + @Override public boolean isArray() { return true; @@ -204,6 +239,8 @@ static FieldType parse(String string) { return Int32FieldType.INSTANCE; case Int64FieldType.NAME: return Int64FieldType.INSTANCE; + case UUIDFieldType.NAME: + return UUIDFieldType.INSTANCE; case StringFieldType.NAME: return StringFieldType.INSTANCE; case BytesFieldType.NAME: @@ -243,6 +280,13 @@ default boolean isStructArray() { return false; } + /** + * Returns true if the serialization of this type is different in flexible versions. + */ + default boolean serializationIsDifferentInFlexibleVersions() { + return false; + } + /** * Returns true if this is a string type. */ @@ -278,6 +322,10 @@ default Optional fixedLength() { return Optional.empty(); } + default boolean isVariableLength() { + return !fixedLength().isPresent(); + } + /** * Convert the field type to a JSON string. */ diff --git a/generator/src/main/java/org/apache/kafka/message/HeaderGenerator.java b/generator/src/main/java/org/apache/kafka/message/HeaderGenerator.java index 6de9736af818..e1b9c1293560 100644 --- a/generator/src/main/java/org/apache/kafka/message/HeaderGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/HeaderGenerator.java @@ -17,6 +17,7 @@ package org.apache.kafka.message; +import java.util.Objects; import java.util.TreeSet; /** @@ -45,31 +46,46 @@ public final class HeaderGenerator { "" }; - static final String PACKAGE = "org.apache.kafka.common.message"; private final CodeBuffer buffer; private final TreeSet imports; + private final String packageName; - public HeaderGenerator() { + private final TreeSet staticImports; + + public HeaderGenerator(String packageName) { this.buffer = new CodeBuffer(); this.imports = new TreeSet<>(); + this.packageName = packageName; + this.staticImports = new TreeSet<>(); } public void addImport(String newImport) { this.imports.add(newImport); } + public void addStaticImport(String newImport) { + this.staticImports.add(newImport); + } + public void generate() { + Objects.requireNonNull(packageName); for (int i = 0; i < HEADER.length; i++) { buffer.printf("%s%n", HEADER[i]); } - buffer.printf("package %s;%n", PACKAGE); + buffer.printf("package %s;%n", packageName); buffer.printf("%n"); for (String newImport : imports) { buffer.printf("import %s;%n", newImport); } buffer.printf("%n"); + if (!staticImports.isEmpty()) { + for (String newImport : staticImports) { + buffer.printf("import static %s;%n", newImport); + } + buffer.printf("%n"); + } } public CodeBuffer buffer() { diff --git a/generator/src/main/java/org/apache/kafka/message/IsNullConditional.java b/generator/src/main/java/org/apache/kafka/message/IsNullConditional.java new file mode 100644 index 000000000000..5ff51dacc665 --- /dev/null +++ b/generator/src/main/java/org/apache/kafka/message/IsNullConditional.java @@ -0,0 +1,105 @@ +/* + * 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.message; + +/** + * Creates an if statement based on whether or not a particular field is null. + */ +public final class IsNullConditional { + static IsNullConditional forName(String name) { + return new IsNullConditional(name); + } + + static IsNullConditional forField(FieldSpec field) { + IsNullConditional cond = new IsNullConditional(field.camelCaseName()); + cond.nullableVersions(field.nullableVersions()); + return cond; + } + + private final String name; + private Versions nullableVersions = Versions.ALL; + private Versions possibleVersions = Versions.ALL; + private Runnable ifNull = null; + private Runnable ifNotNull = null; + private boolean alwaysEmitBlockScope = false; + + private IsNullConditional(String name) { + this.name = name; + } + + IsNullConditional nullableVersions(Versions nullableVersions) { + this.nullableVersions = nullableVersions; + return this; + } + + IsNullConditional possibleVersions(Versions possibleVersions) { + this.possibleVersions = possibleVersions; + return this; + } + + IsNullConditional ifNull(Runnable ifNull) { + this.ifNull = ifNull; + return this; + } + + IsNullConditional ifNotNull(Runnable ifNotNull) { + this.ifNotNull = ifNotNull; + return this; + } + + IsNullConditional alwaysEmitBlockScope(boolean alwaysEmitBlockScope) { + this.alwaysEmitBlockScope = alwaysEmitBlockScope; + return this; + } + + void generate(CodeBuffer buffer) { + if (nullableVersions.intersect(possibleVersions).empty()) { + if (ifNotNull != null) { + if (alwaysEmitBlockScope) { + buffer.printf("{%n"); + buffer.incrementIndent(); + } + ifNotNull.run(); + if (alwaysEmitBlockScope) { + buffer.decrementIndent(); + buffer.printf("}%n"); + } + } + } else { + if (ifNull != null) { + buffer.printf("if (%s == null) {%n", name); + buffer.incrementIndent(); + ifNull.run(); + buffer.decrementIndent(); + if (ifNotNull != null) { + buffer.printf("} else {%n"); + buffer.incrementIndent(); + ifNotNull.run(); + buffer.decrementIndent(); + } + buffer.printf("}%n"); + } else if (ifNotNull != null) { + buffer.printf("if (%s != null) {%n", name); + buffer.incrementIndent(); + ifNotNull.run(); + buffer.decrementIndent(); + buffer.printf("}%n"); + } + } + } +} diff --git a/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java b/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java index d0e5044f1f29..848af660eeda 100644 --- a/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.stream.Collectors; /** @@ -33,9 +34,9 @@ public final class MessageDataGenerator { private final SchemaGenerator schemaGenerator; private final CodeBuffer buffer; - MessageDataGenerator() { + MessageDataGenerator(String packageName) { this.structRegistry = new StructRegistry(); - this.headerGenerator = new HeaderGenerator(); + this.headerGenerator = new HeaderGenerator(packageName); this.schemaGenerator = new SchemaGenerator(headerGenerator, structRegistry); this.buffer = new CodeBuffer(); } @@ -311,6 +312,8 @@ private String fieldAbstractJavaType(FieldSpec field) { return "int"; } else if (field.type() instanceof FieldType.Int64FieldType) { return "long"; + } else if (field.type() instanceof FieldType.UUIDFieldType) { + return "UUID"; } else if (field.type().isString()) { return "String"; } else if (field.type().isBytes()) { @@ -475,6 +478,8 @@ private String readFieldFromReadable(FieldType type) { return "readable.readInt()"; } else if (type instanceof FieldType.Int64FieldType) { return "readable.readLong()"; + } else if (type instanceof FieldType.UUIDFieldType) { + return "readable.readUUID()"; } else if (type.isString()) { return "readable.readNullableString()"; } else if (type.isBytes()) { @@ -581,6 +586,8 @@ private String getBoxedJavaType(FieldType type) { return "Integer"; } else if (type instanceof FieldType.Int64FieldType) { return "Long"; + } else if (type instanceof FieldType.UUIDFieldType) { + return "UUID"; } else if (type.isString()) { return "String"; } else if (type.isStruct()) { @@ -601,6 +608,8 @@ private String readFieldFromStruct(FieldType type, String name) { return String.format("struct.getInt(\"%s\")", name); } else if (type instanceof FieldType.Int64FieldType) { return String.format("struct.getLong(\"%s\")", name); + } else if (type instanceof FieldType.UUIDFieldType) { + return String.format("struct.getUUID(\"%s\")", name); } else if (type.isString()) { return String.format("struct.getString(\"%s\")", name); } else if (type.isBytes()) { @@ -649,6 +658,8 @@ private String writeFieldToWritable(FieldType type, boolean nullable, String nam return String.format("writable.writeInt(%s)", name); } else if (type instanceof FieldType.Int64FieldType) { return String.format("writable.writeLong(%s)", name); + } else if (type instanceof FieldType.UUIDFieldType) { + return String.format("writable.writeUUID(%s)", name); } else if (type instanceof FieldType.StringFieldType) { if (nullable) { return String.format("writable.writeNullableString(%s)", name); @@ -750,6 +761,7 @@ private void generateFieldToStruct(FieldSpec field, Versions curVersions) { (field.type() instanceof FieldType.Int16FieldType) || (field.type() instanceof FieldType.Int32FieldType) || (field.type() instanceof FieldType.Int64FieldType) || + (field.type() instanceof FieldType.UUIDFieldType) || (field.type() instanceof FieldType.StringFieldType)) { boolean maybeAbsent = generateVersionCheck(curVersions, field.versions()); @@ -1039,16 +1051,17 @@ private void generateFieldHashCode(FieldSpec field) { } else if (field.type() instanceof FieldType.Int64FieldType) { buffer.printf("hashCode = 31 * hashCode + ((int) (%s >> 32) ^ (int) %s);%n", field.camelCaseName(), field.camelCaseName()); - } else if (field.type().isString()) { - buffer.printf("hashCode = 31 * hashCode + (%s == null ? 0 : %s.hashCode());%n", - field.camelCaseName(), field.camelCaseName()); } else if (field.type().isBytes()) { headerGenerator.addImport(MessageGenerator.ARRAYS_CLASS); buffer.printf("hashCode = 31 * hashCode + Arrays.hashCode(%s);%n", field.camelCaseName()); - } else if (field.type().isStruct() || field.type().isArray()) { + } else if (field.type().isStruct() + || field.type().isArray() + || field.type().isString() + || field.type() instanceof FieldType.UUIDFieldType + ) { buffer.printf("hashCode = 31 * hashCode + (%s == null ? 0 : %s.hashCode());%n", - field.camelCaseName(), field.camelCaseName()); + field.camelCaseName(), field.camelCaseName()); } else { throw new RuntimeException("Unsupported field type " + field.type()); } @@ -1078,7 +1091,8 @@ private void generateFieldToString(String prefix, FieldSpec field) { } else if ((field.type() instanceof FieldType.Int8FieldType) || (field.type() instanceof FieldType.Int16FieldType) || (field.type() instanceof FieldType.Int32FieldType) || - (field.type() instanceof FieldType.Int64FieldType)) { + (field.type() instanceof FieldType.Int64FieldType) || + (field.type() instanceof FieldType.UUIDFieldType)) { buffer.printf("+ \"%s%s=\" + %s%n", prefix, field.camelCaseName(), field.camelCaseName()); } else if (field.type().isString()) { @@ -1227,6 +1241,19 @@ private String fieldDefault(FieldSpec field) { } return field.defaultString() + "L"; } + } else if (field.type() instanceof FieldType.UUIDFieldType) { + headerGenerator.addImport(MessageGenerator.UUID_CLASS); + if (field.defaultString().isEmpty()) { + return "org.apache.kafka.common.protocol.MessageUtil.ZERO_UUID"; + } else { + try { + UUID.fromString(field.defaultString()); + } catch (IllegalArgumentException e) { + throw new RuntimeException("Invalid default for uuid field " + + field.name() + ": " + field.defaultString(), e); + } + return "UUID.fromString(\"" + field.defaultString() + "\")"; + } } else if (field.type() instanceof FieldType.StringFieldType) { if (field.defaultString().equals("null")) { validateNullDefault(field); diff --git a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java index e227ded0572c..9b5d9ed2da5c 100644 --- a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java @@ -79,6 +79,8 @@ public final class MessageGenerator { static final String BYTES_CLASS = "org.apache.kafka.common.utils.Bytes"; + static final String UUID_CLASS = "java.util.UUID"; + static final String REQUEST_SUFFIX = "Request"; static final String RESPONSE_SUFFIX = "Response"; @@ -96,10 +98,10 @@ public final class MessageGenerator { JSON_SERDE.setSerializationInclusion(JsonInclude.Include.NON_EMPTY); } - public static void processDirectories(String outputDir, String inputDir) throws Exception { + public static void processDirectories(String packageName, String outputDir, String inputDir) throws Exception { Files.createDirectories(Paths.get(outputDir)); int numProcessed = 0; - ApiMessageTypeGenerator messageTypeGenerator = new ApiMessageTypeGenerator(); + ApiMessageTypeGenerator messageTypeGenerator = new ApiMessageTypeGenerator(packageName); HashSet outputFileNames = new HashSet<>(); try (DirectoryStream directoryStream = Files .newDirectoryStream(Paths.get(inputDir), JSON_GLOB)) { @@ -111,7 +113,7 @@ public static void processDirectories(String outputDir, String inputDir) throws outputFileNames.add(javaName); Path outputPath = Paths.get(outputDir, javaName); try (BufferedWriter writer = Files.newBufferedWriter(outputPath)) { - MessageDataGenerator generator = new MessageDataGenerator(); + MessageDataGenerator generator = new MessageDataGenerator(packageName); generator.generate(spec); generator.write(writer); } @@ -122,13 +124,15 @@ public static void processDirectories(String outputDir, String inputDir) throws } } } - Path factoryOutputPath = Paths.get(outputDir, API_MESSAGE_TYPE_JAVA); - outputFileNames.add(API_MESSAGE_TYPE_JAVA); - try (BufferedWriter writer = Files.newBufferedWriter(factoryOutputPath)) { - messageTypeGenerator.generate(); - messageTypeGenerator.write(writer); + if (messageTypeGenerator.hasRegisteredTypes()) { + Path factoryOutputPath = Paths.get(outputDir, API_MESSAGE_TYPE_JAVA); + outputFileNames.add(API_MESSAGE_TYPE_JAVA); + try (BufferedWriter writer = Files.newBufferedWriter(factoryOutputPath)) { + messageTypeGenerator.generate(); + messageTypeGenerator.write(writer); + } + numProcessed++; } - numProcessed++; try (DirectoryStream directoryStream = Files. newDirectoryStream(Paths.get(outputDir))) { for (Path outputPath : directoryStream) { @@ -194,16 +198,16 @@ static String stripSuffix(String str, String suffix) { } } - private final static String USAGE = "MessageGenerator: [output Java file] [input JSON file]"; + private final static String USAGE = "MessageGenerator: [output Java package] [output Java file] [input JSON file]"; public static void main(String[] args) throws Exception { if (args.length == 0) { System.out.println(USAGE); System.exit(0); - } else if (args.length != 2) { + } else if (args.length != 3) { System.out.println(USAGE); System.exit(1); } - processDirectories(args[0], args[1]); + processDirectories(args[0], args[1], args[2]); } } diff --git a/generator/src/main/java/org/apache/kafka/message/SchemaGenerator.java b/generator/src/main/java/org/apache/kafka/message/SchemaGenerator.java index 2ef386867f7e..3b04b7b11a73 100644 --- a/generator/src/main/java/org/apache/kafka/message/SchemaGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/SchemaGenerator.java @@ -188,6 +188,12 @@ private String fieldTypeToSchemaType(FieldType type, boolean nullable, short ver throw new RuntimeException("Type " + type + " cannot be nullable."); } return "Type.INT64"; + } else if (type instanceof FieldType.UUIDFieldType) { + headerGenerator.addImport(MessageGenerator.TYPE_CLASS); + if (nullable) { + throw new RuntimeException("Type " + type + " cannot be nullable."); + } + return "Type.UUID"; } else if (type instanceof FieldType.StringFieldType) { headerGenerator.addImport(MessageGenerator.TYPE_CLASS); return nullable ? "Type.NULLABLE_STRING" : "Type.STRING"; diff --git a/generator/src/main/java/org/apache/kafka/message/VersionConditional.java b/generator/src/main/java/org/apache/kafka/message/VersionConditional.java new file mode 100644 index 000000000000..51b6cb047a0f --- /dev/null +++ b/generator/src/main/java/org/apache/kafka/message/VersionConditional.java @@ -0,0 +1,220 @@ +/* + * 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.message; + +/** + * Creates an if statement based on whether or not the current version + * falls within a given range. + */ +public final class VersionConditional { + /** + * Create a version conditional. + * + * @param containingVersions The versions for which the conditional is true. + * @param possibleVersions The range of possible versions. + * @return The version conditional. + */ + static VersionConditional forVersions(Versions containingVersions, + Versions possibleVersions) { + return new VersionConditional(containingVersions, possibleVersions); + } + + private final Versions containingVersions; + private final Versions possibleVersions; + private ClauseGenerator ifMember = null; + private ClauseGenerator ifNotMember = null; + private boolean alwaysEmitBlockScope = false; + private boolean allowMembershipCheckAlwaysFalse = true; + + private VersionConditional(Versions containingVersions, Versions possibleVersions) { + this.containingVersions = containingVersions; + this.possibleVersions = possibleVersions; + } + + VersionConditional ifMember(ClauseGenerator ifMember) { + this.ifMember = ifMember; + return this; + } + + VersionConditional ifNotMember(ClauseGenerator ifNotMember) { + this.ifNotMember = ifNotMember; + return this; + } + + /** + * If this is set, we will always create a new block scope, even if there + * are no 'if' statements. This is useful for cases where we want to + * declare variables in the clauses without worrying if they conflict with + * other variables of the same name. + */ + VersionConditional alwaysEmitBlockScope(boolean alwaysEmitBlockScope) { + this.alwaysEmitBlockScope = alwaysEmitBlockScope; + return this; + } + + /** + * If this is set, VersionConditional#generate will throw an exception if + * the 'ifMember' clause is never used. This is useful as a sanity check + * in some cases where it doesn't make sense for the condition to always be + * false. For example, when generating a Message#write function, + * we might check that the version we're writing is supported. It wouldn't + * make sense for this check to always be false, since that would mean that + * no versions at all were supported. + */ + VersionConditional allowMembershipCheckAlwaysFalse(boolean allowMembershipCheckAlwaysFalse) { + this.allowMembershipCheckAlwaysFalse = allowMembershipCheckAlwaysFalse; + return this; + } + + private void generateFullRangeCheck(Versions ifVersions, + Versions ifNotVersions, + CodeBuffer buffer) { + if (ifMember != null) { + buffer.printf("if ((version >= %d) && (version <= %d)) {%n", + containingVersions.lowest(), containingVersions.highest()); + buffer.incrementIndent(); + ifMember.generate(ifVersions); + buffer.decrementIndent(); + if (ifNotMember != null) { + buffer.printf("} else {%n"); + buffer.incrementIndent(); + ifNotMember.generate(ifNotVersions); + buffer.decrementIndent(); + } + buffer.printf("}%n"); + } else if (ifNotMember != null) { + buffer.printf("if ((version < %d) || (version > %d)) {%n", + containingVersions.lowest(), containingVersions.highest()); + buffer.incrementIndent(); + ifNotMember.generate(ifNotVersions); + buffer.decrementIndent(); + buffer.printf("}%n"); + } + } + + private void generateLowerRangeCheck(Versions ifVersions, + Versions ifNotVersions, + CodeBuffer buffer) { + if (ifMember != null) { + buffer.printf("if (version >= %d) {%n", containingVersions.lowest()); + buffer.incrementIndent(); + ifMember.generate(ifVersions); + buffer.decrementIndent(); + if (ifNotMember != null) { + buffer.printf("} else {%n"); + buffer.incrementIndent(); + ifNotMember.generate(ifNotVersions); + buffer.decrementIndent(); + } + buffer.printf("}%n"); + } else if (ifNotMember != null) { + buffer.printf("if (version < %d) {%n", containingVersions.lowest()); + buffer.incrementIndent(); + ifNotMember.generate(ifNotVersions); + buffer.decrementIndent(); + buffer.printf("}%n"); + } + } + + private void generateUpperRangeCheck(Versions ifVersions, + Versions ifNotVersions, + CodeBuffer buffer) { + if (ifMember != null) { + buffer.printf("if (version <= %d) {%n", containingVersions.highest()); + buffer.incrementIndent(); + ifMember.generate(ifVersions); + buffer.decrementIndent(); + if (ifNotMember != null) { + buffer.printf("} else {%n"); + buffer.incrementIndent(); + ifNotMember.generate(ifNotVersions); + buffer.decrementIndent(); + } + buffer.printf("}%n"); + } else if (ifNotMember != null) { + buffer.printf("if (version > %d) {%n", containingVersions.highest()); + buffer.incrementIndent(); + ifNotMember.generate(ifNotVersions); + buffer.decrementIndent(); + buffer.printf("}%n"); + } + } + + private void generateAlwaysTrueCheck(Versions ifVersions, CodeBuffer buffer) { + if (ifMember != null) { + if (alwaysEmitBlockScope) { + buffer.printf("{%n"); + buffer.incrementIndent(); + } + ifMember.generate(ifVersions); + if (alwaysEmitBlockScope) { + buffer.decrementIndent(); + buffer.printf("}%n"); + } + } + } + + private void generateAlwaysFalseCheck(Versions ifNotVersions, CodeBuffer buffer) { + if (!allowMembershipCheckAlwaysFalse) { + throw new RuntimeException("Version ranges " + containingVersions + + " and " + possibleVersions + " have no versions in common."); + } + if (ifNotMember != null) { + if (alwaysEmitBlockScope) { + buffer.printf("{%n"); + buffer.incrementIndent(); + } + ifNotMember.generate(ifNotVersions); + if (alwaysEmitBlockScope) { + buffer.decrementIndent(); + buffer.printf("}%n"); + } + } + } + + void generate(CodeBuffer buffer) { + Versions ifVersions = possibleVersions.intersect(containingVersions); + Versions ifNotVersions = possibleVersions.subtract(containingVersions); + // In the case where ifNotVersions would be two ranges rather than one, + // we just pass in the original possibleVersions instead. + // This is slightly less optimal, but allows us to avoid dealing with + // multiple ranges. + if (ifNotVersions == null) { + ifNotVersions = possibleVersions; + } + + if (possibleVersions.lowest() < containingVersions.lowest()) { + if (possibleVersions.highest() > containingVersions.highest()) { + generateFullRangeCheck(ifVersions, ifNotVersions, buffer); + } else if (possibleVersions.highest() >= containingVersions.lowest()) { + generateLowerRangeCheck(ifVersions, ifNotVersions, buffer); + } else { + generateAlwaysFalseCheck(ifNotVersions, buffer); + } + } else if (possibleVersions.highest() >= containingVersions.lowest() && + (possibleVersions.lowest() <= containingVersions.highest())) { + if (possibleVersions.highest() > containingVersions.highest()) { + generateUpperRangeCheck(ifVersions, ifNotVersions, buffer); + } else { + generateAlwaysTrueCheck(ifVersions, buffer); + } + } else { + generateAlwaysFalseCheck(ifNotVersions, buffer); + } + } +} diff --git a/generator/src/main/java/org/apache/kafka/message/Versions.java b/generator/src/main/java/org/apache/kafka/message/Versions.java index 3903680b422f..649c065d043c 100644 --- a/generator/src/main/java/org/apache/kafka/message/Versions.java +++ b/generator/src/main/java/org/apache/kafka/message/Versions.java @@ -109,6 +109,12 @@ public String toString() { } } + /** + * Return the intersection of two version ranges. + * + * @param other The other version range. + * @return A new version range. + */ public Versions intersect(Versions other) { short newLowest = lowest > other.lowest ? lowest : other.lowest; short newHighest = highest < other.highest ? highest : other.highest; @@ -118,6 +124,53 @@ public Versions intersect(Versions other) { return new Versions(newLowest, newHighest); } + /** + * Return a new version range that trims some versions from this range, if possible. + * We can't trim any versions if the resulting range would be disjoint. + * + * Some examples: + * 1-4.trim(1-2) = 3-4 + * 3+.trim(4+) = 3 + * 4+.trim(3+) = none + * 1-5.trim(2-4) = null + * + * @param other The other version range. + * @return A new version range. + */ + public Versions subtract(Versions other) { + if (other.lowest() <= lowest) { + if (other.highest >= highest) { + // Case 1: other is a superset of this. Trim everything. + return Versions.NONE; + } else if (other.highest < lowest) { + // Case 2: other is a disjoint version range that is lower than this. Trim nothing. + return this; + } else { + // Case 3: trim some values from the beginning of this range. + // + // Note: it is safe to assume that other.highest() + 1 will not overflow. + // The reason is because if other.highest() were Short.MAX_VALUE, + // other.highest() < highest could not be true. + return new Versions((short) (other.highest() + 1), highest); + } + } else if (other.highest >= highest) { + int newHighest = other.lowest - 1; + if (newHighest < 0) { + // Case 4: other was NONE. Trim nothing. + return this; + } else if (newHighest < highest) { + // Case 5: trim some values from the end of this range. + return new Versions(lowest, (short) newHighest); + } else { + // Case 6: other is a disjoint range that is higher than this. Trim nothing. + return this; + } + } else { + // Case 7: the difference between this and other would be two ranges, not one. + return null; + } + } + public boolean contains(short version) { return version >= lowest && version <= highest; } diff --git a/generator/src/test/java/org/apache/kafka/message/CodeBufferTest.java b/generator/src/test/java/org/apache/kafka/message/CodeBufferTest.java new file mode 100644 index 000000000000..e896658bfb0e --- /dev/null +++ b/generator/src/test/java/org/apache/kafka/message/CodeBufferTest.java @@ -0,0 +1,73 @@ +/* + * 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.message; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.StringWriter; + +public class CodeBufferTest { + @Rule + final public Timeout globalTimeout = Timeout.millis(120000); + + @Test + public void testWrite() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + buffer.printf("public static void main(String[] args) throws Exception {%n"); + buffer.incrementIndent(); + buffer.printf("System.out.println(\"%s\");%n", "hello world"); + buffer.decrementIndent(); + buffer.printf("}%n"); + StringWriter stringWriter = new StringWriter(); + buffer.write(stringWriter); + Assert.assertEquals( + String.format("public static void main(String[] args) throws Exception {%n") + + String.format(" System.out.println(\"hello world\");%n") + + String.format("}%n"), + stringWriter.toString()); + } + + @Test + public void testEquals() { + CodeBuffer buffer1 = new CodeBuffer(); + CodeBuffer buffer2 = new CodeBuffer(); + Assert.assertEquals(buffer1, buffer2); + buffer1.printf("hello world"); + Assert.assertNotEquals(buffer1, buffer2); + buffer2.printf("hello world"); + Assert.assertEquals(buffer1, buffer2); + buffer1.printf("foo, bar, and baz"); + buffer2.printf("foo, bar, and baz"); + Assert.assertEquals(buffer1, buffer2); + } + + @Test + public void testIndentMustBeNonNegative() { + CodeBuffer buffer = new CodeBuffer(); + buffer.incrementIndent(); + buffer.decrementIndent(); + try { + buffer.decrementIndent(); + } catch (RuntimeException e) { + Assert.assertTrue(e.getMessage().contains("Indent < 0")); + } + } +} diff --git a/generator/src/test/java/org/apache/kafka/message/IsNullConditionalTest.java b/generator/src/test/java/org/apache/kafka/message/IsNullConditionalTest.java new file mode 100644 index 000000000000..a8f7bc68c99a --- /dev/null +++ b/generator/src/test/java/org/apache/kafka/message/IsNullConditionalTest.java @@ -0,0 +1,122 @@ +/* + * 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.message; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +public class IsNullConditionalTest { + @Rule + final public Timeout globalTimeout = Timeout.millis(120000); + + @Test + public void testNullCheck() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + IsNullConditional. + forName("foobar"). + nullableVersions(Versions.parse("2+", null)). + possibleVersions(Versions.parse("0+", null)). + ifNull(() -> { + buffer.printf("System.out.println(\"null\");%n"); + }). + generate(buffer); + VersionConditionalTest.assertEquals(buffer, + "if (foobar == null) {%n", + " System.out.println(\"null\");%n", + "}%n"); + } + + @Test + public void testAnotherNullCheck() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + IsNullConditional. + forName("foobar"). + nullableVersions(Versions.parse("0+", null)). + possibleVersions(Versions.parse("2+", null)). + ifNull(() -> { + buffer.printf("System.out.println(\"null\");%n"); + }). + ifNotNull(() -> { + buffer.printf("System.out.println(\"not null\");%n"); + }). + generate(buffer); + VersionConditionalTest.assertEquals(buffer, + "if (foobar == null) {%n", + " System.out.println(\"null\");%n", + "} else {%n", + " System.out.println(\"not null\");%n", + "}%n"); + } + + @Test + public void testNotNullCheck() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + IsNullConditional. + forName("foobar"). + nullableVersions(Versions.parse("0+", null)). + possibleVersions(Versions.parse("2+", null)). + ifNotNull(() -> { + buffer.printf("System.out.println(\"not null\");%n"); + }). + generate(buffer); + VersionConditionalTest.assertEquals(buffer, + "if (foobar != null) {%n", + " System.out.println(\"not null\");%n", + "}%n"); + } + + @Test + public void testNeverNull() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + IsNullConditional. + forName("baz"). + nullableVersions(Versions.parse("0-2", null)). + possibleVersions(Versions.parse("3+", null)). + ifNull(() -> { + buffer.printf("System.out.println(\"null\");%n"); + }). + ifNotNull(() -> { + buffer.printf("System.out.println(\"not null\");%n"); + }). + generate(buffer); + VersionConditionalTest.assertEquals(buffer, + "System.out.println(\"not null\");%n"); + } + + @Test + public void testNeverNullWithBlockScope() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + IsNullConditional. + forName("baz"). + nullableVersions(Versions.parse("0-2", null)). + possibleVersions(Versions.parse("3+", null)). + ifNull(() -> { + buffer.printf("System.out.println(\"null\");%n"); + }). + ifNotNull(() -> { + buffer.printf("System.out.println(\"not null\");%n"); + }). + alwaysEmitBlockScope(true). + generate(buffer); + VersionConditionalTest.assertEquals(buffer, + "{%n", + " System.out.println(\"not null\");%n", + "}%n"); + } +} diff --git a/generator/src/test/java/org/apache/kafka/message/MessageDataGeneratorTest.java b/generator/src/test/java/org/apache/kafka/message/MessageDataGeneratorTest.java index e8fcaf227000..a77298897492 100644 --- a/generator/src/test/java/org/apache/kafka/message/MessageDataGeneratorTest.java +++ b/generator/src/test/java/org/apache/kafka/message/MessageDataGeneratorTest.java @@ -47,7 +47,7 @@ public void testNullDefaults() throws Exception { " \"nullableVersions\": \"2+\", \"default\": \"null\" }", " ]", "}")), MessageSpec.class); - new MessageDataGenerator().generate(testMessageSpec); + new MessageDataGenerator("org.apache.kafka.common.message").generate(testMessageSpec); } @Test @@ -62,7 +62,7 @@ public void testInvalidNullDefaultForInt() throws Exception { " ]", "}")), MessageSpec.class); try { - new MessageDataGenerator().generate(testMessageSpec); + new MessageDataGenerator("org.apache.kafka.common.message").generate(testMessageSpec); fail("Expected MessageDataGenerator#generate to fail"); } catch (Throwable e) { assertTrue("Invalid error message: " + e.getMessage(), @@ -83,7 +83,7 @@ public void testInvalidNullDefaultForPotentiallyNonNullableArray() throws Except " ]", "}")), MessageSpec.class); try { - new MessageDataGenerator().generate(testMessageSpec); + new MessageDataGenerator("org.apache.kafka.common.message").generate(testMessageSpec); fail("Expected MessageDataGenerator#generate to fail"); } catch (RuntimeException e) { assertTrue("Invalid error message: " + e.getMessage(), diff --git a/generator/src/test/java/org/apache/kafka/message/VersionConditionalTest.java b/generator/src/test/java/org/apache/kafka/message/VersionConditionalTest.java new file mode 100644 index 000000000000..97180f7810b4 --- /dev/null +++ b/generator/src/test/java/org/apache/kafka/message/VersionConditionalTest.java @@ -0,0 +1,245 @@ +/* + * 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.message; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.StringWriter; + +import static org.junit.Assert.assertTrue; + +public class VersionConditionalTest { + @Rule + final public Timeout globalTimeout = Timeout.millis(120000); + + static void assertEquals(CodeBuffer buffer, String... lines) throws Exception { + StringWriter stringWriter = new StringWriter(); + buffer.write(stringWriter); + StringBuilder expectedStringBuilder = new StringBuilder(); + for (String line : lines) { + expectedStringBuilder.append(String.format(line)); + } + Assert.assertEquals(expectedStringBuilder.toString(), stringWriter.toString()); + } + + @Test + public void testAlwaysFalseConditional() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("1-2", null), Versions.parse("3+", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"foobar\");%n"); + }). + generate(buffer); + assertEquals(buffer, + "System.out.println(\"foobar\");%n"); + } + + @Test + public void testAnotherAlwaysFalseConditional() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("3+", null), Versions.parse("1-2", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"foobar\");%n"); + }). + generate(buffer); + assertEquals(buffer, + "System.out.println(\"foobar\");%n"); + } + + @Test + public void testAllowMembershipCheckAlwaysFalseFails() throws Exception { + try { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("1-2", null), Versions.parse("3+", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"foobar\");%n"); + }). + allowMembershipCheckAlwaysFalse(false). + generate(buffer); + } catch (RuntimeException e) { + assertTrue(e.getMessage().contains("no versions in common")); + } + } + + @Test + public void testAlwaysTrueConditional() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("1-5", null), Versions.parse("2-4", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"foobar\");%n"); + }). + allowMembershipCheckAlwaysFalse(false). + generate(buffer); + assertEquals(buffer, + "System.out.println(\"hello world\");%n"); + } + + @Test + public void testAlwaysTrueConditionalWithAlwaysEmitBlockScope() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("1-5", null), Versions.parse("2-4", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"foobar\");%n"); + }). + alwaysEmitBlockScope(true). + generate(buffer); + assertEquals(buffer, + "{%n", + " System.out.println(\"hello world\");%n", + "}%n"); + } + + @Test + public void testLowerRangeCheckWithElse() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("1+", null), Versions.parse("0-100", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"foobar\");%n"); + }). + generate(buffer); + assertEquals(buffer, + "if (version >= 1) {%n", + " System.out.println(\"hello world\");%n", + "} else {%n", + " System.out.println(\"foobar\");%n", + "}%n"); + } + + @Test + public void testLowerRangeCheckWithIfMember() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("1+", null), Versions.parse("0-100", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + generate(buffer); + assertEquals(buffer, + "if (version >= 1) {%n", + " System.out.println(\"hello world\");%n", + "}%n"); + } + + @Test + public void testLowerRangeCheckWithIfNotMember() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("1+", null), Versions.parse("0-100", null)). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + generate(buffer); + assertEquals(buffer, + "if (version < 1) {%n", + " System.out.println(\"hello world\");%n", + "}%n"); + } + + @Test + public void testUpperRangeCheckWithElse() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("0-10", null), Versions.parse("4+", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"foobar\");%n"); + }). + generate(buffer); + assertEquals(buffer, + "if (version <= 10) {%n", + " System.out.println(\"hello world\");%n", + "} else {%n", + " System.out.println(\"foobar\");%n", + "}%n"); + } + + @Test + public void testUpperRangeCheckWithIfMember() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("0-10", null), Versions.parse("4+", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + generate(buffer); + assertEquals(buffer, + "if (version <= 10) {%n", + " System.out.println(\"hello world\");%n", + "}%n"); + } + + @Test + public void testUpperRangeCheckWithIfNotMember() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("1+", null), Versions.parse("0-100", null)). + ifNotMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + generate(buffer); + assertEquals(buffer, + "if (version < 1) {%n", + " System.out.println(\"hello world\");%n", + "}%n"); + } + + @Test + public void testFullRangeCheck() throws Exception { + CodeBuffer buffer = new CodeBuffer(); + VersionConditional. + forVersions(Versions.parse("5-10", null), Versions.parse("1+", null)). + ifMember(__ -> { + buffer.printf("System.out.println(\"hello world\");%n"); + }). + allowMembershipCheckAlwaysFalse(false). + generate(buffer); + assertEquals(buffer, + "if ((version >= 5) && (version <= 10)) {%n", + " System.out.println(\"hello world\");%n", + "}%n"); + } +} diff --git a/generator/src/test/java/org/apache/kafka/message/VersionsTest.java b/generator/src/test/java/org/apache/kafka/message/VersionsTest.java index 051202e46f34..f688299d18f0 100644 --- a/generator/src/test/java/org/apache/kafka/message/VersionsTest.java +++ b/generator/src/test/java/org/apache/kafka/message/VersionsTest.java @@ -72,6 +72,8 @@ public void testIntersections() { assertEquals(Versions.NONE, newVersions(9, Short.MAX_VALUE).intersect( newVersions(2, 8))); + assertEquals(Versions.NONE, + Versions.NONE.intersect(Versions.NONE)); } @Test @@ -87,4 +89,26 @@ public void testContains() { assertTrue(newVersions(2, 3).contains(Versions.NONE)); assertTrue(Versions.ALL.contains(newVersions(1, 2))); } + + @Test + public void testSubtract() { + assertEquals(Versions.NONE, + Versions.NONE.subtract(Versions.NONE)); + assertEquals(newVersions(0, 0), + newVersions(0, 0).subtract(Versions.NONE)); + assertEquals(newVersions(1, 1), + newVersions(1, 2).subtract(newVersions(2, 2))); + assertEquals(newVersions(2, 2), + newVersions(1, 2).subtract(newVersions(1, 1))); + assertEquals(null, + newVersions(0, Short.MAX_VALUE).subtract(newVersions(1, 100))); + assertEquals(newVersions(10, 10), + newVersions(1, 10).subtract(newVersions(1, 9))); + assertEquals(newVersions(1, 1), + newVersions(1, 10).subtract(newVersions(2, 10))); + assertEquals(newVersions(2, 4), + newVersions(2, Short.MAX_VALUE).subtract(newVersions(5, Short.MAX_VALUE))); + assertEquals(newVersions(5, Short.MAX_VALUE), + newVersions(0, Short.MAX_VALUE).subtract(newVersions(0, 4))); + } } diff --git a/gradle.properties b/gradle.properties index 66079af6c97d..2e511c2fb88d 100644 --- a/gradle.properties +++ b/gradle.properties @@ -21,6 +21,6 @@ group=org.apache.kafka # - tests/kafkatest/version.py (variable DEV_VERSION) # - kafka-merge-pr.py version=5.4.0-ccs-SNAPSHOT -scalaVersion=2.12.9 +scalaVersion=2.12.10 task=build org.gradle.jvmargs=-Xmx1024m -Xss2m diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index e1b09c2dc6bc..485fbce2d19c 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -30,8 +30,8 @@ ext { // Add Scala version def defaultScala211Version = '2.11.12' -def defaultScala212Version = '2.12.9' -def defaultScala213Version = '2.13.0' +def defaultScala212Version = '2.12.10' +def defaultScala213Version = '2.13.1' if (hasProperty('scalaVersion')) { if (scalaVersion == '2.11') { versions["scala"] = defaultScala211Version @@ -65,10 +65,10 @@ versions += [ argparse4j: "0.7.0", avro: "1.9.0", avroPlugin: "0.17.0", - bcpkix: "1.62", + bcpkix: "1.63", checkstyle: "8.20", commonsCli: "1.4", - gradle: "5.4.1", + gradle: "5.6.2", gradleVersionsPlugin: "0.21.0", grgit: "3.1.1", httpclient: "4.5.9", @@ -76,7 +76,7 @@ versions += [ jackson: "2.9.9", jacksonDatabind: "2.9.9.3", jacoco: "0.8.3", - jetty: "9.4.19.v20190610", + jetty: "9.4.20.v20190813", jersey: "2.28", jmh: "1.21", hamcrest: "2.1", @@ -112,13 +112,13 @@ versions += [ scoverage: "1.4.0", scoveragePlugin: "2.5.0", shadowPlugin: "4.0.4", - slf4j: "1.7.27", + slf4j: "1.7.28", snappy: "1.1.7.3", spotbugs: "3.1.12", spotbugsPlugin: "1.6.9", spotlessPlugin: "3.23.1", zookeeper: "3.5.5", - zstd: "1.4.2-1" + zstd: "1.4.3-1" ] libs += [ diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index 33eb29ff5dd8..d7a4b22f0ae4 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -156,6 +156,7 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read + @@ -220,6 +221,25 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read + + + + + + + + + + + + + + + + + + + + - * | +-----+-------+ | - * | | | - * | v | - * | +-----+-------+ | - * | | Running (4) | ---->+ - * | +-----+-------+ - * | | - * | v - * | +-----+-------+ - * +---> | Pending | - * | Shutdown (5)| - * +-----+-------+ - * | - * v - * +-----+-------+ - * | Dead (6) | - * +-------------+ + * +-------------+ + * +<---- | Created (0) | + * | +-----+-------+ + * | | + * | v + * | +-----+-------+ + * +<---- | Starting (1)|----->+ + * | +-----+-------+ | + * | | | + * | | | + * | v | + * | +-----+-------+ | + * +<---- | Partitions | | + * | | Revoked (2) | <----+ + * | +-----+-------+ | + * | | ^ | + * | | | | + * | v | | + * | +-----+-------+ | + * +<---- | Partitions | | + * | | Assigned (3)| <----+ + * | +-----+-------+ | + * | | | + * | | | + * | v | + * | +-----+-------+ | + * | | Running (4) | ---->+ + * | +-----+-------+ + * | | + * | | + * | v + * | +-----+-------+ + * +----> | Pending | + * | Shutdown (5)| + * +-----+-------+ + * | + * v + * +-----+-------+ + * | Dead (6) | + * +-------------+ * * * Note the following: @@ -123,15 +127,20 @@ public class StreamThread extends Thread { * State PARTITIONS_REVOKED may want transit to itself indefinitely, in the corner case when * the coordinator repeatedly fails in-between revoking partitions and assigning new partitions. * Also during streams instance start up PARTITIONS_REVOKED may want to transit to itself as well. - * In this case we will forbid the transition but will not treat as an error. + * In this case we will allow the transition but it will be a no-op as the set of revoked partitions + * should be empty. *
    • *
    */ public enum State implements ThreadStateTransitionValidator { - // TODO: the current transitions from other states directly to PARTITIONS_REVOKED is due to - // the fact that onPartitionsRevoked may not be triggered. we need to refactor the - // state diagram more thoroughly after we refactor StreamsPartitionAssignor to support COOPERATIVE - CREATED(1, 5), STARTING(2, 3, 5), PARTITIONS_REVOKED(3, 5), PARTITIONS_ASSIGNED(2, 3, 4, 5), RUNNING(2, 3, 5), PENDING_SHUTDOWN(6), DEAD; + + CREATED(1, 5), // 0 + STARTING(2, 3, 5), // 1 + PARTITIONS_REVOKED(3, 5), // 2 + PARTITIONS_ASSIGNED(2, 3, 4, 5), // 3 + RUNNING(2, 3, 5), // 4 + PENDING_SHUTDOWN(6), // 5 + DEAD; // 6 private final Set validTransitions = new HashSet<>(); @@ -205,12 +214,6 @@ State setState(final State newState) { // when the state is already in NOT_RUNNING, all its transitions // will be refused but we do not throw exception here return null; - } else if (state == State.PARTITIONS_REVOKED && newState == State.PARTITIONS_REVOKED) { - log.debug("Ignoring request to transit from PARTITIONS_REVOKED to PARTITIONS_REVOKED: " + - "self transition is not allowed"); - // when the state is already in PARTITIONS_REVOKED, its transition to itself will be - // refused but we do not throw exception here - return null; } else if (!state.isValidTransition(newState)) { log.error("Unexpected state transition from {} to {}", oldState, newState); throw new StreamsException(logPrefix + "Unexpected state transition from " + oldState + " to " + newState); @@ -244,104 +247,12 @@ public boolean isRunning() { } } - static class RebalanceListener implements ConsumerRebalanceListener { - private final Time time; - private final TaskManager taskManager; - private final StreamThread streamThread; - private final Logger log; - - RebalanceListener(final Time time, - final TaskManager taskManager, - final StreamThread streamThread, - final Logger log) { - this.time = time; - this.taskManager = taskManager; - this.streamThread = streamThread; - this.log = log; - } - - @Override - public void onPartitionsAssigned(final Collection assignment) { - log.debug("at state {}: partitions {} assigned at the end of consumer rebalance.\n" + - "\tcurrent suspended active tasks: {}\n" + - "\tcurrent suspended standby tasks: {}\n", - streamThread.state, - assignment, - taskManager.suspendedActiveTaskIds(), - taskManager.suspendedStandbyTaskIds()); - - if (streamThread.assignmentErrorCode.get() == AssignorError.INCOMPLETE_SOURCE_TOPIC_METADATA.code()) { - log.error("Received error code {} - shutdown", streamThread.assignmentErrorCode.get()); - streamThread.shutdown(); - return; - } - final long start = time.milliseconds(); - try { - if (streamThread.setState(State.PARTITIONS_ASSIGNED) == null) { - log.debug( - "Skipping task creation in rebalance because we are already in {} state.", - streamThread.state() - ); - } else if (streamThread.assignmentErrorCode.get() != AssignorError.NONE.code()) { - log.debug( - "Encountered assignment error during partition assignment: {}. Skipping task initialization", - streamThread.assignmentErrorCode - ); - } else { - log.debug("Creating tasks based on assignment."); - taskManager.createTasks(assignment); - } - } catch (final Throwable t) { - log.error( - "Error caught during partition assignment, " + - "will abort the current process and re-throw at the end of rebalance", t); - streamThread.setRebalanceException(t); - } finally { - log.info("partition assignment took {} ms.\n" + - "\tcurrent active tasks: {}\n" + - "\tcurrent standby tasks: {}\n" + - "\tprevious active tasks: {}\n", - time.milliseconds() - start, - taskManager.activeTaskIds(), - taskManager.standbyTaskIds(), - taskManager.prevActiveTaskIds()); - } - } + int getAssignmentErrorCode() { + return assignmentErrorCode.get(); + } - @Override - public void onPartitionsRevoked(final Collection assignment) { - log.debug("at state {}: partitions {} revoked at the beginning of consumer rebalance.\n" + - "\tcurrent assigned active tasks: {}\n" + - "\tcurrent assigned standby tasks: {}\n", - streamThread.state, - assignment, - taskManager.activeTaskIds(), - taskManager.standbyTaskIds()); - - if (streamThread.setState(State.PARTITIONS_REVOKED) != null) { - final long start = time.milliseconds(); - try { - // suspend active tasks - taskManager.suspendTasksAndState(); - } catch (final Throwable t) { - log.error( - "Error caught during partition revocation, " + - "will abort the current process and re-throw at the end of rebalance: {}", - t - ); - streamThread.setRebalanceException(t); - } finally { - streamThread.clearStandbyRecords(); - - log.info("partition revocation took {} ms.\n" + - "\tsuspended active tasks: {}\n" + - "\tsuspended standby tasks: {}", - time.milliseconds() - start, - taskManager.suspendedActiveTaskIds(), - taskManager.suspendedStandbyTaskIds()); - } - } - } + void setRebalanceException(final Throwable rebalanceException) { + this.rebalanceException = rebalanceException; } static abstract class AbstractTaskCreator { @@ -596,7 +507,11 @@ public static StreamThread create(final InternalTopologyBuilder builder, threadProducer = clientSupplier.getProducer(producerConfigs); } - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, threadClientId); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( + metrics, + threadClientId, + config.getString(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG) + ); final ThreadCache cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics); @@ -700,7 +615,7 @@ public StreamThread(final Time time, this.builder = builder; this.logPrefix = logContext.logPrefix(); this.log = logContext.logger(StreamThread.class); - this.rebalanceListener = new RebalanceListener(time, taskManager, this, this.log); + this.rebalanceListener = new StreamsRebalanceListener(time, taskManager, this, this.log); this.taskManager = taskManager; this.producer = producer; this.restoreConsumer = restoreConsumer; @@ -744,6 +659,10 @@ public static String getSharedAdminClientId(final String clientId) { return clientId + "-admin"; } + public void setRocksDBMetricsRecordingTrigger(final RocksDBMetricsRecordingTrigger rocksDBMetricsRecordingTrigger) { + streamsMetrics.setRocksDBMetricsRecordingTrigger(rocksDBMetricsRecordingTrigger); + } + /** * Execute the stream processors * @@ -775,10 +694,6 @@ public void run() { } } - private void setRebalanceException(final Throwable rebalanceException) { - this.rebalanceException = rebalanceException; - } - /** * Main event loop for polling, and processing records through topologies. * @@ -1222,8 +1137,10 @@ private void completeShutdown(final boolean cleanRun) { log.info("Shutdown complete"); } - private void clearStandbyRecords() { - standbyRecords.clear(); + void clearStandbyRecords(final List partitions) { + for (final TopicPartition tp : partitions) { + standbyRecords.remove(tp); + } } /** @@ -1330,7 +1247,7 @@ public Map consumerMetrics() { } public Map adminClientMetrics() { - final Map adminClientMetrics = taskManager.getAdminClient().metrics(); + final Map adminClientMetrics = taskManager.adminClient().metrics(); final LinkedHashMap result = new LinkedHashMap<>(); result.putAll(adminClientMetrics); return result; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 2d46778124af..b207e1de3945 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.processor.PartitionGrouper; @@ -52,6 +53,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; @@ -164,6 +166,7 @@ public String toString() { private InternalTopicManager internalTopicManager; private CopartitionedTopicsEnforcer copartitionedTopicsEnforcer; + private RebalanceProtocol rebalanceProtocol; protected String userEndPoint() { return userEndPoint; @@ -194,14 +197,24 @@ public void configure(final Map configs) { userEndPoint = assignorConfiguration.getUserEndPoint(); internalTopicManager = assignorConfiguration.getInternalTopicManager(); copartitionedTopicsEnforcer = assignorConfiguration.getCopartitionedTopicsEnforcer(); + rebalanceProtocol = assignorConfiguration.rebalanceProtocol(); } - @Override public String name() { return "stream"; } + @Override + public List supportedProtocols() { + final List supportedProtocols = new ArrayList<>(); + supportedProtocols.add(RebalanceProtocol.EAGER); + if (rebalanceProtocol == RebalanceProtocol.COOPERATIVE) { + supportedProtocols.add(rebalanceProtocol); + } + return supportedProtocols; + } + @Override public ByteBuffer subscriptionUserData(final Set topics) { // Adds the following information to subscription @@ -209,7 +222,7 @@ public ByteBuffer subscriptionUserData(final Set topics) { // 2. Task ids of previously running tasks // 3. Task ids of valid local states on the client's state directory. - final Set previousActiveTasks = taskManager.prevActiveTaskIds(); + final Set previousActiveTasks = taskManager.previousRunningTaskIds(); final Set standbyTasks = taskManager.cachedTasksIds(); standbyTasks.removeAll(previousActiveTasks); final SubscriptionInfo data = new SubscriptionInfo( @@ -525,7 +538,8 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr final StickyTaskAssignor taskAssignor = new StickyTaskAssignor<>(states, partitionsForTask.keySet()); taskAssignor.assign(numStandbyReplicas); - log.info("Assigned tasks to clients as {}.", states); + log.info("Assigned tasks to clients as {}{}.", Utils.NL, states.entrySet().stream() + .map(Map.Entry::toString).collect(Collectors.joining(Utils.NL))); // ---------------- Step Three ---------------- // @@ -777,20 +791,22 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat final Map topicToPartitionInfo = new HashMap<>(); final Map> partitionsByHost; + final Map partitionsToTaskId = new HashMap<>(); + switch (receivedAssignmentMetadataVersion) { case VERSION_ONE: - processVersionOneAssignment(logPrefix, info, partitions, activeTasks); + processVersionOneAssignment(logPrefix, info, partitions, activeTasks, partitionsToTaskId); partitionsByHost = Collections.emptyMap(); break; case VERSION_TWO: - processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo); + processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId); partitionsByHost = info.partitionsByHost(); break; case VERSION_THREE: case VERSION_FOUR: case VERSION_FIVE: upgradeSubscriptionVersionIfNeeded(leaderSupportedVersion); - processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo); + processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId); partitionsByHost = info.partitionsByHost(); break; default: @@ -802,6 +818,7 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat taskManager.setClusterMetadata(Cluster.empty().withPartitions(topicToPartitionInfo)); taskManager.setPartitionsByHostState(partitionsByHost); + taskManager.setPartitionsToTaskId(partitionsToTaskId); taskManager.setAssignmentMetadata(activeTasks, info.standbyTasks()); taskManager.updateSubscriptionsFromAssignment(partitions); } @@ -809,7 +826,8 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat private static void processVersionOneAssignment(final String logPrefix, final AssignmentInfo info, final List partitions, - final Map> activeTasks) { + final Map> activeTasks, + final Map partitionsToTaskId) { // the number of assigned partitions should be the same as number of active tasks, which // could be duplicated if one task has more than one assigned partitions if (partitions.size() != info.activeTasks().size()) { @@ -827,15 +845,17 @@ private static void processVersionOneAssignment(final String logPrefix, final TopicPartition partition = partitions.get(i); final TaskId id = info.activeTasks().get(i); activeTasks.computeIfAbsent(id, k -> new HashSet<>()).add(partition); + partitionsToTaskId.put(partition, id); } } public static void processVersionTwoAssignment(final String logPrefix, - final AssignmentInfo info, - final List partitions, - final Map> activeTasks, - final Map topicToPartitionInfo) { - processVersionOneAssignment(logPrefix, info, partitions, activeTasks); + final AssignmentInfo info, + final List partitions, + final Map> activeTasks, + final Map topicToPartitionInfo, + final Map partitionsToTaskId) { + processVersionOneAssignment(logPrefix, info, partitions, activeTasks, partitionsToTaskId); // process partitions by host final Map> partitionsByHost = info.partitionsByHost(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java new file mode 100644 index 000000000000..17563fd460e4 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java @@ -0,0 +1,169 @@ +/* + * 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.streams.processor.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.StreamThread.State; +import org.apache.kafka.streams.processor.internals.assignment.AssignorError; +import org.slf4j.Logger; + +public class StreamsRebalanceListener implements ConsumerRebalanceListener { + + private final Time time; + private final TaskManager taskManager; + private final StreamThread streamThread; + private final Logger log; + + StreamsRebalanceListener(final Time time, + final TaskManager taskManager, + final StreamThread streamThread, + final Logger log) { + this.time = time; + this.taskManager = taskManager; + this.streamThread = streamThread; + this.log = log; + } + + @Override + public void onPartitionsAssigned(final Collection assignedPartitions) { + log.debug("Current state {}: assigned partitions {} at the end of consumer rebalance.\n" + + "\tpreviously assigned active tasks: {}\n" + + "\tpreviously assigned standby tasks: {}\n", + streamThread.state(), + assignedPartitions, + taskManager.previousActiveTaskIds(), + taskManager.previousStandbyTaskIds()); + + if (streamThread.getAssignmentErrorCode() == AssignorError.INCOMPLETE_SOURCE_TOPIC_METADATA.code()) { + log.error("Received error code {} - shutdown", streamThread.getAssignmentErrorCode()); + streamThread.shutdown(); + return; + } + + final long start = time.milliseconds(); + List revokedStandbyPartitions = null; + + try { + if (streamThread.setState(State.PARTITIONS_ASSIGNED) == null) { + log.debug( + "Skipping task creation in rebalance because we are already in {} state.", + streamThread.state() + ); + } else if (streamThread.getAssignmentErrorCode() != AssignorError.NONE.code()) { + log.debug( + "Encountered assignment error during partition assignment: {}. Skipping task initialization", + streamThread.getAssignmentErrorCode() + ); + } else { + // Close non-reassigned tasks before initializing new ones as we may have suspended active + // tasks that become standbys or vice versa + revokedStandbyPartitions = taskManager.closeRevokedStandbyTasks(); + taskManager.closeRevokedSuspendedTasks(); + taskManager.createTasks(assignedPartitions); + } + } catch (final Throwable t) { + log.error( + "Error caught during partition assignment, " + + "will abort the current process and re-throw at the end of rebalance", t); + streamThread.setRebalanceException(t); + } finally { + if (revokedStandbyPartitions != null) { + streamThread.clearStandbyRecords(revokedStandbyPartitions); + } + log.info("partition assignment took {} ms.\n" + + "\tcurrently assigned active tasks: {}\n" + + "\tcurrently assigned standby tasks: {}\n" + + "\trevoked active tasks: {}\n" + + "\trevoked standby tasks: {}\n", + time.milliseconds() - start, + taskManager.activeTaskIds(), + taskManager.standbyTaskIds(), + taskManager.revokedActiveTaskIds(), + taskManager.revokedStandbyTaskIds()); + } + } + + @Override + public void onPartitionsRevoked(final Collection revokedPartitions) { + log.debug("Current state {}: revoked partitions {} because of consumer rebalance.\n" + + "\tcurrently assigned active tasks: {}\n" + + "\tcurrently assigned standby tasks: {}\n", + streamThread.state(), + revokedPartitions, + taskManager.activeTaskIds(), + taskManager.standbyTaskIds()); + + Set suspendedTasks = new HashSet<>(); + if (streamThread.setState(State.PARTITIONS_REVOKED) != null && !revokedPartitions.isEmpty()) { + final long start = time.milliseconds(); + try { + // suspend only the active tasks, reassigned standby tasks will be closed in onPartitionsAssigned + suspendedTasks = taskManager.suspendActiveTasksAndState(revokedPartitions); + } catch (final Throwable t) { + log.error( + "Error caught during partition revocation, " + + "will abort the current process and re-throw at the end of rebalance: ", + t + ); + streamThread.setRebalanceException(t); + } finally { + log.info("partition revocation took {} ms.\n" + + "\tcurrent suspended active tasks: {}\n", + time.milliseconds() - start, + suspendedTasks); + } + } + } + + @Override + public void onPartitionsLost(final Collection lostPartitions) { + log.info("at state {}: partitions {} lost due to missed rebalance.\n" + + "\tlost active tasks: {}\n" + + "\tlost assigned standby tasks: {}\n", + streamThread.state(), + lostPartitions, + taskManager.activeTaskIds(), + taskManager.standbyTaskIds()); + + Set lostTasks = new HashSet<>(); + final long start = time.milliseconds(); + try { + // close lost active tasks but don't try to commit offsets as we no longer own them + lostTasks = taskManager.closeLostTasks(lostPartitions); + } catch (final Throwable t) { + log.error( + "Error caught during partitions lost, " + + "will abort the current process and re-throw at the end of rebalance: ", + t + ); + streamThread.setRebalanceException(t); + } finally { + log.info("partitions lost took {} ms.\n" + + "\tsuspended lost active tasks: {}\n", + time.milliseconds() - start, + lostTasks); + } + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java index 812e7e1131ca..da9e656ad9e2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java @@ -72,4 +72,5 @@ void close(final boolean clean, String toString(final String indent); + void initializeTaskTime(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index cc448a153b87..0a7acf3336c8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.ArrayList; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.DeleteRecordsResult; import org.apache.kafka.clients.admin.RecordsToDelete; @@ -60,10 +61,18 @@ public class TaskManager { private final Admin adminClient; private DeleteRecordsResult deleteRecordsResult; + // the restore consumer is only ever assigned changelogs from restoring tasks or standbys (but not both) + private boolean restoreConsumerAssignedStandbys = false; + // following information is updated during rebalance phase by the partition assignor private Cluster cluster; - private Map> assignedActiveTasks; - private Map> assignedStandbyTasks; + private Map partitionsToTaskId = new HashMap<>(); + private Map> assignedActiveTasks = new HashMap<>(); + private Map> assignedStandbyTasks = new HashMap<>(); + private Map> addedActiveTasks = new HashMap<>(); + private Map> addedStandbyTasks = new HashMap<>(); + private Map> revokedActiveTasks = new HashMap<>(); + private Map> revokedStandbyTasks = new HashMap<>(); private Consumer consumer; @@ -103,78 +112,54 @@ void createTasks(final Collection assignment) { throw new IllegalStateException(logPrefix + "consumer has not been initialized while adding stream tasks. This should not happen."); } - // do this first as we may have suspended standby tasks that - // will become active or vice versa - standby.closeNonAssignedSuspendedTasks(assignedStandbyTasks); - active.closeNonAssignedSuspendedTasks(assignedActiveTasks); + if (!assignment.isEmpty() && !assignedActiveTasks.isEmpty()) { + resumeSuspended(assignment); + } + if (!addedActiveTasks.isEmpty()) { + addNewActiveTasks(addedActiveTasks); + } + if (!addedStandbyTasks.isEmpty()) { + addNewStandbyTasks(addedStandbyTasks); + } + + // need to clear restore consumer if it was reading standbys but we have active tasks that may need restoring + if (!addedActiveTasks.isEmpty() && restoreConsumerAssignedStandbys) { + restoreConsumer.unsubscribe(); + restoreConsumerAssignedStandbys = false; + } - addStreamTasks(assignment); - addStandbyTasks(); - // Pause all the partitions until the underlying state store is ready for all the active tasks. + // Pause all the new partitions until the underlying state store is ready for all the active tasks. log.trace("Pausing partitions: {}", assignment); consumer.pause(assignment); } - private void addStreamTasks(final Collection assignment) { - if (assignedActiveTasks == null || assignedActiveTasks.isEmpty()) { - return; - } - final Map> newTasks = new HashMap<>(); - // collect newly assigned tasks and reopen re-assigned tasks - log.debug("Adding assigned tasks as active: {}", assignedActiveTasks); - for (final Map.Entry> entry : assignedActiveTasks.entrySet()) { - final TaskId taskId = entry.getKey(); - final Set partitions = entry.getValue(); + private void resumeSuspended(final Collection assignment) { + final Set suspendedTasks = partitionsToTaskSet(assignment); + suspendedTasks.removeAll(addedActiveTasks.keySet()); - if (assignment.containsAll(partitions)) { - try { - if (!active.maybeResumeSuspendedTask(taskId, partitions)) { - newTasks.put(taskId, partitions); - } - } catch (final StreamsException e) { - log.error("Failed to resume an active task {} due to the following error:", taskId, e); - throw e; + for (final TaskId taskId : suspendedTasks) { + final Set partitions = assignedActiveTasks.get(taskId); + try { + if (!active.maybeResumeSuspendedTask(taskId, partitions)) { + // recreate if resuming the suspended task failed because the associated partitions changed + addedActiveTasks.put(taskId, partitions); } - } else { - log.warn("Task {} owned partitions {} are not contained in the assignment {}", taskId, partitions, assignment); + } catch (final StreamsException e) { + log.error("Failed to resume an active task {} due to the following error:", taskId, e); + throw e; } } + } - if (newTasks.isEmpty()) { - return; - } - - // CANNOT FIND RETRY AND BACKOFF LOGIC - // create all newly assigned tasks (guard against race condition with other thread via backoff and retry) - // -> other thread will call removeSuspendedTasks(); eventually - log.trace("New active tasks to be created: {}", newTasks); + private void addNewActiveTasks(final Map> newActiveTasks) { + log.debug("New active tasks to be created: {}", newActiveTasks); - for (final StreamTask task : taskCreator.createTasks(consumer, newTasks)) { + for (final StreamTask task : taskCreator.createTasks(consumer, newActiveTasks)) { active.addNewTask(task); } } - private void addStandbyTasks() { - if (assignedStandbyTasks == null || assignedStandbyTasks.isEmpty()) { - return; - } - log.debug("Adding assigned standby tasks {}", assignedStandbyTasks); - final Map> newStandbyTasks = new HashMap<>(); - // collect newly assigned standby tasks and reopen re-assigned standby tasks - for (final Map.Entry> entry : assignedStandbyTasks.entrySet()) { - final TaskId taskId = entry.getKey(); - final Set partitions = entry.getValue(); - if (!standby.maybeResumeSuspendedTask(taskId, partitions)) { - newStandbyTasks.put(taskId, partitions); - } - } - - if (newStandbyTasks.isEmpty()) { - return; - } - - // create all newly assigned standby tasks (guard against race condition with other thread via backoff and retry) - // -> other thread will call removeSuspendedStandbyTasks(); eventually + private void addNewStandbyTasks(final Map> newStandbyTasks) { log.trace("New standby tasks to be created: {}", newStandbyTasks); for (final StandbyTask task : standbyTaskCreator.createTasks(consumer, newStandbyTasks)) { @@ -182,18 +167,6 @@ private void addStandbyTasks() { } } - Set activeTaskIds() { - return active.allAssignedTaskIds(); - } - - Set standbyTaskIds() { - return standby.allAssignedTaskIds(); - } - - public Set prevActiveTaskIds() { - return active.previousTaskIds(); - } - /** * Returns ids of tasks whose states are kept on the local storage. */ @@ -224,47 +197,90 @@ public Set cachedTasksIds() { return tasks; } - public UUID processId() { - return processId; + /** + * Closes standby tasks that were not reassigned at the end of a rebalance. + * + * @return list of changelog topic partitions from revoked tasks + * @throws TaskMigratedException if the task producer got fenced (EOS only) + */ + List closeRevokedStandbyTasks() { + final List revokedChangelogs = standby.closeRevokedStandbyTasks(revokedStandbyTasks); + + // If the restore consumer is assigned any standby partitions they must be removed + removeChangelogsFromRestoreConsumer(revokedChangelogs, true); + + return revokedChangelogs; } - InternalTopologyBuilder builder() { - return taskCreator.builder(); + /** + * Closes suspended active tasks that were not reassigned at the end of a rebalance. + * + * @throws TaskMigratedException if the task producer got fenced (EOS only) + */ + void closeRevokedSuspendedTasks() { + // changelogs should have already been removed during suspend + final RuntimeException exception = active.closeNotAssignedSuspendedTasks(revokedActiveTasks.keySet()); + + // At this point all revoked tasks should have been closed, we can just throw the exception + if (exception != null) { + throw exception; + } } /** * Similar to shutdownTasksAndState, however does not close the task managers, in the hope that - * soon the tasks will be assigned again + * soon the tasks will be assigned again. + * @return list of suspended tasks * @throws TaskMigratedException if the task producer got fenced (EOS only) */ - void suspendTasksAndState() { - log.debug("Suspending all active tasks {} and standby tasks {}", active.runningTaskIds(), standby.runningTaskIds()); - + Set suspendActiveTasksAndState(final Collection revokedPartitions) { final AtomicReference firstException = new AtomicReference<>(null); + final List revokedChangelogs = new ArrayList<>(); - firstException.compareAndSet(null, active.suspend()); - // close all restoring tasks as well and then reset changelog reader; - // for those restoring and still assigned tasks, they will be re-created - // in addStreamTasks. - firstException.compareAndSet(null, active.closeAllRestoringTasks()); - changelogReader.reset(); + final Set revokedTasks = partitionsToTaskSet(revokedPartitions); - firstException.compareAndSet(null, standby.suspend()); + firstException.compareAndSet(null, active.suspendOrCloseTasks(revokedTasks, revokedChangelogs)); - // remove the changelog partitions from restore consumer - restoreConsumer.unsubscribe(); + changelogReader.remove(revokedChangelogs); + removeChangelogsFromRestoreConsumer(revokedChangelogs, false); final Exception exception = firstException.get(); if (exception != null) { throw new StreamsException(logPrefix + "failed to suspend stream tasks", exception); } + return active.suspendedTaskIds(); + } + + /** + * Closes active tasks as zombies, as these partitions have been lost and are no longer owned. + * @return list of lost tasks + */ + Set closeLostTasks(final Collection lostPartitions) { + final Set zombieTasks = partitionsToTaskSet(lostPartitions); + log.debug("Closing lost tasks as zombies: {}", zombieTasks); + + final List lostTaskChangelogs = new ArrayList<>(); + + final RuntimeException exception = active.closeZombieTasks(zombieTasks, lostTaskChangelogs); + + assignedActiveTasks.keySet().removeAll(zombieTasks); + changelogReader.remove(lostTaskChangelogs); + removeChangelogsFromRestoreConsumer(lostTaskChangelogs, false); + + if (exception != null) { + throw exception; + } else if (!assignedActiveTasks.isEmpty()) { + throw new IllegalStateException("TaskManager had leftover tasks after removing all zombies"); + } + + return zombieTasks; } void shutdown(final boolean clean) { final AtomicReference firstException = new AtomicReference<>(null); - log.debug("Shutting down all active tasks {}, standby tasks {}, suspended tasks {}, and suspended standby tasks {}", active.runningTaskIds(), standby.runningTaskIds(), - active.previousTaskIds(), standby.previousTaskIds()); + log.debug("Shutting down all active tasks {}, standby tasks {}, and suspended tasks {}", active.runningTaskIds(), standby.runningTaskIds(), + active.suspendedTaskIds()); try { active.close(clean); @@ -288,16 +304,38 @@ void shutdown(final boolean clean) { } } - Admin getAdminClient() { - return adminClient; + Set activeTaskIds() { + return active.allAssignedTaskIds(); } - Set suspendedActiveTaskIds() { - return active.previousTaskIds(); + Set standbyTaskIds() { + return standby.allAssignedTaskIds(); } - Set suspendedStandbyTaskIds() { - return standby.previousTaskIds(); + Set revokedActiveTaskIds() { + return revokedActiveTasks.keySet(); + } + + Set revokedStandbyTaskIds() { + return revokedStandbyTasks.keySet(); + } + + public Set previousRunningTaskIds() { + return active.previousRunningTaskIds(); + } + + Set previousActiveTaskIds() { + final HashSet previousActiveTasks = new HashSet<>(assignedActiveTasks.keySet()); + previousActiveTasks.addAll(revokedActiveTasks.keySet()); + previousActiveTasks.removeAll(addedActiveTasks.keySet()); + return previousActiveTasks; + } + + Set previousStandbyTaskIds() { + final HashSet previousStandbyTasks = new HashSet<>(assignedStandbyTasks.keySet()); + previousStandbyTasks.addAll(revokedStandbyTasks.keySet()); + previousStandbyTasks.removeAll(addedStandbyTasks.keySet()); + return previousStandbyTasks; } StreamTask activeTask(final TopicPartition partition) { @@ -320,6 +358,14 @@ void setConsumer(final Consumer consumer) { this.consumer = consumer; } + public UUID processId() { + return processId; + } + + InternalTopologyBuilder builder() { + return taskCreator.builder(); + } + /** * @throws IllegalStateException If store gets registered after initialized is already finished * @throws StreamsException if the store's change log does not contain the partition @@ -329,8 +375,8 @@ boolean updateNewAndRestoringTasks() { standby.initializeNewTasks(); final Collection restored = changelogReader.restore(active); - active.updateRestored(restored); + removeChangelogsFromRestoreConsumer(restored, false); if (active.allTasksRunning()) { final Set assignment = consumer.assignment(); @@ -357,6 +403,7 @@ private void assignStandbyPartitions() { checkpointedOffsets.putAll(standbyTask.checkpointedOffsets()); } + restoreConsumerAssignedStandbys = true; restoreConsumer.assign(checkpointedOffsets.keySet()); for (final Map.Entry entry : checkpointedOffsets.entrySet()) { final TopicPartition partition = entry.getKey(); @@ -377,8 +424,40 @@ public void setPartitionsByHostState(final Map> pa this.streamsMetadataState.onChange(partitionsByHostState, cluster); } + public void setPartitionsToTaskId(final Map partitionsToTaskId) { + this.partitionsToTaskId = partitionsToTaskId; + } + public void setAssignmentMetadata(final Map> activeTasks, final Map> standbyTasks) { + addedActiveTasks.clear(); + for (final Map.Entry> entry : activeTasks.entrySet()) { + if (!assignedActiveTasks.containsKey(entry.getKey())) { + addedActiveTasks.put(entry.getKey(), entry.getValue()); + } + } + + addedStandbyTasks.clear(); + for (final Map.Entry> entry : standbyTasks.entrySet()) { + if (!assignedStandbyTasks.containsKey(entry.getKey())) { + addedStandbyTasks.put(entry.getKey(), entry.getValue()); + } + } + + revokedActiveTasks.clear(); + for (final Map.Entry> entry : assignedActiveTasks.entrySet()) { + if (!activeTasks.containsKey(entry.getKey())) { + revokedActiveTasks.put(entry.getKey(), entry.getValue()); + } + } + + revokedStandbyTasks.clear(); + for (final Map.Entry> entry : assignedStandbyTasks.entrySet()) { + if (!standbyTasks.containsKey(entry.getKey())) { + revokedStandbyTasks.put(entry.getKey(), entry.getValue()); + } + } + this.assignedActiveTasks = activeTasks; this.assignedStandbyTasks = standbyTasks; } @@ -482,6 +561,30 @@ public String toString(final String indent) { return builder.toString(); } + // this should be safe to call whether the restore consumer is assigned standby or active restoring partitions + // as the removal will be a no-op + private void removeChangelogsFromRestoreConsumer(final Collection changelogs, final boolean areStandbyPartitions) { + if (!changelogs.isEmpty() && areStandbyPartitions == restoreConsumerAssignedStandbys) { + final Set updatedAssignment = new HashSet<>(restoreConsumer.assignment()); + updatedAssignment.removeAll(changelogs); + restoreConsumer.assign(updatedAssignment); + } + } + + private Set partitionsToTaskSet(final Collection partitions) { + final Set taskIds = new HashSet<>(); + for (final TopicPartition tp : partitions) { + final TaskId id = partitionsToTaskId.get(tp); + if (id != null) { + taskIds.add(id); + } else { + log.error("Failed to lookup taskId for partition {}", tp); + throw new StreamsException("Found partition in assignment with no corresponding task"); + } + } + return taskIds; + } + // the following functions are for testing only Map> assignedActiveTasks() { return assignedActiveTasks; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java index 5c668cbd8f76..080c6145f663 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.utils.LogContext; @@ -131,6 +132,29 @@ public TaskManager getTaskManager() { return taskManager; } + public RebalanceProtocol rebalanceProtocol() { + final String upgradeFrom = streamsConfig.getString(StreamsConfig.UPGRADE_FROM_CONFIG); + if (upgradeFrom != null) { + switch (upgradeFrom) { + case StreamsConfig.UPGRADE_FROM_0100: + case StreamsConfig.UPGRADE_FROM_0101: + case StreamsConfig.UPGRADE_FROM_0102: + case StreamsConfig.UPGRADE_FROM_0110: + case StreamsConfig.UPGRADE_FROM_10: + case StreamsConfig.UPGRADE_FROM_11: + case StreamsConfig.UPGRADE_FROM_20: + case StreamsConfig.UPGRADE_FROM_21: + case StreamsConfig.UPGRADE_FROM_22: + case StreamsConfig.UPGRADE_FROM_23: + log.info("Turning off cooperative rebalancing for upgrade from {}.x", upgradeFrom); + return RebalanceProtocol.EAGER; + default: + throw new IllegalArgumentException("Unknown configuration value for parameter 'upgrade.from': " + upgradeFrom); + } + } + return RebalanceProtocol.EAGER; + } + public String logPrefix() { return logPrefix; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java index 34338e1502d9..0341ab768cdc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java @@ -25,11 +25,14 @@ import org.apache.kafka.common.metrics.stats.CumulativeCount; import org.apache.kafka.common.metrics.stats.CumulativeSum; import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Min; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.metrics.stats.Value; import org.apache.kafka.common.metrics.stats.WindowedCount; import org.apache.kafka.common.metrics.stats.WindowedSum; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecordingTrigger; import java.util.Arrays; import java.util.Collections; @@ -42,22 +45,33 @@ import java.util.concurrent.TimeUnit; public class StreamsMetricsImpl implements StreamsMetrics { + + public enum Version { + LATEST, + FROM_100_TO_23 + } + private final Metrics metrics; private final Map parentSensors; private final String threadName; + private final Version version; private final Deque threadLevelSensors = new LinkedList<>(); private final Map> taskLevelSensors = new HashMap<>(); private final Map> nodeLevelSensors = new HashMap<>(); private final Map> cacheLevelSensors = new HashMap<>(); private final Map> storeLevelSensors = new HashMap<>(); + private RocksDBMetricsRecordingTrigger rocksDBMetricsRecordingTrigger; + private static final String SENSOR_PREFIX_DELIMITER = "."; private static final String SENSOR_NAME_DELIMITER = ".s."; - public static final String THREAD_ID_TAG = "client-id"; + public static final String THREAD_ID_TAG = "thread-id"; + public static final String THREAD_ID_TAG_0100_TO_23 = "client-id"; public static final String TASK_ID_TAG = "task-id"; public static final String STORE_ID_TAG = "state-id"; + public static final String RECORD_CACHE_ID_TAG = "record-cache-id"; public static final String ROLLUP_VALUE = "all"; @@ -69,6 +83,11 @@ public class StreamsMetricsImpl implements StreamsMetrics { public static final String TOTAL_SUFFIX = "-total"; public static final String RATIO_SUFFIX = "-ratio"; + public static final String AVG_VALUE_DOC = "The average value of "; + public static final String MAX_VALUE_DOC = "The maximum value of "; + public static final String AVG_LATENCY_DOC = "The average latency of "; + public static final String MAX_LATENCY_DOC = "The maximum latency of "; + public static final String GROUP_PREFIX_WO_DELIMITER = "stream"; public static final String GROUP_PREFIX = GROUP_PREFIX_WO_DELIMITER + "-"; public static final String GROUP_SUFFIX = "-metrics"; @@ -76,6 +95,7 @@ public class StreamsMetricsImpl implements StreamsMetrics { public static final String THREAD_LEVEL_GROUP = GROUP_PREFIX_WO_DELIMITER + GROUP_SUFFIX; public static final String TASK_LEVEL_GROUP = GROUP_PREFIX + "task" + GROUP_SUFFIX; public static final String STATE_LEVEL_GROUP = GROUP_PREFIX + "state" + GROUP_SUFFIX; + public static final String CACHE_LEVEL_GROUP = GROUP_PREFIX + "record-cache" + GROUP_SUFFIX; public static final String PROCESSOR_NODE_METRICS_GROUP = "stream-processor-node-metrics"; public static final String PROCESSOR_NODE_ID_TAG = "processor-node-id"; @@ -83,14 +103,36 @@ public class StreamsMetricsImpl implements StreamsMetrics { public static final String EXPIRED_WINDOW_RECORD_DROP = "expired-window-record-drop"; public static final String LATE_RECORD_DROP = "late-record-drop"; - public StreamsMetricsImpl(final Metrics metrics, final String threadName) { + public StreamsMetricsImpl(final Metrics metrics, final String threadName, final String builtInMetricsVersion) { Objects.requireNonNull(metrics, "Metrics cannot be null"); + Objects.requireNonNull(builtInMetricsVersion, "Built-in metrics version cannot be null"); this.metrics = metrics; this.threadName = threadName; + this.version = parseBuiltInMetricsVersion(builtInMetricsVersion); this.parentSensors = new HashMap<>(); } + private static Version parseBuiltInMetricsVersion(final String builtInMetricsVersion) { + if (builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST)) { + return Version.LATEST; + } else { + return Version.FROM_100_TO_23; + } + } + + public Version version() { + return version; + } + + public void setRocksDBMetricsRecordingTrigger(final RocksDBMetricsRecordingTrigger rocksDBMetricsRecordingTrigger) { + this.rocksDBMetricsRecordingTrigger = rocksDBMetricsRecordingTrigger; + } + + public RocksDBMetricsRecordingTrigger rocksDBMetricsRecordingTrigger() { + return rocksDBMetricsRecordingTrigger; + } + public final Sensor threadLevelSensor(final String sensorName, final RecordingLevel recordingLevel, final Sensor... parents) { @@ -108,7 +150,7 @@ private String threadSensorPrefix() { public Map threadLevelTagMap() { final Map tagMap = new LinkedHashMap<>(); - tagMap.put(THREAD_ID_TAG, threadName); + tagMap.put(THREAD_ID_TAG_0100_TO_23, threadName); return tagMap; } @@ -215,12 +257,12 @@ private String nodeSensorPrefix(final String taskName, final String processorNod return taskSensorPrefix(taskName) + SENSOR_PREFIX_DELIMITER + "node" + SENSOR_PREFIX_DELIMITER + processorNodeName; } - public final Sensor cacheLevelSensor(final String taskName, - final String cacheName, - final String sensorName, - final Sensor.RecordingLevel recordingLevel, - final Sensor... parents) { - final String key = cacheSensorPrefix(taskName, cacheName); + public Sensor cacheLevelSensor(final String taskName, + final String storeName, + final String sensorName, + final Sensor.RecordingLevel recordingLevel, + final Sensor... parents) { + final String key = cacheSensorPrefix(taskName, storeName); synchronized (cacheLevelSensors) { if (!cacheLevelSensors.containsKey(key)) { cacheLevelSensors.put(key, new LinkedList<>()); @@ -236,6 +278,18 @@ public final Sensor cacheLevelSensor(final String taskName, } } + public Map cacheLevelTagMap(final String taskName, final String storeName) { + final Map tagMap = new LinkedHashMap<>(); + tagMap.put(TASK_ID_TAG, taskName); + tagMap.put(RECORD_CACHE_ID_TAG, storeName); + if (version == Version.FROM_100_TO_23) { + tagMap.put(THREAD_ID_TAG_0100_TO_23, Thread.currentThread().getName()); + } else { + tagMap.put(THREAD_ID_TAG, Thread.currentThread().getName()); + } + return tagMap; + } + public final void removeAllCacheLevelSensors(final String taskName, final String cacheName) { final String key = cacheSensorPrefix(taskName, cacheName); synchronized (cacheLevelSensors) { @@ -401,15 +455,17 @@ private String externalParentSensorName(final String operationName) { } - public static void addAvgAndMaxToSensor(final Sensor sensor, + private static void addAvgAndMaxToSensor(final Sensor sensor, final String group, final Map tags, - final String operation) { + final String operation, + final String descriptionOfAvg, + final String descriptionOfMax) { sensor.add( new MetricName( operation + AVG_SUFFIX, group, - "The average value of " + operation + ".", + descriptionOfAvg, tags), new Avg() ); @@ -417,12 +473,26 @@ public static void addAvgAndMaxToSensor(final Sensor sensor, new MetricName( operation + MAX_SUFFIX, group, - "The max value of " + operation + ".", + descriptionOfMax, tags), new Max() ); } + public static void addAvgAndMaxToSensor(final Sensor sensor, + final String group, + final Map tags, + final String operation) { + addAvgAndMaxToSensor( + sensor, + group, + tags, + operation, + AVG_VALUE_DOC + operation + ".", + MAX_VALUE_DOC + operation + "." + ); + } + public static void addAvgAndMaxLatencyToSensor(final Sensor sensor, final String group, final Map tags, @@ -431,7 +501,7 @@ public static void addAvgAndMaxLatencyToSensor(final Sensor sensor, new MetricName( operation + "-latency-avg", group, - "The average latency of " + operation + " operation.", + AVG_LATENCY_DOC + operation + " operation.", tags), new Avg() ); @@ -439,12 +509,30 @@ public static void addAvgAndMaxLatencyToSensor(final Sensor sensor, new MetricName( operation + "-latency-max", group, - "The max latency of " + operation + " operation.", + MAX_LATENCY_DOC + operation + " operation.", tags), new Max() ); } + public static void addAvgAndMinAndMaxToSensor(final Sensor sensor, + final String group, + final Map tags, + final String operation, + final String descriptionOfAvg, + final String descriptionOfMin, + final String descriptionOfMax) { + addAvgAndMaxToSensor(sensor, group, tags, operation, descriptionOfAvg, descriptionOfMax); + sensor.add( + new MetricName( + operation + MIN_SUFFIX, + group, + descriptionOfMin, + tags), + new Min() + ); + } + public static void addInvocationRateAndCountToSensor(final Sensor sensor, final String group, final Map tags, @@ -509,7 +597,24 @@ public static void addSumMetricToSensor(final Sensor sensor, final Map tags, final String operation, final String description) { - sensor.add(new MetricName(operation + TOTAL_SUFFIX, group, description, tags), new CumulativeSum()); + addSumMetricToSensor(sensor, group, tags, operation, true, description); + } + + public static void addSumMetricToSensor(final Sensor sensor, + final String group, + final Map tags, + final String operation, + final boolean withSuffix, + final String description) { + sensor.add( + new MetricName( + withSuffix ? operation + TOTAL_SUFFIX : operation, + group, + description, + tags + ), + new CumulativeSum() + ); } public static void addValueMetricToSensor(final Sensor sensor, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java index 773589375462..01510c14790a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java @@ -24,7 +24,7 @@ */ class KeyValueSegments extends AbstractSegments { - final private RocksDBMetricsRecorder metricsRecorder; + private final RocksDBMetricsRecorder metricsRecorder; KeyValueSegments(final String name, final String metricsScope, @@ -51,10 +51,4 @@ public KeyValueSegment getOrCreateSegment(final long segmentId, return newSegment; } } - - @Override - public void close() { - metricsRecorder.close(); - super.close(); - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Murmur3.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Murmur3.java new file mode 100644 index 000000000000..5581a0364831 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Murmur3.java @@ -0,0 +1,548 @@ +/* + * 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.streams.state.internals; + +/** + * This class was taken from Hive org.apache.hive.common.util; + * https://github.com/apache/hive/blob/master/storage-api/src/java/org/apache/hive/common/util/Murmur3.java + * Commit: dffa3a16588bc8e95b9d0ab5af295a74e06ef702 + * + * + * Murmur3 is successor to Murmur2 fast non-crytographic hash algorithms. + * + * Murmur3 32 and 128 bit variants. + * 32-bit Java port of https://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp#94 + * 128-bit Java port of https://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp#255 + * + * This is a public domain code with no copyrights. + * From homepage of MurmurHash (https://code.google.com/p/smhasher/), + * "All MurmurHash versions are public domain software, and the author disclaims all copyright + * to their code." + */ +@SuppressWarnings("fallthrough") +public class Murmur3 { + // from 64-bit linear congruential generator + public static final long NULL_HASHCODE = 2862933555777941757L; + + // Constants for 32 bit variant + private static final int C1_32 = 0xcc9e2d51; + private static final int C2_32 = 0x1b873593; + private static final int R1_32 = 15; + private static final int R2_32 = 13; + private static final int M_32 = 5; + private static final int N_32 = 0xe6546b64; + + // Constants for 128 bit variant + private static final long C1 = 0x87c37b91114253d5L; + private static final long C2 = 0x4cf5ad432745937fL; + private static final int R1 = 31; + private static final int R2 = 27; + private static final int R3 = 33; + private static final int M = 5; + private static final int N1 = 0x52dce729; + private static final int N2 = 0x38495ab5; + + public static final int DEFAULT_SEED = 104729; + + public static int hash32(long l0, long l1) { + return hash32(l0, l1, DEFAULT_SEED); + } + + public static int hash32(long l0) { + return hash32(l0, DEFAULT_SEED); + } + + /** + * Murmur3 32-bit variant. + */ + public static int hash32(long l0, int seed) { + int hash = seed; + final long r0 = Long.reverseBytes(l0); + + hash = mix32((int) r0, hash); + hash = mix32((int) (r0 >>> 32), hash); + + return fmix32(Long.BYTES, hash); + } + + /** + * Murmur3 32-bit variant. + */ + public static int hash32(long l0, long l1, int seed) { + int hash = seed; + final long r0 = Long.reverseBytes(l0); + final long r1 = Long.reverseBytes(l1); + + hash = mix32((int) r0, hash); + hash = mix32((int) (r0 >>> 32), hash); + hash = mix32((int) (r1), hash); + hash = mix32((int) (r1 >>> 32), hash); + + return fmix32(Long.BYTES * 2, hash); + } + + /** + * Murmur3 32-bit variant. + * + * @param data - input byte array + * @return - hashcode + */ + public static int hash32(byte[] data) { + return hash32(data, 0, data.length, DEFAULT_SEED); + } + + /** + * Murmur3 32-bit variant. + * + * @param data - input byte array + * @param length - length of array + * @return - hashcode + */ + public static int hash32(byte[] data, int length) { + return hash32(data, 0, length, DEFAULT_SEED); + } + + /** + * Murmur3 32-bit variant. + * + * @param data - input byte array + * @param length - length of array + * @param seed - seed. (default 0) + * @return - hashcode + */ + public static int hash32(byte[] data, int length, int seed) { + return hash32(data, 0, length, seed); + } + + /** + * Murmur3 32-bit variant. + * + * @param data - input byte array + * @param offset - offset of data + * @param length - length of array + * @param seed - seed. (default 0) + * @return - hashcode + */ + public static int hash32(byte[] data, int offset, int length, int seed) { + int hash = seed; + final int nblocks = length >> 2; + + // body + for (int i = 0; i < nblocks; i++) { + int i_4 = i << 2; + int k = (data[offset + i_4] & 0xff) + | ((data[offset + i_4 + 1] & 0xff) << 8) + | ((data[offset + i_4 + 2] & 0xff) << 16) + | ((data[offset + i_4 + 3] & 0xff) << 24); + + hash = mix32(k, hash); + } + + // tail + int idx = nblocks << 2; + int k1 = 0; + switch (length - idx) { + case 3: + k1 ^= data[offset + idx + 2] << 16; + case 2: + k1 ^= data[offset + idx + 1] << 8; + case 1: + k1 ^= data[offset + idx]; + + // mix functions + k1 *= C1_32; + k1 = Integer.rotateLeft(k1, R1_32); + k1 *= C2_32; + hash ^= k1; + } + + return fmix32(length, hash); + } + + private static int mix32(int k, int hash) { + k *= C1_32; + k = Integer.rotateLeft(k, R1_32); + k *= C2_32; + hash ^= k; + return Integer.rotateLeft(hash, R2_32) * M_32 + N_32; + } + + private static int fmix32(int length, int hash) { + hash ^= length; + hash ^= (hash >>> 16); + hash *= 0x85ebca6b; + hash ^= (hash >>> 13); + hash *= 0xc2b2ae35; + hash ^= (hash >>> 16); + + return hash; + } + + /** + * Murmur3 64-bit variant. This is essentially MSB 8 bytes of Murmur3 128-bit variant. + * + * @param data - input byte array + * @return - hashcode + */ + public static long hash64(byte[] data) { + return hash64(data, 0, data.length, DEFAULT_SEED); + } + + public static long hash64(long data) { + long hash = DEFAULT_SEED; + long k = Long.reverseBytes(data); + int length = Long.BYTES; + // mix functions + k *= C1; + k = Long.rotateLeft(k, R1); + k *= C2; + hash ^= k; + hash = Long.rotateLeft(hash, R2) * M + N1; + // finalization + hash ^= length; + hash = fmix64(hash); + return hash; + } + + public static long hash64(int data) { + long k1 = Integer.reverseBytes(data) & (-1L >>> 32); + int length = Integer.BYTES; + long hash = DEFAULT_SEED; + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + hash ^= k1; + // finalization + hash ^= length; + hash = fmix64(hash); + return hash; + } + + public static long hash64(short data) { + long hash = DEFAULT_SEED; + long k1 = 0; + k1 ^= ((long) data & 0xff) << 8; + k1 ^= ((long)((data & 0xFF00) >> 8) & 0xff); + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + hash ^= k1; + + // finalization + hash ^= Short.BYTES; + hash = fmix64(hash); + return hash; + } + + public static long hash64(byte[] data, int offset, int length) { + return hash64(data, offset, length, DEFAULT_SEED); + } + + /** + * Murmur3 64-bit variant. This is essentially MSB 8 bytes of Murmur3 128-bit variant. + * + * @param data - input byte array + * @param length - length of array + * @param seed - seed. (default is 0) + * @return - hashcode + */ + public static long hash64(byte[] data, int offset, int length, int seed) { + long hash = seed; + final int nblocks = length >> 3; + + // body + for (int i = 0; i < nblocks; i++) { + final int i8 = i << 3; + long k = ((long) data[offset + i8] & 0xff) + | (((long) data[offset + i8 + 1] & 0xff) << 8) + | (((long) data[offset + i8 + 2] & 0xff) << 16) + | (((long) data[offset + i8 + 3] & 0xff) << 24) + | (((long) data[offset + i8 + 4] & 0xff) << 32) + | (((long) data[offset + i8 + 5] & 0xff) << 40) + | (((long) data[offset + i8 + 6] & 0xff) << 48) + | (((long) data[offset + i8 + 7] & 0xff) << 56); + + // mix functions + k *= C1; + k = Long.rotateLeft(k, R1); + k *= C2; + hash ^= k; + hash = Long.rotateLeft(hash, R2) * M + N1; + } + + // tail + long k1 = 0; + int tailStart = nblocks << 3; + switch (length - tailStart) { + case 7: + k1 ^= ((long) data[offset + tailStart + 6] & 0xff) << 48; + case 6: + k1 ^= ((long) data[offset + tailStart + 5] & 0xff) << 40; + case 5: + k1 ^= ((long) data[offset + tailStart + 4] & 0xff) << 32; + case 4: + k1 ^= ((long) data[offset + tailStart + 3] & 0xff) << 24; + case 3: + k1 ^= ((long) data[offset + tailStart + 2] & 0xff) << 16; + case 2: + k1 ^= ((long) data[offset + tailStart + 1] & 0xff) << 8; + case 1: + k1 ^= ((long) data[offset + tailStart] & 0xff); + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + hash ^= k1; + } + + // finalization + hash ^= length; + hash = fmix64(hash); + + return hash; + } + + /** + * Murmur3 128-bit variant. + * + * @param data - input byte array + * @return - hashcode (2 longs) + */ + public static long[] hash128(byte[] data) { + return hash128(data, 0, data.length, DEFAULT_SEED); + } + + /** + * Murmur3 128-bit variant. + * + * @param data - input byte array + * @param offset - the first element of array + * @param length - length of array + * @param seed - seed. (default is 0) + * @return - hashcode (2 longs) + */ + public static long[] hash128(byte[] data, int offset, int length, int seed) { + long h1 = seed; + long h2 = seed; + final int nblocks = length >> 4; + + // body + for (int i = 0; i < nblocks; i++) { + final int i16 = i << 4; + long k1 = ((long) data[offset + i16] & 0xff) + | (((long) data[offset + i16 + 1] & 0xff) << 8) + | (((long) data[offset + i16 + 2] & 0xff) << 16) + | (((long) data[offset + i16 + 3] & 0xff) << 24) + | (((long) data[offset + i16 + 4] & 0xff) << 32) + | (((long) data[offset + i16 + 5] & 0xff) << 40) + | (((long) data[offset + i16 + 6] & 0xff) << 48) + | (((long) data[offset + i16 + 7] & 0xff) << 56); + + long k2 = ((long) data[offset + i16 + 8] & 0xff) + | (((long) data[offset + i16 + 9] & 0xff) << 8) + | (((long) data[offset + i16 + 10] & 0xff) << 16) + | (((long) data[offset + i16 + 11] & 0xff) << 24) + | (((long) data[offset + i16 + 12] & 0xff) << 32) + | (((long) data[offset + i16 + 13] & 0xff) << 40) + | (((long) data[offset + i16 + 14] & 0xff) << 48) + | (((long) data[offset + i16 + 15] & 0xff) << 56); + + // mix functions for k1 + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + h1 ^= k1; + h1 = Long.rotateLeft(h1, R2); + h1 += h2; + h1 = h1 * M + N1; + + // mix functions for k2 + k2 *= C2; + k2 = Long.rotateLeft(k2, R3); + k2 *= C1; + h2 ^= k2; + h2 = Long.rotateLeft(h2, R1); + h2 += h1; + h2 = h2 * M + N2; + } + + // tail + long k1 = 0; + long k2 = 0; + int tailStart = nblocks << 4; + switch (length - tailStart) { + case 15: + k2 ^= (long) (data[offset + tailStart + 14] & 0xff) << 48; + case 14: + k2 ^= (long) (data[offset + tailStart + 13] & 0xff) << 40; + case 13: + k2 ^= (long) (data[offset + tailStart + 12] & 0xff) << 32; + case 12: + k2 ^= (long) (data[offset + tailStart + 11] & 0xff) << 24; + case 11: + k2 ^= (long) (data[offset + tailStart + 10] & 0xff) << 16; + case 10: + k2 ^= (long) (data[offset + tailStart + 9] & 0xff) << 8; + case 9: + k2 ^= (long) (data[offset + tailStart + 8] & 0xff); + k2 *= C2; + k2 = Long.rotateLeft(k2, R3); + k2 *= C1; + h2 ^= k2; + + case 8: + k1 ^= (long) (data[offset + tailStart + 7] & 0xff) << 56; + case 7: + k1 ^= (long) (data[offset + tailStart + 6] & 0xff) << 48; + case 6: + k1 ^= (long) (data[offset + tailStart + 5] & 0xff) << 40; + case 5: + k1 ^= (long) (data[offset + tailStart + 4] & 0xff) << 32; + case 4: + k1 ^= (long) (data[offset + tailStart + 3] & 0xff) << 24; + case 3: + k1 ^= (long) (data[offset + tailStart + 2] & 0xff) << 16; + case 2: + k1 ^= (long) (data[offset + tailStart + 1] & 0xff) << 8; + case 1: + k1 ^= (long) (data[offset + tailStart] & 0xff); + k1 *= C1; + k1 = Long.rotateLeft(k1, R1); + k1 *= C2; + h1 ^= k1; + } + + // finalization + h1 ^= length; + h2 ^= length; + + h1 += h2; + h2 += h1; + + h1 = fmix64(h1); + h2 = fmix64(h2); + + h1 += h2; + h2 += h1; + + return new long[]{h1, h2}; + } + + private static long fmix64(long h) { + h ^= (h >>> 33); + h *= 0xff51afd7ed558ccdL; + h ^= (h >>> 33); + h *= 0xc4ceb9fe1a85ec53L; + h ^= (h >>> 33); + return h; + } + + public static class IncrementalHash32 { + byte[] tail = new byte[3]; + int tailLen; + int totalLen; + int hash; + + public final void start(int hash) { + tailLen = totalLen = 0; + this.hash = hash; + } + + public final void add(byte[] data, int offset, int length) { + if (length == 0) return; + totalLen += length; + if (tailLen + length < 4) { + System.arraycopy(data, offset, tail, tailLen, length); + tailLen += length; + return; + } + int offset2 = 0; + if (tailLen > 0) { + offset2 = (4 - tailLen); + int k = -1; + switch (tailLen) { + case 1: + k = orBytes(tail[0], data[offset], data[offset + 1], data[offset + 2]); + break; + case 2: + k = orBytes(tail[0], tail[1], data[offset], data[offset + 1]); + break; + case 3: + k = orBytes(tail[0], tail[1], tail[2], data[offset]); + break; + default: throw new AssertionError(tailLen); + } + // mix functions + k *= C1_32; + k = Integer.rotateLeft(k, R1_32); + k *= C2_32; + hash ^= k; + hash = Integer.rotateLeft(hash, R2_32) * M_32 + N_32; + } + int length2 = length - offset2; + offset += offset2; + final int nblocks = length2 >> 2; + + for (int i = 0; i < nblocks; i++) { + int i_4 = (i << 2) + offset; + int k = orBytes(data[i_4], data[i_4 + 1], data[i_4 + 2], data[i_4 + 3]); + + // mix functions + k *= C1_32; + k = Integer.rotateLeft(k, R1_32); + k *= C2_32; + hash ^= k; + hash = Integer.rotateLeft(hash, R2_32) * M_32 + N_32; + } + + int consumed = (nblocks << 2); + tailLen = length2 - consumed; + if (consumed == length2) return; + System.arraycopy(data, offset + consumed, tail, 0, tailLen); + } + + public final int end() { + int k1 = 0; + switch (tailLen) { + case 3: + k1 ^= tail[2] << 16; + case 2: + k1 ^= tail[1] << 8; + case 1: + k1 ^= tail[0]; + + // mix functions + k1 *= C1_32; + k1 = Integer.rotateLeft(k1, R1_32); + k1 *= C2_32; + hash ^= k1; + } + + // finalization + hash ^= totalLen; + hash ^= (hash >>> 16); + hash *= 0x85ebca6b; + hash ^= (hash >>> 13); + hash *= 0xc2b2ae35; + hash ^= (hash >>> 16); + return hash; + } + } + + private static int orBytes(byte b1, byte b2, byte b3, byte b4) { + return (b1 & 0xff) | ((b2 & 0xff) << 8) | ((b3 & 0xff) << 16) | ((b4 & 0xff) << 24); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java index a1d0aab27705..2d53440ef472 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java @@ -19,14 +19,12 @@ import java.util.NavigableMap; import java.util.TreeMap; import java.util.TreeSet; -import org.apache.kafka.common.MetricName; + import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.metrics.stats.Avg; -import org.apache.kafka.common.metrics.stats.Max; -import org.apache.kafka.common.metrics.stats.Min; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.internals.metrics.NamedCacheMetrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,19 +32,22 @@ import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; -import java.util.Map; import java.util.Set; class NamedCache { private static final Logger log = LoggerFactory.getLogger(NamedCache.class); private final String name; + private final String storeName; + private final String taskName; private final NavigableMap cache = new TreeMap<>(); private final Set dirtyKeys = new LinkedHashSet<>(); private ThreadCache.DirtyEntryFlushListener listener; private LRUNode tail; private LRUNode head; private long currentSizeBytes; - private final NamedCacheMetrics namedCacheMetrics; + + private final StreamsMetricsImpl streamsMetrics; + private final Sensor hitRatioSensor; // internal stats private long numReadHits = 0; @@ -54,9 +55,12 @@ class NamedCache { private long numOverwrites = 0; private long numFlushes = 0; - NamedCache(final String name, final StreamsMetricsImpl metrics) { + NamedCache(final String name, final StreamsMetricsImpl streamsMetrics) { this.name = name; - this.namedCacheMetrics = new NamedCacheMetrics(metrics, name); + this.streamsMetrics = streamsMetrics; + storeName = ThreadCache.underlyingStoreNamefromCacheName(name); + taskName = ThreadCache.taskIDfromCacheName(name); + hitRatioSensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, taskName, storeName); } synchronized final String name() { @@ -187,7 +191,7 @@ private LRUNode getInternal(final Bytes key) { return null; } else { numReadHits++; - namedCacheMetrics.hitRatioSensor.record((double) numReadHits / (double) (numReadHits + numReadMisses)); + hitRatioSensor.record((double) numReadHits / (double) (numReadHits + numReadMisses)); } return node; } @@ -311,7 +315,7 @@ synchronized void close() { currentSizeBytes = 0; dirtyKeys.clear(); cache.clear(); - namedCacheMetrics.removeAllSensors(); + streamsMetrics.removeAllCacheLevelSensors(taskName, storeName); } /** @@ -357,68 +361,4 @@ private void update(final LRUCacheEntry entry) { } } - private static class NamedCacheMetrics { - private final StreamsMetricsImpl metrics; - - private final Sensor hitRatioSensor; - private final String taskName; - private final String cacheName; - - private NamedCacheMetrics(final StreamsMetricsImpl metrics, final String cacheName) { - taskName = ThreadCache.taskIDfromCacheName(cacheName); - this.cacheName = cacheName; - this.metrics = metrics; - final String group = "stream-record-cache-metrics"; - - // add parent - final Map allMetricTags = metrics.tagMap( - "task-id", taskName, - "record-cache-id", "all" - ); - final Sensor taskLevelHitRatioSensor = metrics.taskLevelSensor(taskName, "hitRatio", Sensor.RecordingLevel.DEBUG); - taskLevelHitRatioSensor.add( - new MetricName("hitRatio-avg", group, "The average cache hit ratio.", allMetricTags), - new Avg() - ); - taskLevelHitRatioSensor.add( - new MetricName("hitRatio-min", group, "The minimum cache hit ratio.", allMetricTags), - new Min() - ); - taskLevelHitRatioSensor.add( - new MetricName("hitRatio-max", group, "The maximum cache hit ratio.", allMetricTags), - new Max() - ); - - // add child - final Map metricTags = metrics.tagMap( - "task-id", taskName, - "record-cache-id", ThreadCache.underlyingStoreNamefromCacheName(cacheName) - ); - - hitRatioSensor = metrics.cacheLevelSensor( - taskName, - cacheName, - "hitRatio", - Sensor.RecordingLevel.DEBUG, - taskLevelHitRatioSensor - ); - hitRatioSensor.add( - new MetricName("hitRatio-avg", group, "The average cache hit ratio.", metricTags), - new Avg() - ); - hitRatioSensor.add( - new MetricName("hitRatio-min", group, "The minimum cache hit ratio.", metricTags), - new Min() - ); - hitRatioSensor.add( - new MetricName("hitRatio-max", group, "The maximum cache hit ratio.", metricTags), - new Max() - ); - - } - - private void removeAllSensors() { - metrics.removeAllCacheLevelSensors(taskName, cacheName); - } - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 4e56868e6b9e..74ccd04becfb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -104,8 +104,7 @@ public class RocksDBStore implements KeyValueStore, BulkLoadingSt private RocksDBConfigSetter configSetter; private final RocksDBMetricsRecorder metricsRecorder; - private boolean closeMetricsRecorder = false; - private boolean removeStatisticsFromMetricsRecorder = false; + private boolean isStatisticsRegistered = false; private volatile boolean prepareForBulkload = false; ProcessorContext internalProcessorContext; @@ -117,7 +116,6 @@ public class RocksDBStore implements KeyValueStore, BulkLoadingSt RocksDBStore(final String name, final String metricsScope) { this(name, DB_FILE_DIR, new RocksDBMetricsRecorder(metricsScope, name)); - closeMetricsRecorder = true; } RocksDBStore(final String name, @@ -190,26 +188,21 @@ void openDB(final ProcessorContext context) { throw new ProcessorStateException(fatal); } - setUpMetrics(context, configs); + maybeSetUpMetricsRecorder(context, configs); openRocksDB(dbOptions, columnFamilyOptions); open = true; } - private void setUpMetrics(final ProcessorContext context, final Map configs) { + private void maybeSetUpMetricsRecorder(final ProcessorContext context, final Map configs) { if (userSpecifiedOptions.statistics() == null && RecordingLevel.forName((String) configs.get(METRICS_RECORDING_LEVEL_CONFIG)) == RecordingLevel.DEBUG) { + isStatisticsRegistered = true; // metrics recorder will clean up statistics object final Statistics statistics = new Statistics(); userSpecifiedOptions.setStatistics(statistics); - metricsRecorder.addStatistics( - name, - statistics, - (StreamsMetricsImpl) context.metrics(), - context.taskId() - ); - removeStatisticsFromMetricsRecorder = true; + metricsRecorder.addStatistics(name, statistics, (StreamsMetricsImpl) context.metrics(), context.taskId()); } } @@ -434,7 +427,7 @@ public synchronized void close() { configSetter = null; } - closeOrUpdateMetricsRecorder(); + maybeRemoveStatisticsFromMetricsRecorder(); // Important: do not rearrange the order in which the below objects are closed! // Order of closing must follow: ColumnFamilyHandle > RocksDB > DBOptions > ColumnFamilyOptions @@ -455,6 +448,13 @@ public synchronized void close() { cache = null; } + private void maybeRemoveStatisticsFromMetricsRecorder() { + if (isStatisticsRegistered) { + metricsRecorder.removeStatistics(name); + isStatisticsRegistered = false; + } + } + private void closeOpenIterators() { final HashSet> iterators; synchronized (openIterators) { @@ -468,14 +468,6 @@ private void closeOpenIterators() { } } - private void closeOrUpdateMetricsRecorder() { - if (closeMetricsRecorder) { - metricsRecorder.close(); - } else if (removeStatisticsFromMetricsRecorder) { - metricsRecorder.removeStatistics(name); - } - } - interface RocksDBAccessor { void put(final byte[] key, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java index eb713390002c..6545de0f691f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java @@ -51,10 +51,4 @@ public TimestampedSegment getOrCreateSegment(final long segmentId, return newSegment; } } - - @Override - public void close() { - metricsRecorder.close(); - super.close(); - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetrics.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetrics.java new file mode 100644 index 000000000000..3744428ba582 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetrics.java @@ -0,0 +1,86 @@ +/* + * 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.streams.state.internals.metrics; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; + +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.CACHE_LEVEL_GROUP; +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.ROLLUP_VALUE; +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version.FROM_100_TO_23; +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addAvgAndMinAndMaxToSensor; + +public class NamedCacheMetrics { + private NamedCacheMetrics() {} + + private static final String HIT_RATIO_0100_TO_23 = "hitRatio"; + private static final String HIT_RATIO = "hit-ratio"; + private static final String HIT_RATIO_AVG_DESCRIPTION = "The average cache hit ratio"; + private static final String HIT_RATIO_MIN_DESCRIPTION = "The minimum cache hit ratio"; + private static final String HIT_RATIO_MAX_DESCRIPTION = "The maximum cache hit ratio"; + + + public static Sensor hitRatioSensor(final StreamsMetricsImpl streamsMetrics, + final String taskName, + final String storeName) { + + final Sensor hitRatioSensor; + final String hitRatioName; + if (streamsMetrics.version() == FROM_100_TO_23) { + hitRatioName = HIT_RATIO_0100_TO_23; + final Sensor taskLevelHitRatioSensor = streamsMetrics.taskLevelSensor( + taskName, + hitRatioName, + Sensor.RecordingLevel.DEBUG + ); + addAvgAndMinAndMaxToSensor( + taskLevelHitRatioSensor, + CACHE_LEVEL_GROUP, + streamsMetrics.cacheLevelTagMap(taskName, ROLLUP_VALUE), + hitRatioName, + HIT_RATIO_AVG_DESCRIPTION, + HIT_RATIO_MIN_DESCRIPTION, + HIT_RATIO_MAX_DESCRIPTION + ); + hitRatioSensor = streamsMetrics.cacheLevelSensor( + taskName, + storeName, + hitRatioName, + Sensor.RecordingLevel.DEBUG, + taskLevelHitRatioSensor + ); + } else { + hitRatioName = HIT_RATIO; + hitRatioSensor = streamsMetrics.cacheLevelSensor( + taskName, + storeName, + hitRatioName, + Sensor.RecordingLevel.DEBUG + ); + } + addAvgAndMinAndMaxToSensor( + hitRatioSensor, + CACHE_LEVEL_GROUP, + streamsMetrics.cacheLevelTagMap(taskName, storeName), + hitRatioName, + HIT_RATIO_AVG_DESCRIPTION, + HIT_RATIO_MIN_DESCRIPTION, + HIT_RATIO_MAX_DESCRIPTION + ); + return hitRatioSensor; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetrics.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetrics.java index a0b8dc64a63d..04da4bc53ab7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetrics.java @@ -366,12 +366,13 @@ public static Sensor compactionTimeMaxSensor(final StreamsMetricsImpl streamsMet public static Sensor numberOfOpenFilesSensor(final StreamsMetricsImpl streamsMetrics, final RocksDBMetricContext metricContext) { final Sensor sensor = createSensor(streamsMetrics, metricContext, NUMBER_OF_OPEN_FILES); - addValueMetricToSensor( + addSumMetricToSensor( sensor, STATE_LEVEL_GROUP, streamsMetrics .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), NUMBER_OF_OPEN_FILES, + false, NUMBER_OF_OPEN_FILES_DESCRIPTION ); return sensor; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java index 5649f4ae06dc..de2a37c07bec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java @@ -17,117 +17,175 @@ package org.apache.kafka.streams.state.internals.metrics; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetrics.RocksDBMetricContext; import org.rocksdb.Statistics; import org.rocksdb.StatsLevel; +import org.rocksdb.TickerType; +import org.slf4j.Logger; -import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; public class RocksDBMetricsRecorder { + private final Logger logger; private Sensor bytesWrittenToDatabaseSensor; - private Sensor bytesReadToDatabaseSensor; + private Sensor bytesReadFromDatabaseSensor; private Sensor memtableBytesFlushedSensor; private Sensor memtableHitRatioSensor; - private Sensor memtableAvgFlushTimeSensor; - private Sensor memtableMinFlushTimeSensor; - private Sensor memtableMaxFlushTimeSensor; private Sensor writeStallDurationSensor; private Sensor blockCacheDataHitRatioSensor; private Sensor blockCacheIndexHitRatioSensor; private Sensor blockCacheFilterHitRatioSensor; private Sensor bytesReadDuringCompactionSensor; private Sensor bytesWrittenDuringCompactionSensor; - private Sensor compactionTimeAvgSensor; - private Sensor compactionTimeMinSensor; - private Sensor compactionTimeMaxSensor; private Sensor numberOfOpenFilesSensor; private Sensor numberOfFileErrorsSensor; - final private Map statisticsToRecord = new HashMap<>(); - final private String metricsScope; - final private String storeName; + private final Map statisticsToRecord = new ConcurrentHashMap<>(); + private final String metricsScope; + private final String storeName; + private TaskId taskId; + private StreamsMetricsImpl streamsMetrics; + private boolean isInitialized = false; public RocksDBMetricsRecorder(final String metricsScope, final String storeName) { this.metricsScope = metricsScope; this.storeName = storeName; + final LogContext logContext = new LogContext(String.format("[RocksDB Metrics Recorder for %s] ", storeName)); + logger = logContext.logger(RocksDBMetricsRecorder.class); } - private void init(final StreamsMetricsImpl streamsMetrics, final TaskId taskId) { + public String storeName() { + return storeName; + } + + public TaskId taskId() { + return taskId; + } + + public void addStatistics(final String segmentName, + final Statistics statistics, + final StreamsMetricsImpl streamsMetrics, + final TaskId taskId) { + if (!isInitialized) { + initSensors(streamsMetrics, taskId); + this.taskId = taskId; + this.streamsMetrics = streamsMetrics; + isInitialized = true; + } + if (this.taskId != taskId) { + throw new IllegalStateException("Statistics of store \"" + segmentName + "\" for task " + taskId + + " cannot be added to metrics recorder for task " + this.taskId + ". This is a bug in Kafka Streams."); + } + if (statisticsToRecord.isEmpty()) { + logger.debug( + "Adding metrics recorder of task {} to metrics recording trigger", + taskId + ); + streamsMetrics.rocksDBMetricsRecordingTrigger().addMetricsRecorder(this); + } else if (statisticsToRecord.containsKey(segmentName)) { + throw new IllegalStateException("Statistics for store \"" + segmentName + "\" of task " + taskId + + " has been already added. This is a bug in Kafka Streams."); + } + statistics.setStatsLevel(StatsLevel.EXCEPT_DETAILED_TIMERS); + logger.debug("Adding statistics for store {} of task {}", segmentName, taskId); + statisticsToRecord.put(segmentName, statistics); + } + + private void initSensors(final StreamsMetricsImpl streamsMetrics, final TaskId taskId) { final RocksDBMetricContext metricContext = new RocksDBMetricContext(taskId.toString(), metricsScope, storeName); bytesWrittenToDatabaseSensor = RocksDBMetrics.bytesWrittenToDatabaseSensor(streamsMetrics, metricContext); - bytesReadToDatabaseSensor = RocksDBMetrics.bytesReadFromDatabaseSensor(streamsMetrics, metricContext); + bytesReadFromDatabaseSensor = RocksDBMetrics.bytesReadFromDatabaseSensor(streamsMetrics, metricContext); memtableBytesFlushedSensor = RocksDBMetrics.memtableBytesFlushedSensor(streamsMetrics, metricContext); memtableHitRatioSensor = RocksDBMetrics.memtableHitRatioSensor(streamsMetrics, metricContext); - memtableAvgFlushTimeSensor = RocksDBMetrics.memtableAvgFlushTimeSensor(streamsMetrics, metricContext); - memtableMinFlushTimeSensor = RocksDBMetrics.memtableMinFlushTimeSensor(streamsMetrics, metricContext); - memtableMaxFlushTimeSensor = RocksDBMetrics.memtableMaxFlushTimeSensor(streamsMetrics, metricContext); writeStallDurationSensor = RocksDBMetrics.writeStallDurationSensor(streamsMetrics, metricContext); blockCacheDataHitRatioSensor = RocksDBMetrics.blockCacheDataHitRatioSensor(streamsMetrics, metricContext); blockCacheIndexHitRatioSensor = RocksDBMetrics.blockCacheIndexHitRatioSensor(streamsMetrics, metricContext); blockCacheFilterHitRatioSensor = RocksDBMetrics.blockCacheFilterHitRatioSensor(streamsMetrics, metricContext); - bytesReadDuringCompactionSensor = RocksDBMetrics.bytesReadDuringCompactionSensor(streamsMetrics, metricContext); bytesWrittenDuringCompactionSensor = RocksDBMetrics.bytesWrittenDuringCompactionSensor(streamsMetrics, metricContext); - compactionTimeAvgSensor = RocksDBMetrics.compactionTimeAvgSensor(streamsMetrics, metricContext); - compactionTimeMinSensor = RocksDBMetrics.compactionTimeMinSensor(streamsMetrics, metricContext); - compactionTimeMaxSensor = RocksDBMetrics.compactionTimeMaxSensor(streamsMetrics, metricContext); + bytesReadDuringCompactionSensor = RocksDBMetrics.bytesReadDuringCompactionSensor(streamsMetrics, metricContext); numberOfOpenFilesSensor = RocksDBMetrics.numberOfOpenFilesSensor(streamsMetrics, metricContext); numberOfFileErrorsSensor = RocksDBMetrics.numberOfFileErrorsSensor(streamsMetrics, metricContext); } - public void addStatistics(final String storeName, - final Statistics statistics, - final StreamsMetricsImpl streamsMetrics, - final TaskId taskId) { - if (statisticsToRecord.isEmpty()) { - init(streamsMetrics, taskId); - } - if (statisticsToRecord.containsKey(storeName)) { - throw new IllegalStateException("A statistics for store " + storeName + "has been already registered. " - + "This is a bug in Kafka Streams."); + public void removeStatistics(final String segmentName) { + logger.debug("Removing statistics for store {} of task {}", segmentName, taskId); + final Statistics removedStatistics = statisticsToRecord.remove(segmentName); + if (removedStatistics == null) { + throw new IllegalStateException("No statistics for store \"" + segmentName + "\" of task " + taskId + + " could be found. This is a bug in Kafka Streams."); } - statistics.setStatsLevel(StatsLevel.EXCEPT_DETAILED_TIMERS); - statisticsToRecord.put(storeName, statistics); - } - - public void removeStatistics(final String storeName) { - final Statistics removedStatistics = statisticsToRecord.remove(storeName); - if (removedStatistics != null) { - removedStatistics.close(); + removedStatistics.close(); + if (statisticsToRecord.isEmpty()) { + logger.debug( + "Removing metrics recorder for store {} of task {} from metrics recording trigger", + storeName, + taskId + ); + streamsMetrics.rocksDBMetricsRecordingTrigger().removeMetricsRecorder(this); } } public void record() { - // TODO: this block of record calls merely avoids compiler warnings. - // The actual computations of the metrics will be implemented in the next PR - bytesWrittenToDatabaseSensor.record(0); - bytesReadToDatabaseSensor.record(0); - memtableBytesFlushedSensor.record(0); - memtableHitRatioSensor.record(0); - memtableAvgFlushTimeSensor.record(0); - memtableMinFlushTimeSensor.record(0); - memtableMaxFlushTimeSensor.record(0); - writeStallDurationSensor.record(0); - blockCacheDataHitRatioSensor.record(0); - blockCacheIndexHitRatioSensor.record(0); - blockCacheFilterHitRatioSensor.record(0); - bytesReadDuringCompactionSensor.record(0); - bytesWrittenDuringCompactionSensor.record(0); - compactionTimeAvgSensor.record(0); - compactionTimeMinSensor.record(0); - compactionTimeMaxSensor.record(0); - numberOfOpenFilesSensor.record(0); - numberOfFileErrorsSensor.record(0); + logger.debug("Recording metrics for store {}", storeName); + long bytesWrittenToDatabase = 0; + long bytesReadFromDatabase = 0; + long memtableBytesFlushed = 0; + long memtableHits = 0; + long memtableMisses = 0; + long blockCacheDataHits = 0; + long blockCacheDataMisses = 0; + long blockCacheIndexHits = 0; + long blockCacheIndexMisses = 0; + long blockCacheFilterHits = 0; + long blockCacheFilterMisses = 0; + long writeStallDuration = 0; + long bytesWrittenDuringCompaction = 0; + long bytesReadDuringCompaction = 0; + long numberOfOpenFiles = 0; + long numberOfFileErrors = 0; + for (final Statistics statistics : statisticsToRecord.values()) { + bytesWrittenToDatabase += statistics.getAndResetTickerCount(TickerType.BYTES_WRITTEN); + bytesReadFromDatabase += statistics.getAndResetTickerCount(TickerType.BYTES_READ); + memtableBytesFlushed += statistics.getAndResetTickerCount(TickerType.FLUSH_WRITE_BYTES); + memtableHits += statistics.getAndResetTickerCount(TickerType.MEMTABLE_HIT); + memtableMisses += statistics.getAndResetTickerCount(TickerType.MEMTABLE_MISS); + blockCacheDataHits += statistics.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_HIT); + blockCacheDataMisses += statistics.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_MISS); + blockCacheIndexHits += statistics.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_HIT); + blockCacheIndexMisses += statistics.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_MISS); + blockCacheFilterHits += statistics.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_HIT); + blockCacheFilterMisses += statistics.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_MISS); + writeStallDuration += statistics.getAndResetTickerCount(TickerType.STALL_MICROS); + bytesWrittenDuringCompaction += statistics.getAndResetTickerCount(TickerType.COMPACT_WRITE_BYTES); + bytesReadDuringCompaction += statistics.getAndResetTickerCount(TickerType.COMPACT_READ_BYTES); + numberOfOpenFiles += statistics.getAndResetTickerCount(TickerType.NO_FILE_OPENS) + - statistics.getAndResetTickerCount(TickerType.NO_FILE_CLOSES); + numberOfFileErrors += statistics.getAndResetTickerCount(TickerType.NO_FILE_ERRORS); + } + bytesWrittenToDatabaseSensor.record(bytesWrittenToDatabase); + bytesReadFromDatabaseSensor.record(bytesReadFromDatabase); + memtableBytesFlushedSensor.record(memtableBytesFlushed); + memtableHitRatioSensor.record(computeHitRatio(memtableHits, memtableMisses)); + blockCacheDataHitRatioSensor.record(computeHitRatio(blockCacheDataHits, blockCacheDataMisses)); + blockCacheIndexHitRatioSensor.record(computeHitRatio(blockCacheIndexHits, blockCacheIndexMisses)); + blockCacheFilterHitRatioSensor.record(computeHitRatio(blockCacheFilterHits, blockCacheFilterMisses)); + writeStallDurationSensor.record(writeStallDuration); + bytesWrittenDuringCompactionSensor.record(bytesWrittenDuringCompaction); + bytesReadDuringCompactionSensor.record(bytesReadDuringCompaction); + numberOfOpenFilesSensor.record(numberOfOpenFiles); + numberOfFileErrorsSensor.record(numberOfFileErrors); } - public void close() { - for (final Statistics statistics : statisticsToRecord.values()) { - statistics.close(); + private double computeHitRatio(final long hits, final long misses) { + if (hits == 0) { + return 0; } - statisticsToRecord.clear(); + return (double) hits / (hits + misses); } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTrigger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTrigger.java new file mode 100644 index 000000000000..ce4c01af6fcb --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTrigger.java @@ -0,0 +1,56 @@ +/* + * 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.streams.state.internals.metrics; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class RocksDBMetricsRecordingTrigger implements Runnable { + + private final Map metricsRecordersToTrigger = new ConcurrentHashMap<>(); + + public void addMetricsRecorder(final RocksDBMetricsRecorder metricsRecorder) { + final String metricsRecorderName = metricsRecorderName(metricsRecorder); + if (metricsRecordersToTrigger.containsKey(metricsRecorderName)) { + throw new IllegalStateException("RocksDB metrics recorder for store \"" + metricsRecorder.storeName() + + "\" of task " + metricsRecorder.taskId().toString() + " has already been added. " + + "This is a bug in Kafka Streams."); + } + metricsRecordersToTrigger.put(metricsRecorderName, metricsRecorder); + } + + public void removeMetricsRecorder(final RocksDBMetricsRecorder metricsRecorder) { + final RocksDBMetricsRecorder removedMetricsRecorder = + metricsRecordersToTrigger.remove(metricsRecorderName(metricsRecorder)); + if (removedMetricsRecorder == null) { + throw new IllegalStateException("No RocksDB metrics recorder for store " + + "\"" + metricsRecorder.storeName() + "\" of task " + metricsRecorder.taskId() + " could be found. " + + "This is a bug in Kafka Streams."); + } + } + + private String metricsRecorderName(final RocksDBMetricsRecorder metricsRecorder) { + return metricsRecorder.taskId().toString() + "-" + metricsRecorder.storeName(); + } + + @Override + public void run() { + for (final RocksDBMetricsRecorder metricsRecorder : metricsRecordersToTrigger.values()) { + metricsRecorder.record(); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index 2cd31ba73629..5bf21ea9ce1f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -16,286 +16,438 @@ */ package org.apache.kafka.streams; -import org.apache.kafka.clients.CommonClientConfigs; -import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.network.Selectable; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; -import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; -import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.ThreadMetadata; +import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.internals.GlobalStreamThread; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.processor.internals.StateDirectory; import org.apache.kafka.streams.processor.internals.StreamThread; +import org.apache.kafka.streams.processor.internals.StreamsMetadataState; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; -import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecordingTrigger; import org.apache.kafka.test.MockClientSupplier; import org.apache.kafka.test.MockMetricsReporter; import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.test.MockStateRestoreListener; import org.apache.kafka.test.TestUtils; -import org.junit.After; +import org.easymock.Capture; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Before; -import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; +import java.net.InetSocketAddress; import java.time.Duration; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.Set; -import java.util.concurrent.CountDownLatch; +import java.util.UUID; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; +import java.util.concurrent.atomic.AtomicReference; -import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.easymock.EasyMock.anyInt; +import static org.easymock.EasyMock.anyLong; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -@Category({IntegrationTest.class}) +@RunWith(PowerMockRunner.class) +@PrepareForTest({KafkaStreams.class, StreamThread.class}) public class KafkaStreamsTest { - private static final int NUM_BROKERS = 1; private static final int NUM_THREADS = 2; - // We need this to avoid the KafkaConsumer hanging on poll - // (this may occur if the test doesn't complete quickly enough) - @ClassRule - public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); - private final StreamsBuilder builder = new StreamsBuilder(); - private KafkaStreams globalStreams; - private Properties props; @Rule public TestName testName = new TestName(); + private MockClientSupplier supplier; + private MockTime time; + + private Properties props; + + @Mock + private StateDirectory stateDirectory; + @Mock + private StreamThread streamThreadOne; + @Mock + private StreamThread streamThreadTwo; + @Mock + private GlobalStreamThread globalStreamThread; + @Mock + private ScheduledExecutorService cleanupSchedule; + @Mock + private Metrics metrics; + + private StateListenerStub streamsStateListener; + private Capture> metricsReportersCapture; + private Capture threadStatelistenerCapture; + + public static class StateListenerStub implements KafkaStreams.StateListener { + int numChanges = 0; + KafkaStreams.State oldState; + KafkaStreams.State newState; + public Map mapStates = new HashMap<>(); + + @Override + public void onChange(final KafkaStreams.State newState, + final KafkaStreams.State oldState) { + final long prevCount = mapStates.containsKey(newState) ? mapStates.get(newState) : 0; + numChanges++; + this.oldState = oldState; + this.newState = newState; + mapStates.put(newState, prevCount + 1); + } + } + @Before - public void before() { + public void before() throws Exception { + time = new MockTime(); + supplier = new MockClientSupplier(); + supplier.setClusterForAdminClient(Cluster.bootstrap(singletonList(new InetSocketAddress("localhost", 9999)))); + streamsStateListener = new StateListenerStub(); + threadStatelistenerCapture = EasyMock.newCapture(); + metricsReportersCapture = EasyMock.newCapture(); + props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "appId"); props.put(StreamsConfig.CLIENT_ID_CONFIG, "clientId"); - props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2018"); props.put(StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); props.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS); - globalStreams = new KafkaStreams(builder.build(), props); - } - @After - public void cleanup() { - if (globalStreams != null) { - globalStreams.close(); - } + prepareStreams(); } - @Test - public void testOsDefaultSocketBufferSizes() { - props.put(CommonClientConfigs.SEND_BUFFER_CONFIG, Selectable.USE_DEFAULT_BUFFER_SIZE); - props.put(CommonClientConfigs.RECEIVE_BUFFER_CONFIG, Selectable.USE_DEFAULT_BUFFER_SIZE); - final KafkaStreams streams = new KafkaStreams(builder.build(), props); - streams.close(); - } - - @Test(expected = KafkaException.class) - public void testInvalidSocketSendBufferSize() { - props.put(CommonClientConfigs.SEND_BUFFER_CONFIG, -2); - final KafkaStreams streams = new KafkaStreams(builder.build(), props); - streams.close(); - } - - @Test(expected = KafkaException.class) - public void testInvalidSocketReceiveBufferSize() { - props.put(CommonClientConfigs.RECEIVE_BUFFER_CONFIG, -2); - final KafkaStreams streams = new KafkaStreams(builder.build(), props); - streams.close(); + private void prepareStreams() throws Exception { + // setup metrics + PowerMock.expectNew(Metrics.class, + anyObject(MetricConfig.class), + EasyMock.capture(metricsReportersCapture), + EasyMock.anyObject(Time.class) + ).andAnswer(() -> { + for (final MetricsReporter reporter : metricsReportersCapture.getValue()) { + reporter.init(Collections.emptyList()); + } + return metrics; + }).anyTimes(); + metrics.close(); + EasyMock.expectLastCall().andAnswer(() -> { + for (final MetricsReporter reporter : metricsReportersCapture.getValue()) { + reporter.close(); + } + return null; + }).anyTimes(); + + // setup stream threads + PowerMock.mockStatic(StreamThread.class); + EasyMock.expect(StreamThread.create( + anyObject(InternalTopologyBuilder.class), + anyObject(StreamsConfig.class), + anyObject(KafkaClientSupplier.class), + anyObject(Admin.class), + anyObject(UUID.class), + anyObject(String.class), + anyObject(Metrics.class), + anyObject(Time.class), + anyObject(StreamsMetadataState.class), + anyLong(), + anyObject(StateDirectory.class), + anyObject(StateRestoreListener.class), + anyInt() + )).andReturn(streamThreadOne).andReturn(streamThreadTwo); + EasyMock.expect(StreamThread.getSharedAdminClientId( + anyString() + )).andReturn("admin").anyTimes(); + + EasyMock.expect(streamThreadOne.getId()).andReturn(0L).anyTimes(); + EasyMock.expect(streamThreadTwo.getId()).andReturn(1L).anyTimes(); + prepareStreamThread(streamThreadOne, true); + prepareStreamThread(streamThreadTwo, false); + + // setup global threads + final AtomicReference globalThreadState = new AtomicReference<>(GlobalStreamThread.State.CREATED); + PowerMock.expectNew(GlobalStreamThread.class, + anyObject(ProcessorTopology.class), + anyObject(StreamsConfig.class), + anyObject(Consumer.class), + anyObject(StateDirectory.class), + anyLong(), + anyObject(Metrics.class), + anyObject(Time.class), + anyString(), + anyObject(StateRestoreListener.class), + anyObject(RocksDBMetricsRecordingTrigger.class) + ).andReturn(globalStreamThread).anyTimes(); + EasyMock.expect(globalStreamThread.state()).andAnswer(globalThreadState::get).anyTimes(); + globalStreamThread.setStateListener(EasyMock.capture(threadStatelistenerCapture)); + EasyMock.expectLastCall().anyTimes(); + + globalStreamThread.start(); + EasyMock.expectLastCall().andAnswer(() -> { + globalThreadState.set(GlobalStreamThread.State.RUNNING); + threadStatelistenerCapture.getValue().onChange(globalStreamThread, + GlobalStreamThread.State.RUNNING, + GlobalStreamThread.State.CREATED); + return null; + }).anyTimes(); + globalStreamThread.shutdown(); + EasyMock.expectLastCall().andAnswer(() -> { + supplier.restoreConsumer.close(); + for (final MockProducer producer : supplier.producers) { + producer.close(); + } + globalThreadState.set(GlobalStreamThread.State.DEAD); + threadStatelistenerCapture.getValue().onChange(globalStreamThread, + GlobalStreamThread.State.PENDING_SHUTDOWN, + GlobalStreamThread.State.RUNNING); + threadStatelistenerCapture.getValue().onChange(globalStreamThread, + GlobalStreamThread.State.DEAD, + GlobalStreamThread.State.PENDING_SHUTDOWN); + return null; + }).anyTimes(); + EasyMock.expect(globalStreamThread.stillRunning()).andReturn(globalThreadState.get() == GlobalStreamThread.State.RUNNING).anyTimes(); + globalStreamThread.join(); + EasyMock.expectLastCall().anyTimes(); + + PowerMock.replay(StreamThread.class, Metrics.class, metrics, streamThreadOne, streamThreadTwo, GlobalStreamThread.class, globalStreamThread); + } + + private void prepareStreamThread(final StreamThread thread, final boolean terminable) throws Exception { + final AtomicReference state = new AtomicReference<>(StreamThread.State.CREATED); + EasyMock.expect(thread.state()).andAnswer(state::get).anyTimes(); + + thread.setStateListener(EasyMock.capture(threadStatelistenerCapture)); + EasyMock.expectLastCall().anyTimes(); + thread.setRocksDBMetricsRecordingTrigger(EasyMock.anyObject(RocksDBMetricsRecordingTrigger.class)); + EasyMock.expectLastCall().anyTimes(); + + thread.start(); + EasyMock.expectLastCall().andAnswer(() -> { + state.set(StreamThread.State.STARTING); + threadStatelistenerCapture.getValue().onChange(thread, + StreamThread.State.STARTING, + StreamThread.State.CREATED); + threadStatelistenerCapture.getValue().onChange(thread, + StreamThread.State.PARTITIONS_REVOKED, + StreamThread.State.STARTING); + threadStatelistenerCapture.getValue().onChange(thread, + StreamThread.State.PARTITIONS_ASSIGNED, + StreamThread.State.PARTITIONS_REVOKED); + threadStatelistenerCapture.getValue().onChange(thread, + StreamThread.State.RUNNING, + StreamThread.State.PARTITIONS_ASSIGNED); + return null; + }).anyTimes(); + thread.shutdown(); + EasyMock.expectLastCall().andAnswer(() -> { + supplier.consumer.close(); + supplier.restoreConsumer.close(); + for (final MockProducer producer : supplier.producers) { + producer.close(); + } + state.set(StreamThread.State.DEAD); + threadStatelistenerCapture.getValue().onChange(thread, StreamThread.State.PENDING_SHUTDOWN, StreamThread.State.RUNNING); + threadStatelistenerCapture.getValue().onChange(thread, StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN); + return null; + }).anyTimes(); + EasyMock.expect(thread.isRunning()).andReturn(state.get() == StreamThread.State.RUNNING).anyTimes(); + thread.join(); + if (terminable) + EasyMock.expectLastCall().anyTimes(); + else + EasyMock.expectLastCall().andAnswer(() -> { + Thread.sleep(50L); + return null; + }).anyTimes(); } @Test - public void stateShouldTransitToNotRunningIfCloseRightAfterCreated() { - globalStreams.close(); + public void testShouldTransitToNotRunningIfCloseRightAfterCreated() { + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.close(); - Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, globalStreams.state()); + Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, streams.state()); } @Test public void stateShouldTransitToRunningIfNonDeadThreadsBackToRunning() throws InterruptedException { - final StateListenerStub stateListener = new StateListenerStub(); - globalStreams.setStateListener(stateListener); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.setStateListener(streamsStateListener); - Assert.assertEquals(0, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.CREATED, globalStreams.state()); + Assert.assertEquals(0, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.CREATED, streams.state()); - globalStreams.start(); + streams.start(); TestUtils.waitForCondition( - () -> stateListener.numChanges == 2, + () -> streamsStateListener.numChanges == 2, "Streams never started."); - Assert.assertEquals(KafkaStreams.State.RUNNING, globalStreams.state()); + Assert.assertEquals(KafkaStreams.State.RUNNING, streams.state()); - for (final StreamThread thread: globalStreams.threads) { - thread.stateListener().onChange( + for (final StreamThread thread: streams.threads) { + threadStatelistenerCapture.getValue().onChange( thread, StreamThread.State.PARTITIONS_REVOKED, StreamThread.State.RUNNING); } - Assert.assertEquals(3, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.REBALANCING, globalStreams.state()); + Assert.assertEquals(3, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.REBALANCING, streams.state()); - for (final StreamThread thread : globalStreams.threads) { - thread.stateListener().onChange( + for (final StreamThread thread : streams.threads) { + threadStatelistenerCapture.getValue().onChange( thread, StreamThread.State.PARTITIONS_ASSIGNED, StreamThread.State.PARTITIONS_REVOKED); } - Assert.assertEquals(3, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.REBALANCING, globalStreams.state()); + Assert.assertEquals(3, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.REBALANCING, streams.state()); - globalStreams.threads[NUM_THREADS - 1].stateListener().onChange( - globalStreams.threads[NUM_THREADS - 1], + threadStatelistenerCapture.getValue().onChange( + streams.threads[NUM_THREADS - 1], StreamThread.State.PENDING_SHUTDOWN, StreamThread.State.PARTITIONS_ASSIGNED); - globalStreams.threads[NUM_THREADS - 1].stateListener().onChange( - globalStreams.threads[NUM_THREADS - 1], + threadStatelistenerCapture.getValue().onChange( + streams.threads[NUM_THREADS - 1], StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN); - Assert.assertEquals(3, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.REBALANCING, globalStreams.state()); + Assert.assertEquals(3, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.REBALANCING, streams.state()); - for (final StreamThread thread : globalStreams.threads) { - if (thread != globalStreams.threads[NUM_THREADS - 1]) { - thread.stateListener().onChange( + for (final StreamThread thread : streams.threads) { + if (thread != streams.threads[NUM_THREADS - 1]) { + threadStatelistenerCapture.getValue().onChange( thread, StreamThread.State.RUNNING, StreamThread.State.PARTITIONS_ASSIGNED); } } - Assert.assertEquals(4, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.RUNNING, globalStreams.state()); + Assert.assertEquals(4, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.RUNNING, streams.state()); - globalStreams.close(); + streams.close(); TestUtils.waitForCondition( - () -> stateListener.numChanges == 6, + () -> streamsStateListener.numChanges == 6, "Streams never closed."); - Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, globalStreams.state()); + Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, streams.state()); } @Test public void stateShouldTransitToErrorIfAllThreadsDead() throws InterruptedException { - final StateListenerStub stateListener = new StateListenerStub(); - globalStreams.setStateListener(stateListener); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.setStateListener(streamsStateListener); - Assert.assertEquals(0, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.CREATED, globalStreams.state()); + Assert.assertEquals(0, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.CREATED, streams.state()); - globalStreams.start(); + streams.start(); TestUtils.waitForCondition( - () -> stateListener.numChanges == 2, + () -> streamsStateListener.numChanges == 2, "Streams never started."); - Assert.assertEquals(KafkaStreams.State.RUNNING, globalStreams.state()); + Assert.assertEquals(KafkaStreams.State.RUNNING, streams.state()); - for (final StreamThread thread : globalStreams.threads) { - thread.stateListener().onChange( + for (final StreamThread thread : streams.threads) { + threadStatelistenerCapture.getValue().onChange( thread, StreamThread.State.PARTITIONS_REVOKED, StreamThread.State.RUNNING); } - Assert.assertEquals(3, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.REBALANCING, globalStreams.state()); + Assert.assertEquals(3, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.REBALANCING, streams.state()); - globalStreams.threads[NUM_THREADS - 1].stateListener().onChange( - globalStreams.threads[NUM_THREADS - 1], + threadStatelistenerCapture.getValue().onChange( + streams.threads[NUM_THREADS - 1], StreamThread.State.PENDING_SHUTDOWN, StreamThread.State.PARTITIONS_REVOKED); - globalStreams.threads[NUM_THREADS - 1].stateListener().onChange( - globalStreams.threads[NUM_THREADS - 1], + threadStatelistenerCapture.getValue().onChange( + streams.threads[NUM_THREADS - 1], StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN); - Assert.assertEquals(3, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.REBALANCING, globalStreams.state()); + Assert.assertEquals(3, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.REBALANCING, streams.state()); - for (final StreamThread thread : globalStreams.threads) { - if (thread != globalStreams.threads[NUM_THREADS - 1]) { - thread.stateListener().onChange( + for (final StreamThread thread : streams.threads) { + if (thread != streams.threads[NUM_THREADS - 1]) { + threadStatelistenerCapture.getValue().onChange( thread, StreamThread.State.PENDING_SHUTDOWN, StreamThread.State.PARTITIONS_REVOKED); - thread.stateListener().onChange( + threadStatelistenerCapture.getValue().onChange( thread, StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN); } } - Assert.assertEquals(4, stateListener.numChanges); - Assert.assertEquals(KafkaStreams.State.ERROR, globalStreams.state()); + Assert.assertEquals(4, streamsStateListener.numChanges); + Assert.assertEquals(KafkaStreams.State.ERROR, streams.state()); - globalStreams.close(); + streams.close(); // the state should not stuck with ERROR, but transit to NOT_RUNNING in the end TestUtils.waitForCondition( - () -> stateListener.numChanges == 6, + () -> streamsStateListener.numChanges == 6, "Streams never closed."); - Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, globalStreams.state()); + Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, streams.state()); } @Test public void shouldCleanupResourcesOnCloseWithoutPreviousStart() throws Exception { + final StreamsBuilder builder = new StreamsBuilder(); builder.globalTable("anyTopic"); - final List nodes = Collections.singletonList(new Node(0, "localhost", 8121)); - final Cluster cluster = new Cluster("mockClusterId", nodes, - Collections.emptySet(), Collections.emptySet(), - Collections.emptySet(), nodes.get(0)); - final MockClientSupplier clientSupplier = new MockClientSupplier(); - clientSupplier.setClusterForAdminClient(cluster); - final KafkaStreams streams = new KafkaStreams(builder.build(), props, clientSupplier); + + final KafkaStreams streams = new KafkaStreams(builder.build(), props, supplier, time); streams.close(); + TestUtils.waitForCondition( () -> streams.state() == KafkaStreams.State.NOT_RUNNING, "Streams never stopped."); - // Ensure that any created clients are closed - assertTrue(clientSupplier.consumer.closed()); - assertTrue(clientSupplier.restoreConsumer.closed()); - for (final MockProducer p : clientSupplier.producers) { + assertTrue(supplier.consumer.closed()); + assertTrue(supplier.restoreConsumer.closed()); + for (final MockProducer p : supplier.producers) { assertTrue(p.closed()); } } @@ -303,15 +455,12 @@ public void shouldCleanupResourcesOnCloseWithoutPreviousStart() throws Exception @Test public void testStateThreadClose() throws Exception { // make sure we have the global state thread running too + final StreamsBuilder builder = new StreamsBuilder(); builder.globalTable("anyTopic"); - final KafkaStreams streams = new KafkaStreams(builder.build(), props); + final KafkaStreams streams = new KafkaStreams(builder.build(), props, supplier, time); try { - final java.lang.reflect.Field threadsField = streams.getClass().getDeclaredField("threads"); - threadsField.setAccessible(true); - final StreamThread[] threads = (StreamThread[]) threadsField.get(streams); - - assertEquals(NUM_THREADS, threads.length); + assertEquals(NUM_THREADS, streams.threads.length); assertEquals(streams.state(), KafkaStreams.State.CREATED); streams.start(); @@ -320,12 +469,11 @@ public void testStateThreadClose() throws Exception { "Streams never started."); for (int i = 0; i < NUM_THREADS; i++) { - final StreamThread tmpThread = threads[i]; + final StreamThread tmpThread = streams.threads[i]; tmpThread.shutdown(); - TestUtils.waitForCondition( - () -> tmpThread.state() == StreamThread.State.DEAD, + TestUtils.waitForCondition(() -> tmpThread.state() == StreamThread.State.DEAD, "Thread never stopped."); - threads[i].join(); + streams.threads[i].join(); } TestUtils.waitForCondition( () -> streams.state() == KafkaStreams.State.ERROR, @@ -338,26 +486,23 @@ public void testStateThreadClose() throws Exception { () -> streams.state() == KafkaStreams.State.NOT_RUNNING, "Streams never stopped."); - final java.lang.reflect.Field globalThreadField = streams.getClass().getDeclaredField("globalStreamThread"); - globalThreadField.setAccessible(true); - final GlobalStreamThread globalStreamThread = (GlobalStreamThread) globalThreadField.get(streams); - assertNull(globalStreamThread); + assertNull(streams.globalStreamThread); } @Test public void testStateGlobalThreadClose() throws Exception { // make sure we have the global state thread running too + final StreamsBuilder builder = new StreamsBuilder(); builder.globalTable("anyTopic"); - final KafkaStreams streams = new KafkaStreams(builder.build(), props); + final KafkaStreams streams = new KafkaStreams(builder.build(), props, supplier, time); try { streams.start(); TestUtils.waitForCondition( () -> streams.state() == KafkaStreams.State.RUNNING, "Streams never started."); - final java.lang.reflect.Field globalThreadField = streams.getClass().getDeclaredField("globalStreamThread"); - globalThreadField.setAccessible(true); - final GlobalStreamThread globalStreamThread = (GlobalStreamThread) globalThreadField.get(streams); + + final GlobalStreamThread globalStreamThread = streams.globalStreamThread; globalStreamThread.shutdown(); TestUtils.waitForCondition( () -> globalStreamThread.state() == GlobalStreamThread.State.DEAD, @@ -371,51 +516,11 @@ public void testStateGlobalThreadClose() throws Exception { assertEquals(streams.state(), KafkaStreams.State.NOT_RUNNING); } - @Test - public void globalThreadShouldTimeoutWhenBrokerConnectionCannotBeEstablished() { - final Properties props = new Properties(); - props.put(StreamsConfig.APPLICATION_ID_CONFIG, "appId"); - props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:1"); - props.put(StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); - props.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS); - - props.put(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, 200); - - // make sure we have the global state thread running too - builder.globalTable("anyTopic"); - try (final KafkaStreams streams = new KafkaStreams(builder.build(), props)) { - streams.start(); - fail("expected start() to time out and throw an exception."); - } catch (final StreamsException expected) { - // This is a result of not being able to connect to the broker. - } - // There's nothing to assert... We're testing that this operation actually completes. - } - - @Test - public void testLocalThreadCloseWithoutConnectingToBroker() { - final Properties props = new Properties(); - props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "appId"); - props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:1"); - props.setProperty(StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); - props.setProperty(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS); - - // make sure we have the global state thread running too - builder.table("anyTopic"); - try (final KafkaStreams streams = new KafkaStreams(builder.build(), props)) { - streams.start(); - } - // There's nothing to assert... We're testing that this operation actually completes. - } - - @Test public void testInitializesAndDestroysMetricsReporters() { final int oldInitCount = MockMetricsReporter.INIT_COUNT.get(); - try (final KafkaStreams streams = new KafkaStreams(builder.build(), props)) { + try (final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time)) { final int newInitCount = MockMetricsReporter.INIT_COUNT.get(); final int initDiff = newInitCount - oldInitCount; assertTrue("some reporters should be initialized by calling on construction", initDiff > 0); @@ -429,60 +534,50 @@ public void testInitializesAndDestroysMetricsReporters() { @Test public void testCloseIsIdempotent() { - globalStreams.close(); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.close(); final int closeCount = MockMetricsReporter.CLOSE_COUNT.get(); - globalStreams.close(); + streams.close(); Assert.assertEquals("subsequent close() calls should do nothing", closeCount, MockMetricsReporter.CLOSE_COUNT.get()); } @Test public void testCannotStartOnceClosed() { - globalStreams.start(); - globalStreams.close(); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.start(); + streams.close(); try { - globalStreams.start(); + streams.start(); fail("Should have throw IllegalStateException"); } catch (final IllegalStateException expected) { // this is ok } finally { - globalStreams.close(); - } - } - - @Test - public void testCannotStartTwice() { - globalStreams.start(); - - try { - globalStreams.start(); - fail("Should throw an IllegalStateException"); - } catch (final IllegalStateException e) { - // this is ok - } finally { - globalStreams.close(); + streams.close(); } } @Test public void shouldNotSetGlobalRestoreListenerAfterStarting() { - globalStreams.start(); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.start(); try { - globalStreams.setGlobalStateRestoreListener(new MockStateRestoreListener()); + streams.setGlobalStateRestoreListener(null); fail("Should throw an IllegalStateException"); } catch (final IllegalStateException e) { // expected } finally { - globalStreams.close(); + streams.close(); } } @Test public void shouldThrowExceptionSettingUncaughtExceptionHandlerNotInCreateState() { - globalStreams.start(); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.start(); try { - globalStreams.setUncaughtExceptionHandler(null); + streams.setUncaughtExceptionHandler(null); fail("Should throw IllegalStateException"); } catch (final IllegalStateException e) { // expected @@ -491,9 +586,10 @@ public void shouldThrowExceptionSettingUncaughtExceptionHandlerNotInCreateState( @Test public void shouldThrowExceptionSettingStateListenerNotInCreateState() { - globalStreams.start(); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.start(); try { - globalStreams.setStateListener(null); + streams.setStateListener(null); fail("Should throw IllegalStateException"); } catch (final IllegalStateException e) { // expected @@ -501,193 +597,121 @@ public void shouldThrowExceptionSettingStateListenerNotInCreateState() { } @Test - public void testIllegalMetricsConfig() { - props.setProperty(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "illegalConfig"); - + public void shouldAllowCleanupBeforeStartAndAfterClose() { + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); try { - new KafkaStreams(builder.build(), props); - fail("Should have throw ConfigException"); - } catch (final ConfigException expected) { /* expected */ } + streams.cleanUp(); + streams.start(); + } finally { + streams.close(); + streams.cleanUp(); + } } @Test - public void testLegalMetricsConfig() { - props.setProperty(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.INFO.toString()); - new KafkaStreams(builder.build(), props).close(); + public void shouldThrowOnCleanupWhileRunning() throws InterruptedException { + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.start(); + TestUtils.waitForCondition( + () -> streams.state() == KafkaStreams.State.RUNNING, + "Streams never started."); - props.setProperty(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.DEBUG.toString()); - new KafkaStreams(builder.build(), props).close(); + try { + streams.cleanUp(); + fail("Should have thrown IllegalStateException"); + } catch (final IllegalStateException expected) { + assertEquals("Cannot clean up while running.", expected.getMessage()); + } } @Test(expected = IllegalStateException.class) public void shouldNotGetAllTasksWhenNotRunning() { - globalStreams.allMetadata(); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.allMetadata(); } @Test(expected = IllegalStateException.class) public void shouldNotGetAllTasksWithStoreWhenNotRunning() { - globalStreams.allMetadataForStore("store"); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.allMetadataForStore("store"); } @Test(expected = IllegalStateException.class) public void shouldNotGetTaskWithKeyAndSerializerWhenNotRunning() { - globalStreams.metadataForKey("store", "key", Serdes.String().serializer()); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.metadataForKey("store", "key", Serdes.String().serializer()); } @Test(expected = IllegalStateException.class) public void shouldNotGetTaskWithKeyAndPartitionerWhenNotRunning() { - globalStreams.metadataForKey("store", "key", (topic, key, value, numPartitions) -> 0); + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.metadataForKey("store", "key", (topic, key, value, numPartitions) -> 0); } @Test - public void shouldReturnFalseOnCloseWhenThreadsHaventTerminated() throws Exception { - final AtomicBoolean keepRunning = new AtomicBoolean(true); - KafkaStreams streams = null; - try { - final StreamsBuilder builder = new StreamsBuilder(); - final CountDownLatch latch = new CountDownLatch(1); - final String topic = "input"; - CLUSTER.createTopics(topic); - - builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String())) - .foreach((key, value) -> { - try { - latch.countDown(); - while (keepRunning.get()) { - Thread.sleep(10); - } - } catch (final InterruptedException e) { - // no-op - } - }); - streams = new KafkaStreams(builder.build(), props); - streams.start(); - IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(topic, - Collections.singletonList(new KeyValue<>("A", "A")), - TestUtils.producerConfig( - CLUSTER.bootstrapServers(), - StringSerializer.class, - StringSerializer.class, - new Properties()), - System.currentTimeMillis()); - - assertTrue("Timed out waiting to receive single message", latch.await(30, TimeUnit.SECONDS)); - assertFalse(streams.close(Duration.ofMillis(10))); - } finally { - // stop the thread so we don't interfere with other tests etc - keepRunning.set(false); - if (streams != null) { - streams.close(); - } + public void shouldReturnFalseOnCloseWhenThreadsHaventTerminated() { + // do not use mock time so that it can really elapse + try (final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier)) { + assertFalse(streams.close(Duration.ofMillis(10L))); } } - @Test - public void shouldReturnThreadMetadata() { - globalStreams.start(); - final Set threadMetadata = globalStreams.localThreadsMetadata(); - assertNotNull(threadMetadata); - assertEquals(2, threadMetadata.size()); - for (final ThreadMetadata metadata : threadMetadata) { - assertTrue("#threadState() was: " + metadata.threadState() + "; expected either RUNNING, STARTING, PARTITIONS_REVOKED, PARTITIONS_ASSIGNED, or CREATED", - asList("RUNNING", "STARTING", "PARTITIONS_REVOKED", "PARTITIONS_ASSIGNED", "CREATED").contains(metadata.threadState())); - assertEquals(0, metadata.standbyTasks().size()); - assertEquals(0, metadata.activeTasks().size()); - final String threadName = metadata.threadName(); - assertTrue(threadName.startsWith("clientId-StreamThread-")); - assertEquals(threadName + "-consumer", metadata.consumerClientId()); - assertEquals(threadName + "-restore-consumer", metadata.restoreConsumerClientId()); - assertEquals(Collections.singleton(threadName + "-producer"), metadata.producerClientIds()); - assertEquals("clientId-admin", metadata.adminClientId()); + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnNegativeTimeoutForClose() { + try (final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time)) { + streams.close(Duration.ofMillis(-1L)); } } @Test - public void shouldAllowCleanupBeforeStartAndAfterClose() { - try { - globalStreams.cleanUp(); - globalStreams.start(); - } finally { - globalStreams.close(); + public void shouldNotBlockInCloseForZeroDuration() { + try (final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time)) { + // with mock time that does not elapse, close would not return if it ever waits on the state transition + assertFalse(streams.close(Duration.ZERO)); } - globalStreams.cleanUp(); } @Test - public void shouldThrowOnCleanupWhileRunning() throws InterruptedException { - globalStreams.start(); - TestUtils.waitForCondition( - () -> globalStreams.state() == KafkaStreams.State.RUNNING, - "Streams never started."); - - try { - globalStreams.cleanUp(); - fail("Should have thrown IllegalStateException"); - } catch (final IllegalStateException expected) { - assertEquals("Cannot clean up while running.", expected.getMessage()); - } - } + public void shouldCleanupOldStateDirs() throws Exception { + PowerMock.mockStatic(Executors.class); + EasyMock.expect(Executors.newSingleThreadScheduledExecutor( + anyObject(ThreadFactory.class) + )).andReturn(cleanupSchedule).anyTimes(); + + cleanupSchedule.scheduleAtFixedRate( + EasyMock.anyObject(Runnable.class), + EasyMock.eq(1L), + EasyMock.eq(1L), + EasyMock.eq(TimeUnit.MILLISECONDS) + ); + EasyMock.expectLastCall().andReturn(null); + cleanupSchedule.shutdownNow(); + EasyMock.expectLastCall().andReturn(null); + + PowerMock.expectNew(StateDirectory.class, + anyObject(StreamsConfig.class), + anyObject(Time.class), + EasyMock.eq(true) + ).andReturn(stateDirectory); + + PowerMock.replayAll(Executors.class, cleanupSchedule, stateDirectory); - @Test - public void shouldCleanupOldStateDirs() throws InterruptedException { props.setProperty(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG, "1"); - final String topic = "topic"; - CLUSTER.createTopic(topic); final StreamsBuilder builder = new StreamsBuilder(); + builder.table("topic", Materialized.as("store")); - builder.table(topic, Materialized.as("store")); - - try (final KafkaStreams streams = new KafkaStreams(builder.build(), props)) { - final CountDownLatch latch = new CountDownLatch(1); - streams.setStateListener((newState, oldState) -> { - if (newState == KafkaStreams.State.RUNNING && oldState == KafkaStreams.State.REBALANCING) { - latch.countDown(); - } - }); - final String appDir = props.getProperty(StreamsConfig.STATE_DIR_CONFIG) + File.separator + props.getProperty(StreamsConfig.APPLICATION_ID_CONFIG); - final File oldTaskDir = new File(appDir, "10_1"); - assertTrue(oldTaskDir.mkdirs()); - + try (final KafkaStreams streams = new KafkaStreams(builder.build(), props, supplier, time)) { streams.start(); - latch.await(30, TimeUnit.SECONDS); - verifyCleanupStateDir(appDir, oldTaskDir); - assertTrue(oldTaskDir.mkdirs()); - verifyCleanupStateDir(appDir, oldTaskDir); - } - } - - @Test - public void shouldThrowOnNegativeTimeoutForClose() { - try (final KafkaStreams streams = new KafkaStreams(builder.build(), props)) { - streams.close(Duration.ofMillis(-1L)); - fail("should not accept negative close parameter"); - } catch (final IllegalArgumentException e) { - // expected } - } - @Test - public void shouldNotBlockInCloseForZeroDuration() throws InterruptedException { - final KafkaStreams streams = new KafkaStreams(builder.build(), props); - final Thread th = new Thread(() -> streams.close(Duration.ofMillis(0L))); - - th.start(); - - try { - th.join(30_000L); - assertFalse(th.isAlive()); - } finally { - streams.close(); - } + PowerMock.verifyAll(); } @Test public void statelessTopologyShouldNotCreateStateDirectory() throws Exception { final String inputTopic = testName.getMethodName() + "-input"; final String outputTopic = testName.getMethodName() + "-output"; - CLUSTER.createTopics(inputTopic, outputTopic); - final Topology topology = new Topology(); topology.addSource("source", Serdes.String().deserializer(), Serdes.String().deserializer(), inputTopic) .addProcessor("process", () -> new AbstractProcessor() { @@ -699,7 +723,7 @@ public void process(final String key, final String value) { } }, "source") .addSink("sink", outputTopic, new StringSerializer(), new StringSerializer(), "process"); - startStreamsAndCheckDirExists(topology, Collections.singleton(inputTopic), outputTopic, false); + startStreamsAndCheckDirExists(topology, false); } @Test @@ -710,7 +734,7 @@ public void inMemoryStatefulTopologyShouldNotCreateStateDirectory() throws Excep final String storeName = testName.getMethodName() + "-counts"; final String globalStoreName = testName.getMethodName() + "-globalStore"; final Topology topology = getStatefulTopology(inputTopic, outputTopic, globalTopicName, storeName, globalStoreName, false); - startStreamsAndCheckDirExists(topology, asList(inputTopic, globalTopicName), outputTopic, false); + startStreamsAndCheckDirExists(topology, false); } @Test @@ -721,7 +745,7 @@ public void statefulTopologyShouldCreateStateDirectory() throws Exception { final String storeName = testName.getMethodName() + "-counts"; final String globalStoreName = testName.getMethodName() + "-globalStore"; final Topology topology = getStatefulTopology(inputTopic, outputTopic, globalTopicName, storeName, globalStoreName, true); - startStreamsAndCheckDirExists(topology, asList(inputTopic, globalTopicName), outputTopic, true); + startStreamsAndCheckDirExists(topology, true); } @SuppressWarnings("unchecked") @@ -730,8 +754,7 @@ private Topology getStatefulTopology(final String inputTopic, final String globalTopicName, final String storeName, final String globalStoreName, - final boolean isPersistentStore) throws Exception { - CLUSTER.createTopics(inputTopic, outputTopic, globalTopicName); + final boolean isPersistentStore) { final StoreBuilder> storeBuilder = Stores.keyValueStoreBuilder( isPersistentStore ? Stores.persistentKeyValueStore(storeName) @@ -740,110 +763,45 @@ private Topology getStatefulTopology(final String inputTopic, Serdes.Long()); final Topology topology = new Topology(); topology.addSource("source", Serdes.String().deserializer(), Serdes.String().deserializer(), inputTopic) - .addProcessor("process", () -> new AbstractProcessor() { - @Override - public void process(final String key, final String value) { - final KeyValueStore kvStore = - (KeyValueStore) context().getStateStore(storeName); - kvStore.put(key, 5L); - - context().forward(key, "5"); - context().commit(); - } - }, "source") - .addStateStore(storeBuilder, "process") - .addSink("sink", outputTopic, new StringSerializer(), new StringSerializer(), "process"); + .addProcessor("process", () -> new AbstractProcessor() { + @Override + public void process(final String key, final String value) { + final KeyValueStore kvStore = + (KeyValueStore) context().getStateStore(storeName); + kvStore.put(key, 5L); + + context().forward(key, "5"); + context().commit(); + } + }, "source") + .addStateStore(storeBuilder, "process") + .addSink("sink", outputTopic, new StringSerializer(), new StringSerializer(), "process"); final StoreBuilder> globalStoreBuilder = Stores.keyValueStoreBuilder( - isPersistentStore ? Stores.persistentKeyValueStore(globalStoreName) : Stores.inMemoryKeyValueStore(globalStoreName), - Serdes.String(), Serdes.String()).withLoggingDisabled(); + isPersistentStore ? Stores.persistentKeyValueStore(globalStoreName) : Stores.inMemoryKeyValueStore(globalStoreName), + Serdes.String(), Serdes.String()).withLoggingDisabled(); topology.addGlobalStore(globalStoreBuilder, - "global", - Serdes.String().deserializer(), - Serdes.String().deserializer(), - globalTopicName, - globalTopicName + "-processor", - new MockProcessorSupplier()); + "global", + Serdes.String().deserializer(), + Serdes.String().deserializer(), + globalTopicName, + globalTopicName + "-processor", + new MockProcessorSupplier()); return topology; } private void startStreamsAndCheckDirExists(final Topology topology, - final Collection inputTopics, - final String outputTopic, final boolean shouldFilesExist) throws Exception { - final File baseDir = new File(TestUtils.IO_TMP_DIR + File.separator + "kafka-" + TestUtils.randomString(5)); - final Path basePath = baseDir.toPath(); - if (!baseDir.exists()) { - Files.createDirectory(basePath); - } - // changing the path of state directory to make sure that it should not clash with other test cases. - final Properties localProps = new Properties(); - localProps.putAll(props); - localProps.put(StreamsConfig.STATE_DIR_CONFIG, baseDir.getAbsolutePath()); - - final KafkaStreams streams = new KafkaStreams(topology, localProps); - streams.start(); - - for (final String topic : inputTopics) { - IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(topic, - Collections.singletonList(new KeyValue<>("A", "A")), - TestUtils.producerConfig( - CLUSTER.bootstrapServers(), - StringSerializer.class, - StringSerializer.class, - new Properties()), - System.currentTimeMillis()); - } - - IntegrationTestUtils.readKeyValues(outputTopic, - TestUtils.consumerConfig( - CLUSTER.bootstrapServers(), - outputTopic + "-group", - StringDeserializer.class, - StringDeserializer.class), - 5000, 1); - - try { - final List files = Files.find(basePath, 999, (p, bfa) -> !p.equals(basePath)).collect(Collectors.toList()); - if (shouldFilesExist && files.isEmpty()) { - Assert.fail("Files should have existed, but it didn't: " + files); - } - if (!shouldFilesExist && !files.isEmpty()) { - Assert.fail("Files should not have existed, but it did: " + files); - } - } catch (final IOException e) { - Assert.fail("Couldn't read the state directory : " + baseDir.getPath()); - } finally { - streams.close(); - streams.cleanUp(); - Utils.delete(baseDir); - } - } + PowerMock.expectNew(StateDirectory.class, + anyObject(StreamsConfig.class), + anyObject(Time.class), + EasyMock.eq(shouldFilesExist) + ).andReturn(stateDirectory); - private void verifyCleanupStateDir(final String appDir, - final File oldTaskDir) throws InterruptedException { - final File taskDir = new File(appDir, "0_0"); - TestUtils.waitForCondition( - () -> !oldTaskDir.exists() && taskDir.exists(), - "cleanup has not successfully run"); - assertTrue(taskDir.exists()); - } + PowerMock.replayAll(); - public static class StateListenerStub implements KafkaStreams.StateListener { - int numChanges = 0; - KafkaStreams.State oldState; - KafkaStreams.State newState; - public Map mapStates = new HashMap<>(); + new KafkaStreams(topology, props, supplier, time); - @Override - public void onChange(final KafkaStreams.State newState, - final KafkaStreams.State oldState) { - final long prevCount = mapStates.containsKey(newState) ? mapStates.get(newState) : 0; - numChanges++; - this.oldState = oldState; - this.newState = newState; - mapStates.put(newState, prevCount + 1); - } + PowerMock.verifyAll(); } - } diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java index 92b471c77ad5..197e2344153d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.ForeachAction; +import org.apache.kafka.streams.kstream.Grouped; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.Joined; import org.apache.kafka.streams.kstream.KStream; @@ -450,8 +451,8 @@ public void shouldUseSpecifiedNameForTableSourceProcessor() { assertSpecifiedNameForOperation( topology, + expected + "-source", expected, - expected + "-table-source", "KSTREAM-SOURCE-0000000004", "KTABLE-SOURCE-0000000005"); } @@ -738,6 +739,27 @@ public void shouldUseSpecifiedNameForToStreamWithMapper() { "KSTREAM-KEY-SELECT-0000000004"); } + @Test + public void shouldUseSpecifiedNameForAggregateOperationGivenTable() { + builder.table(STREAM_TOPIC).groupBy(KeyValue::pair, Grouped.as("group-operation")).count(Named.as(STREAM_OPERATION_NAME)); + builder.build(); + final ProcessorTopology topology = builder.internalTopologyBuilder.rewriteTopology(new StreamsConfig(props)).build(); + assertSpecifiedNameForStateStore( + topology.stateStores(), + STREAM_TOPIC + "-STATE-STORE-0000000000", + "KTABLE-AGGREGATE-STATE-STORE-0000000004"); + + assertSpecifiedNameForOperation( + topology, + "KSTREAM-SOURCE-0000000001", + "KTABLE-SOURCE-0000000002", + "group-operation", + STREAM_OPERATION_NAME + "-sink", + STREAM_OPERATION_NAME + "-source", + STREAM_OPERATION_NAME); + } + + private static void assertSpecifiedNameForOperation(final ProcessorTopology topology, final String... expected) { final List processors = topology.processors(); assertEquals("Invalid number of expected processors", expected.length, processors.size()); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index c9a168912a0a..461500e52999 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -80,6 +81,31 @@ public void setUp() { streamsConfig = new StreamsConfig(props); } + @Test(expected = ConfigException.class) + public void testIllegalMetricsRecordingLevel() { + props.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "illegalConfig"); + new StreamsConfig(props); + } + + @Test + public void testOsDefaultSocketBufferSizes() { + props.put(StreamsConfig.SEND_BUFFER_CONFIG, CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND); + props.put(StreamsConfig.RECEIVE_BUFFER_CONFIG, CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND); + new StreamsConfig(props); + } + + @Test(expected = ConfigException.class) + public void testInvalidSocketSendBufferSize() { + props.put(StreamsConfig.SEND_BUFFER_CONFIG, -2); + new StreamsConfig(props); + } + + @Test(expected = ConfigException.class) + public void testInvalidSocketReceiveBufferSize() { + props.put(StreamsConfig.RECEIVE_BUFFER_CONFIG, -2); + new StreamsConfig(props); + } + @Test(expected = ConfigException.class) public void shouldThrowExceptionIfApplicationIdIsNotSet() { props.remove(StreamsConfig.APPLICATION_ID_CONFIG); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java index 0104d024e986..88659c355988 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java @@ -151,9 +151,12 @@ public class MetricsIntegrationTest { private static final String SKIPPED_RECORDS_TOTAL = "skipped-records-total"; private static final String RECORD_LATENESS_AVG = "record-lateness-avg"; private static final String RECORD_LATENESS_MAX = "record-lateness-max"; - private static final String HIT_RATIO_AVG = "hitRatio-avg"; - private static final String HIT_RATIO_MIN = "hitRatio-min"; - private static final String HIT_RATIO_MAX = "hitRatio-max"; + private static final String HIT_RATIO_AVG_BEFORE_24 = "hitRatio-avg"; + private static final String HIT_RATIO_MIN_BEFORE_24 = "hitRatio-min"; + private static final String HIT_RATIO_MAX_BEFORE_24 = "hitRatio-max"; + private static final String HIT_RATIO_AVG = "hit-ratio-avg"; + private static final String HIT_RATIO_MIN = "hit-ratio-min"; + private static final String HIT_RATIO_MAX = "hit-ratio-max"; // RocksDB metrics private static final String BYTES_WRITTEN_RATE = "bytes-written-rate"; @@ -275,7 +278,18 @@ private void closeApplication() throws Exception { } @Test - public void shouldAddMetricsOnAllLevels() throws Exception { + public void shouldAddMetricsOnAllLevelsWithBuiltInMetricsLatestVersion() throws Exception { + shouldAddMetricsOnAllLevels(StreamsConfig.METRICS_LATEST); + } + + @Test + public void shouldAddMetricsOnAllLevelsWithBuiltInMetricsVersion0100To23() throws Exception { + shouldAddMetricsOnAllLevels(StreamsConfig.METRICS_0100_TO_23); + } + + private void shouldAddMetricsOnAllLevels(final String builtInMetricsVersion) throws Exception { + streamsConfiguration.put(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); + builder.stream(STREAM_INPUT, Consumed.with(Serdes.Integer(), Serdes.String())) .to(STREAM_OUTPUT_1, Produced.with(Serdes.Integer(), Serdes.String())); builder.table(STREAM_OUTPUT_1, @@ -299,7 +313,7 @@ public void shouldAddMetricsOnAllLevels() throws Exception { checkKeyValueStoreMetricsByGroup(STREAM_STORE_ROCKSDB_STATE_METRICS); checkKeyValueStoreMetricsByGroup(STREAM_STORE_IN_MEMORY_LRU_STATE_METRICS); checkRocksDBMetricsByTag("rocksdb-state-id"); - checkCacheMetrics(); + checkCacheMetrics(builtInMetricsVersion); closeApplication(); @@ -461,9 +475,6 @@ private void checkRocksDBMetricsByTag(final String tag) { checkMetricByName(listMetricStore, MEMTABLE_BYTES_FLUSHED_RATE, 1); checkMetricByName(listMetricStore, MEMTABLE_BYTES_FLUSHED_TOTAL, 1); checkMetricByName(listMetricStore, MEMTABLE_HIT_RATIO, 1); - checkMetricByName(listMetricStore, MEMTABLE_FLUSH_TIME_AVG, 1); - checkMetricByName(listMetricStore, MEMTABLE_FLUSH_TIME_MIN, 1); - checkMetricByName(listMetricStore, MEMTABLE_FLUSH_TIME_MAX, 1); checkMetricByName(listMetricStore, WRITE_STALL_DURATION_AVG, 1); checkMetricByName(listMetricStore, WRITE_STALL_DURATION_TOTAL, 1); checkMetricByName(listMetricStore, BLOCK_CACHE_DATA_HIT_RATIO, 1); @@ -471,9 +482,6 @@ private void checkRocksDBMetricsByTag(final String tag) { checkMetricByName(listMetricStore, BLOCK_CACHE_FILTER_HIT_RATIO, 1); checkMetricByName(listMetricStore, BYTES_READ_DURING_COMPACTION_RATE, 1); checkMetricByName(listMetricStore, BYTES_WRITTEN_DURING_COMPACTION_RATE, 1); - checkMetricByName(listMetricStore, COMPACTION_TIME_AVG, 1); - checkMetricByName(listMetricStore, COMPACTION_TIME_MIN, 1); - checkMetricByName(listMetricStore, COMPACTION_TIME_MAX, 1); checkMetricByName(listMetricStore, NUMBER_OF_OPEN_FILES, 1); checkMetricByName(listMetricStore, NUMBER_OF_FILE_ERRORS, 1); } @@ -526,13 +534,25 @@ private void checkMetricsDeregistration() { assertThat(listMetricAfterClosingApp.size(), is(0)); } - private void checkCacheMetrics() { + private void checkCacheMetrics(final String builtInMetricsVersion) { final List listMetricCache = new ArrayList(kafkaStreams.metrics().values()).stream() .filter(m -> m.metricName().group().equals(STREAM_CACHE_NODE_METRICS)) .collect(Collectors.toList()); - checkMetricByName(listMetricCache, HIT_RATIO_AVG, 6); - checkMetricByName(listMetricCache, HIT_RATIO_MIN, 6); - checkMetricByName(listMetricCache, HIT_RATIO_MAX, 6); + checkMetricByName( + listMetricCache, + builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? HIT_RATIO_AVG : HIT_RATIO_AVG_BEFORE_24, + builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? 3 : 6 /* includes parent sensors */ + ); + checkMetricByName( + listMetricCache, + builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? HIT_RATIO_MIN : HIT_RATIO_MIN_BEFORE_24, + builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? 3 : 6 /* includes parent sensors */ + ); + checkMetricByName( + listMetricCache, + builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? HIT_RATIO_MAX : HIT_RATIO_MAX_BEFORE_24, + builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? 3 : 6 /* includes parent sensors */ + ); } private void checkWindowStoreMetrics() { @@ -621,7 +641,7 @@ private void checkMetricByName(final List listMetric, final String metri final List metrics = listMetric.stream() .filter(m -> m.metricName().name().equals(metricName)) .collect(Collectors.toList()); - Assert.assertEquals("Size of metrics of type:'" + metricName + "' must be equal to:" + numMetric + " but it's equal to " + metrics.size(), numMetric, metrics.size()); + Assert.assertEquals("Size of metrics of type:'" + metricName + "' must be equal to " + numMetric + " but it's equal to " + metrics.size(), numMetric, metrics.size()); for (final Metric m : metrics) { Assert.assertNotNull("Metric:'" + m.metricName() + "' must be not null", m.metricValue()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java new file mode 100644 index 000000000000..22083c117ab4 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java @@ -0,0 +1,335 @@ +/* + * 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.streams.integration; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assert.fail; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KafkaStreams.State; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.processor.StateRestoreListener; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(IntegrationTest.class) +public class OptimizedKTableIntegrationTest { + private static final int NUM_BROKERS = 1; + + private static final String INPUT_TOPIC_NAME = "input-topic"; + private static final String TABLE_NAME = "source-table"; + + @Rule + public final EmbeddedKafkaCluster cluster = new EmbeddedKafkaCluster(NUM_BROKERS); + + private final Map kafkaStreamsStates = new HashMap<>(); + private final Lock kafkaStreamsStatesLock = new ReentrantLock(); + private final Condition kafkaStreamsStateUpdate = kafkaStreamsStatesLock.newCondition(); + private final MockTime mockTime = cluster.time; + + @Before + public void before() throws InterruptedException { + cluster.createTopic(INPUT_TOPIC_NAME, 2, 1); + } + + @After + public void after() { + for (final KafkaStreams kafkaStreams : kafkaStreamsStates.keySet()) { + kafkaStreams.close(); + } + } + + @Test + public void standbyShouldNotPerformRestoreAtStartup() throws Exception { + final int numMessages = 10; + final int key = 1; + final Semaphore semaphore = new Semaphore(0); + + final StreamsBuilder builder = new StreamsBuilder(); + builder + .table(INPUT_TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer()), + Materialized.>as(TABLE_NAME) + .withCachingDisabled()) + .toStream() + .peek((k, v) -> semaphore.release()); + + final KafkaStreams kafkaStreams1 = createKafkaStreams(builder, streamsConfiguration()); + final KafkaStreams kafkaStreams2 = createKafkaStreams(builder, streamsConfiguration()); + final List kafkaStreamsList = Arrays.asList(kafkaStreams1, kafkaStreams2); + + produceValueRange(key, 0, 10); + + final AtomicLong restoreStartOffset = new AtomicLong(-1); + kafkaStreamsList.forEach(kafkaStreams -> { + kafkaStreams.setGlobalStateRestoreListener(createTrackingRestoreListener(restoreStartOffset, new AtomicLong())); + kafkaStreams.start(); + }); + waitForKafkaStreamssToEnterRunningState(kafkaStreamsList, 60, TimeUnit.SECONDS); + + // Assert that all messages in the first batch were processed in a timely manner + assertThat(semaphore.tryAcquire(numMessages, 60, TimeUnit.SECONDS), is(equalTo(true))); + + // Assert that no restore occurred + assertThat(restoreStartOffset.get(), is(equalTo(-1L))); + } + + @Test + public void shouldApplyUpdatesToStandbyStore() throws Exception { + final int batch1NumMessages = 100; + final int batch2NumMessages = 100; + final int key = 1; + final Semaphore semaphore = new Semaphore(0); + + final StreamsBuilder builder = new StreamsBuilder(); + builder + .table(INPUT_TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer()), + Materialized.>as(TABLE_NAME) + .withCachingDisabled()) + .toStream() + .peek((k, v) -> semaphore.release()); + + final KafkaStreams kafkaStreams1 = createKafkaStreams(builder, streamsConfiguration()); + final KafkaStreams kafkaStreams2 = createKafkaStreams(builder, streamsConfiguration()); + final List kafkaStreamsList = Arrays.asList(kafkaStreams1, kafkaStreams2); + + final AtomicLong restoreStartOffset = new AtomicLong(-1L); + final AtomicLong restoreEndOffset = new AtomicLong(-1L); + kafkaStreamsList.forEach(kafkaStreams -> { + kafkaStreams.setGlobalStateRestoreListener(createTrackingRestoreListener(restoreStartOffset, restoreEndOffset)); + kafkaStreams.start(); + }); + waitForKafkaStreamssToEnterRunningState(kafkaStreamsList, 60, TimeUnit.SECONDS); + + produceValueRange(key, 0, batch1NumMessages); + + // Assert that all messages in the first batch were processed in a timely manner + assertThat(semaphore.tryAcquire(batch1NumMessages, 60, TimeUnit.SECONDS), is(equalTo(true))); + + final ReadOnlyKeyValueStore store1 = kafkaStreams1 + .store(TABLE_NAME, QueryableStoreTypes.keyValueStore()); + + final ReadOnlyKeyValueStore store2 = kafkaStreams2 + .store(TABLE_NAME, QueryableStoreTypes.keyValueStore()); + + final boolean kafkaStreams1WasFirstActive; + if (store1.get(key) != null) { + kafkaStreams1WasFirstActive = true; + } else { + // Assert that data from the job was sent to the store + assertThat(store2.get(key), is(notNullValue())); + kafkaStreams1WasFirstActive = false; + } + + // Assert that no restore has occurred, ensures that when we check later that the restore + // notification actually came from after the rebalance. + assertThat(restoreStartOffset.get(), is(equalTo(-1L))); + + // Assert that the current value in store reflects all messages being processed + assertThat(kafkaStreams1WasFirstActive ? store1.get(key) : store2.get(key), is(equalTo(batch1NumMessages - 1))); + + if (kafkaStreams1WasFirstActive) { + kafkaStreams1.close(); + } else { + kafkaStreams2.close(); + } + + final ReadOnlyKeyValueStore newActiveStore = + kafkaStreams1WasFirstActive ? store2 : store1; + retryOnExceptionWithTimeout(100, 60 * 1000, TimeUnit.MILLISECONDS, () -> { + // Assert that after failover we have recovered to the last store write + assertThat(newActiveStore.get(key), is(equalTo(batch1NumMessages - 1))); + }); + + final int totalNumMessages = batch1NumMessages + batch2NumMessages; + + produceValueRange(key, batch1NumMessages, totalNumMessages); + + // Assert that all messages in the second batch were processed in a timely manner + assertThat(semaphore.tryAcquire(batch2NumMessages, 60, TimeUnit.SECONDS), is(equalTo(true))); + + // Assert that either restore was unnecessary or we restored from an offset later than 0 + assertThat(restoreStartOffset.get(), is(anyOf(greaterThan(0L), equalTo(-1L)))); + + // Assert that either restore was unnecessary or we restored to the last offset before we closed the kafkaStreams + assertThat(restoreEndOffset.get(), is(anyOf(equalTo(batch1NumMessages - 1), equalTo(-1L)))); + + // Assert that the current value in store reflects all messages being processed + assertThat(newActiveStore.get(key), is(equalTo(totalNumMessages - 1))); + } + + private void produceValueRange(final int key, final int start, final int endExclusive) throws Exception { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); + + IntegrationTestUtils.produceKeyValuesSynchronously( + INPUT_TOPIC_NAME, + IntStream.range(start, endExclusive) + .mapToObj(i -> KeyValue.pair(key, i)) + .collect(Collectors.toList()), + producerProps, + mockTime); + } + + private void retryOnExceptionWithTimeout(final long pollInterval, + final long timeout, + final TimeUnit timeUnit, + final Runnable runnable) throws InterruptedException { + final long expectedEnd = System.currentTimeMillis() + timeUnit.toMillis(timeout); + + while (true) { + try { + runnable.run(); + return; + } catch (final Throwable t) { + if (expectedEnd <= System.currentTimeMillis()) { + throw new AssertionError(t); + } + Thread.sleep(timeUnit.toMillis(pollInterval)); + } + } + } + + private void waitForKafkaStreamssToEnterRunningState(final Collection kafkaStreamss, + final long time, + final TimeUnit timeUnit) throws InterruptedException { + + final long expectedEnd = System.currentTimeMillis() + timeUnit.toMillis(time); + + kafkaStreamsStatesLock.lock(); + try { + while (!kafkaStreamss.stream().allMatch(kafkaStreams -> kafkaStreamsStates.get(kafkaStreams) == State.RUNNING)) { + if (expectedEnd <= System.currentTimeMillis()) { + fail("one or more kafkaStreamss did not enter RUNNING in a timely manner"); + } + final long millisRemaining = Math.max(1, expectedEnd - System.currentTimeMillis()); + kafkaStreamsStateUpdate.await(millisRemaining, TimeUnit.MILLISECONDS); + } + } finally { + kafkaStreamsStatesLock.unlock(); + } + } + + private KafkaStreams createKafkaStreams(final StreamsBuilder builder, final Properties config) { + final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(config), config); + kafkaStreamsStatesLock.lock(); + try { + kafkaStreamsStates.put(kafkaStreams, kafkaStreams.state()); + } finally { + kafkaStreamsStatesLock.unlock(); + } + + kafkaStreams.setStateListener((newState, oldState) -> { + kafkaStreamsStatesLock.lock(); + try { + kafkaStreamsStates.put(kafkaStreams, newState); + if (newState == State.RUNNING) { + if (kafkaStreamsStates.values().stream().allMatch(state -> state == State.RUNNING)) { + kafkaStreamsStateUpdate.signalAll(); + } + } + } finally { + kafkaStreamsStatesLock.unlock(); + } + }); + return kafkaStreams; + } + + private StateRestoreListener createTrackingRestoreListener(final AtomicLong restoreStartOffset, + final AtomicLong restoreEndOffset) { + return new StateRestoreListener() { + @Override + public void onRestoreStart(final TopicPartition topicPartition, + final String storeName, + final long startingOffset, + final long endingOffset) { + restoreStartOffset.set(startingOffset); + restoreEndOffset.set(endingOffset); + } + + @Override + public void onBatchRestored(final TopicPartition topicPartition, final String storeName, + final long batchEndOffset, final long numRestored) { + + } + + @Override + public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, + final long totalRestored) { + + } + }; + } + + private Properties streamsConfiguration() { + final String applicationId = "streamsApp"; + final Properties config = new Properties(); + config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE); + config.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); + config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath()); + config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + config.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); + config.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 100); + config.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 200); + config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); + return config; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java index 8c00ee4f4829..6cecb5691897 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java @@ -413,7 +413,9 @@ private static class TheConsumerRebalanceListener implements ConsumerRebalanceLi @Override public void onPartitionsRevoked(final Collection partitions) { - assignedTopics.clear(); + for (final TopicPartition partition : partitions) { + assignedTopics.remove(partition.topic()); + } listener.onPartitionsRevoked(partitions); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java new file mode 100644 index 000000000000..75fa48233a06 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java @@ -0,0 +1,190 @@ +/* + * 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.streams.integration; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValueTimestamp; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Properties; + +import static java.util.Arrays.asList; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkProperties; +import static org.apache.kafka.streams.StreamsConfig.AT_LEAST_ONCE; +import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateAfterTest; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.getStartedStreams; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +@RunWith(Parameterized.class) +@Category({IntegrationTest.class}) +public class ResetPartitionTimeIntegrationTest { + private static final int NUM_BROKERS = 1; + private static final Properties BROKER_CONFIG; + static { + BROKER_CONFIG = new Properties(); + BROKER_CONFIG.put("transaction.state.log.replication.factor", (short) 1); + BROKER_CONFIG.put("transaction.state.log.min.isr", 1); + } + @ClassRule + public static final EmbeddedKafkaCluster CLUSTER = + new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG, 0L); + + private static final StringDeserializer STRING_DESERIALIZER = new StringDeserializer(); + private static final StringSerializer STRING_SERIALIZER = new StringSerializer(); + private static final Serde STRING_SERDE = Serdes.String(); + private static final int DEFAULT_TIMEOUT = 100; + private final boolean eosEnabled; + private static long lastRecordedTimestamp = -2L; + + @Parameters(name = "{index}: eosEnabled={0}") + public static Collection parameters() { + return asList( + new Object[] {false}, + new Object[] {true} + ); + } + + public ResetPartitionTimeIntegrationTest(final boolean eosEnabled) { + this.eosEnabled = eosEnabled; + } + + @Test + public void shouldPreservePartitionTimeOnKafkaStreamRestart() throws Exception { + final String appId = "appId"; + final String input = "input"; + final String outputRaw = "output-raw"; + + cleanStateBeforeTest(CLUSTER, 2, input, outputRaw); + + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream( + input, + Consumed.with(STRING_SERDE, STRING_SERDE)) + .to(outputRaw); + + final Properties streamsConfig = new Properties(); + streamsConfig.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MaxTimestampExtractor.class); + streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); + streamsConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + streamsConfig.put(StreamsConfig.POLL_MS_CONFIG, Integer.toString(DEFAULT_TIMEOUT)); + streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Integer.toString(DEFAULT_TIMEOUT)); + streamsConfig.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosEnabled ? EXACTLY_ONCE : AT_LEAST_ONCE); + streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); + + KafkaStreams kafkaStreams = getStartedStreams(streamsConfig, builder, true); + try { + // start sending some records to have partition time committed + produceSynchronouslyToPartitionZero( + input, + asList( + new KeyValueTimestamp<>("k3", "v3", 5000) + ) + ); + verifyOutput( + outputRaw, + asList( + new KeyValueTimestamp<>("k3", "v3", 5000) + ) + ); + assertThat(lastRecordedTimestamp, is(-1L)); + lastRecordedTimestamp = -2L; + + kafkaStreams.close(); + assertThat(kafkaStreams.state(), is(KafkaStreams.State.NOT_RUNNING)); + + kafkaStreams = getStartedStreams(streamsConfig, builder, true); + + // resend some records and retrieve the last committed timestamp + produceSynchronouslyToPartitionZero( + input, + asList( + new KeyValueTimestamp<>("k5", "v5", 4999) + ) + ); + verifyOutput( + outputRaw, + asList( + new KeyValueTimestamp<>("k5", "v5", 4999) + ) + ); + assertThat(lastRecordedTimestamp, is(5000L)); + } finally { + kafkaStreams.close(); + cleanStateAfterTest(CLUSTER, kafkaStreams); + } + } + + public static final class MaxTimestampExtractor implements TimestampExtractor { + @Override + public long extract(final ConsumerRecord record, final long partitionTime) { + lastRecordedTimestamp = partitionTime; + return record.timestamp(); + } + } + + private void verifyOutput(final String topic, final List> keyValueTimestamps) { + final Properties properties = mkProperties( + mkMap( + mkEntry(ConsumerConfig.GROUP_ID_CONFIG, "test-group"), + mkEntry(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), + mkEntry(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ((Deserializer) STRING_DESERIALIZER).getClass().getName()), + mkEntry(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ((Deserializer) STRING_DESERIALIZER).getClass().getName()) + ) + ); + IntegrationTestUtils.verifyKeyValueTimestamps(properties, topic, keyValueTimestamps); + } + + private static void produceSynchronouslyToPartitionZero(final String topic, final List> toProduce) { + final Properties producerConfig = mkProperties(mkMap( + mkEntry(ProducerConfig.CLIENT_ID_CONFIG, "anything"), + mkEntry(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ((Serializer) STRING_SERIALIZER).getClass().getName()), + mkEntry(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ((Serializer) STRING_SERIALIZER).getClass().getName()), + mkEntry(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()) + )); + IntegrationTestUtils.produceSynchronously(producerConfig, false, topic, Optional.of(0), toProduce); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index d4d2e1c02117..f30ecedc3aaa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -94,6 +94,7 @@ public void start() throws IOException, InterruptedException { putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.GroupMinSessionTimeoutMsProp(), 0); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.GroupInitialRebalanceDelayMsProp(), 0); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.OffsetsTopicReplicationFactorProp(), (short) 1); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.OffsetsTopicPartitionsProp(), 5); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); for (int i = 0; i < brokers.length; i++) { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java index a0a61b978e74..fad89b7d7013 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java @@ -31,7 +31,7 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.Headers; -import org.apache.kafka.common.requests.UpdateMetadataRequest; +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KafkaStreams; @@ -705,13 +705,13 @@ public static void waitUntilMetadataIsPropagated(final List servers TestUtils.waitForCondition(() -> { for (final KafkaServer server : servers) { final MetadataCache metadataCache = server.dataPlaneRequestProcessor().metadataCache(); - final Option partitionInfo = + final Option partitionInfo = metadataCache.getPartitionInfo(topic, partition); if (partitionInfo.isEmpty()) { return false; } - final UpdateMetadataRequest.PartitionState metadataPartitionState = partitionInfo.get(); - if (!Request.isValidBrokerId(metadataPartitionState.basePartitionState.leader)) { + final UpdateMetadataPartitionState metadataPartitionState = partitionInfo.get(); + if (!Request.isValidBrokerId(metadataPartitionState.leader())) { return false; } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java index d1e5448fad06..b693dc7640fb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Merger; +import org.apache.kafka.streams.kstream.Named; import org.apache.kafka.streams.kstream.SessionWindowedKStream; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; @@ -277,14 +278,20 @@ public void shouldThrowNullPointerOnMaterializedReduceIfReducerIsNull() { } @Test(expected = NullPointerException.class) + @SuppressWarnings("unchecked") public void shouldThrowNullPointerOnMaterializedReduceIfMaterializedIsNull() { - stream.reduce(MockReducer.STRING_ADDER, - null); + stream.reduce(MockReducer.STRING_ADDER, (Materialized) null); + } + + @Test(expected = NullPointerException.class) + @SuppressWarnings("unchecked") + public void shouldThrowNullPointerOnMaterializedReduceIfNamedIsNull() { + stream.reduce(MockReducer.STRING_ADDER, (Named) null); } @Test(expected = NullPointerException.class) public void shouldThrowNullPointerOnCountIfMaterializedIsNull() { - stream.count(null); + stream.count((Materialized>) null); } private void processData(final TopologyTestDriver driver) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java index 0c4685a45e3a..0327ebd20cad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.streams.kstream.Grouped; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Named; import org.apache.kafka.streams.kstream.TimeWindowedKStream; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.Windowed; @@ -292,15 +293,24 @@ public void shouldThrowNullPointerOnMaterializedReduceIfReducerIsNull() { } @Test(expected = NullPointerException.class) + @SuppressWarnings("unchecked") public void shouldThrowNullPointerOnMaterializedReduceIfMaterializedIsNull() { windowedStream.reduce( MockReducer.STRING_ADDER, - null); + (Materialized) null); + } + + @Test(expected = NullPointerException.class) + @SuppressWarnings("unchecked") + public void shouldThrowNullPointerOnMaterializedReduceIfNamedIsNull() { + windowedStream.reduce( + MockReducer.STRING_ADDER, + (Named) null); } @Test(expected = NullPointerException.class) public void shouldThrowNullPointerOnCountIfMaterializedIsNull() { - windowedStream.count(null); + windowedStream.count((Materialized>) null); } private void processData(final TopologyTestDriver driver) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java index c5080d780310..a8526bf82067 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java @@ -16,20 +16,28 @@ */ package org.apache.kafka.streams.processor.internals; +import static org.apache.kafka.streams.processor.internals.ProcessorTopologyFactories.withLocalStores; +import static org.easymock.EasyMock.expect; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.File; +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.MockConsumer; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.AuthorizationException; -import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.LockException; -import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.test.InternalMockProcessorContext; @@ -40,22 +48,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.File; -import java.io.IOException; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import static org.apache.kafka.streams.processor.internals.ProcessorTopologyFactories.withLocalStores; -import static org.easymock.EasyMock.expect; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - public class AbstractTaskTest { private final TaskId id = new TaskId(0, 0); @@ -72,27 +64,6 @@ public void before() { expect(stateDirectory.directoryForTask(id)).andReturn(TestUtils.tempDirectory()); } - @Test(expected = ProcessorStateException.class) - public void shouldThrowProcessorStateExceptionOnInitializeOffsetsWhenAuthorizationException() { - final Consumer consumer = mockConsumer(new AuthorizationException("blah")); - final AbstractTask task = createTask(consumer, Collections.emptyMap()); - task.updateOffsetLimits(); - } - - @Test(expected = ProcessorStateException.class) - public void shouldThrowProcessorStateExceptionOnInitializeOffsetsWhenKafkaException() { - final Consumer consumer = mockConsumer(new KafkaException("blah")); - final AbstractTask task = createTask(consumer, Collections.emptyMap()); - task.updateOffsetLimits(); - } - - @Test(expected = WakeupException.class) - public void shouldThrowWakeupExceptionOnInitializeOffsetsWhenWakeupException() { - final Consumer consumer = mockConsumer(new WakeupException()); - final AbstractTask task = createTask(consumer, Collections.emptyMap()); - task.updateOffsetLimits(); - } - @Test public void shouldThrowLockExceptionIfFailedToLockStateDirectoryWhenTopologyHasStores() throws IOException { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); @@ -246,6 +217,9 @@ private AbstractTask createTask(final Consumer consumer, stateDirectory, config) { + @Override + public void initializeTaskTime() {} + @Override public void resume() {} @@ -270,14 +244,4 @@ public boolean initializeStateStores() { public void initializeTopology() {} }; } - - private Consumer mockConsumer(final RuntimeException toThrow) { - return new MockConsumer(OffsetResetStrategy.EARLIEST) { - @Override - public OffsetAndMetadata committed(final TopicPartition partition) { - throw toThrow; - } - }; - } - } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasksTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasksTest.java index a00969debd3d..94808b8043a7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasksTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasksTest.java @@ -26,8 +26,10 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Set; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetResetStrategy; @@ -62,11 +64,15 @@ public class AssignedStreamsTasksTest { private final TaskId taskId2 = new TaskId(1, 0); private AssignedStreamsTasks assignedTasks; + private final List revokedChangelogs = new ArrayList<>(); + @Before public void before() { assignedTasks = new AssignedStreamsTasks(new LogContext("log ")); EasyMock.expect(t1.id()).andReturn(taskId1).anyTimes(); EasyMock.expect(t2.id()).andReturn(taskId2).anyTimes(); + + revokedChangelogs.clear(); } @Test @@ -84,16 +90,20 @@ public void shouldInitializeNewTasks() { @Test public void shouldMoveInitializedTasksNeedingRestoreToRestoring() { EasyMock.expect(t1.initializeStateStores()).andReturn(false); + t1.initializeTaskTime(); t1.initializeTopology(); EasyMock.expectLastCall().once(); EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)); EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptySet()); EasyMock.expect(t2.initializeStateStores()).andReturn(true); + t1.initializeTaskTime(); t2.initializeTopology(); EasyMock.expectLastCall().once(); final Set t2partitions = Collections.singleton(tp2); EasyMock.expect(t2.partitions()).andReturn(t2partitions); EasyMock.expect(t2.changelogPartitions()).andReturn(Collections.emptyList()); + t1.initializeTaskTime(); + t2.initializeTaskTime(); EasyMock.replay(t1, t2); @@ -110,6 +120,7 @@ public void shouldMoveInitializedTasksNeedingRestoreToRestoring() { @Test public void shouldMoveInitializedTasksThatDontNeedRestoringToRunning() { EasyMock.expect(t2.initializeStateStores()).andReturn(true); + t2.initializeTaskTime(); t2.initializeTopology(); EasyMock.expectLastCall().once(); EasyMock.expect(t2.partitions()).andReturn(Collections.singleton(tp2)); @@ -130,6 +141,7 @@ public void shouldTransitionFullyRestoredTasksToRunning() { EasyMock.expect(t1.partitions()).andReturn(task1Partitions).anyTimes(); EasyMock.expect(t1.changelogPartitions()).andReturn(Utils.mkSet(changeLog1, changeLog2)).anyTimes(); EasyMock.expect(t1.hasStateStores()).andReturn(true).anyTimes(); + t1.initializeTaskTime(); t1.initializeTopology(); EasyMock.expectLastCall().once(); EasyMock.replay(t1); @@ -149,33 +161,35 @@ public void shouldSuspendRunningTasks() { assertThat(suspendTask(), nullValue()); - assertThat(assignedTasks.previousTaskIds(), equalTo(Collections.singleton(taskId1))); + assertThat(assignedTasks.suspendedTaskIds(), equalTo(Collections.singleton(taskId1))); EasyMock.verify(t1); } @Test public void shouldCloseRestoringTasks() { EasyMock.expect(t1.initializeStateStores()).andReturn(false); - EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)); - EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptySet()); + EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)).times(2); + EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptySet()).times(2); t1.closeStateManager(true); EasyMock.expectLastCall(); EasyMock.replay(t1); addAndInitTask(); - assertThat(assignedTasks.closeAllRestoringTasks(), nullValue()); + assertThat(assignedTasks.closeRestoringTasks(assignedTasks.restoringTaskIds(), revokedChangelogs), nullValue()); EasyMock.verify(t1); } @Test public void shouldClosedUnInitializedTasksOnSuspend() { + EasyMock.expect(t1.changelogPartitions()).andAnswer(Collections::emptyList); + t1.close(false, false); EasyMock.expectLastCall(); EasyMock.replay(t1); assignedTasks.addNewTask(t1); - assertThat(assignedTasks.suspend(), nullValue()); + assertThat(assignedTasks.suspendOrCloseTasks(assignedTasks.allAssignedTaskIds(), revokedChangelogs), nullValue()); EasyMock.verify(t1); } @@ -186,7 +200,7 @@ public void shouldNotSuspendSuspendedTasks() { EasyMock.replay(t1); assertThat(suspendTask(), nullValue()); - assertThat(assignedTasks.suspend(), nullValue()); + assertThat(assignedTasks.suspendOrCloseTasks(assignedTasks.allAssignedTaskIds(), revokedChangelogs), nullValue()); EasyMock.verify(t1); } @@ -194,20 +208,28 @@ public void shouldNotSuspendSuspendedTasks() { @Test public void shouldCloseTaskOnSuspendWhenRuntimeException() { mockTaskInitialization(); + EasyMock.expect(t1.partitions()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptySet()).anyTimes(); + t1.suspend(); EasyMock.expectLastCall().andThrow(new RuntimeException("KABOOM!")); t1.close(false, false); EasyMock.expectLastCall(); + EasyMock.replay(t1); assertThat(suspendTask(), not(nullValue())); - assertThat(assignedTasks.previousTaskIds(), equalTo(Collections.singleton(taskId1))); + assertTrue(assignedTasks.runningTaskIds().isEmpty()); + assertTrue(assignedTasks.suspendedTaskIds().isEmpty()); EasyMock.verify(t1); } @Test public void shouldCloseTaskOnSuspendIfTaskMigratedException() { mockTaskInitialization(); + EasyMock.expect(t1.partitions()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptySet()).anyTimes(); + t1.suspend(); EasyMock.expectLastCall().andThrow(new TaskMigratedException()); t1.close(false, true); @@ -215,7 +237,7 @@ public void shouldCloseTaskOnSuspendIfTaskMigratedException() { EasyMock.replay(t1); assertThat(suspendTask(), nullValue()); - assertTrue(assignedTasks.previousTaskIds().isEmpty()); + assertTrue(assignedTasks.runningTaskIds().isEmpty()); EasyMock.verify(t1); } @@ -224,6 +246,7 @@ public void shouldResumeMatchingSuspendedTasks() { mockRunningTaskSuspension(); t1.resume(); EasyMock.expectLastCall(); + t1.initializeTaskTime(); t1.initializeTopology(); EasyMock.expectLastCall().once(); EasyMock.replay(t1); @@ -239,6 +262,7 @@ public void shouldResumeMatchingSuspendedTasks() { public void shouldCloseTaskOnResumeSuspendedIfTaskMigratedException() { mockRunningTaskSuspension(); t1.resume(); + t1.initializeTaskTime(); t1.initializeTopology(); EasyMock.expectLastCall().andThrow(new TaskMigratedException()); t1.close(false, true); @@ -258,6 +282,7 @@ public void shouldCloseTaskOnResumeSuspendedIfTaskMigratedException() { private void mockTaskInitialization() { EasyMock.expect(t1.initializeStateStores()).andReturn(true); + t1.initializeTaskTime(); t1.initializeTopology(); EasyMock.expectLastCall().once(); EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)); @@ -520,7 +545,7 @@ public void shouldCloseCleanlyWithSuspendedTaskAndEOS() { assignedTasks.addNewTask(task); assignedTasks.initializeNewTasks(); - assertNull(assignedTasks.suspend()); + assertNull(assignedTasks.suspendOrCloseTasks(assignedTasks.allAssignedTaskIds(), revokedChangelogs)); assignedTasks.close(true); } @@ -532,11 +557,12 @@ private void addAndInitTask() { private RuntimeException suspendTask() { addAndInitTask(); - return assignedTasks.suspend(); + return assignedTasks.suspendOrCloseTasks(assignedTasks.allAssignedTaskIds(), revokedChangelogs); } private void mockRunningTaskSuspension() { EasyMock.expect(t1.initializeStateStores()).andReturn(true); + t1.initializeTaskTime(); t1.initializeTopology(); EasyMock.expectLastCall().once(); EasyMock.expect(t1.hasStateStores()).andReturn(false).anyTimes(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java index 508b000789ee..ce08a6ac83e2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java @@ -101,15 +101,18 @@ public String newStoreName(final String prefix) { properties.put(StreamsConfig.APPLICATION_ID_CONFIG, "blah"); properties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()); config = new StreamsConfig(properties); - globalStreamThread = new GlobalStreamThread(builder.rewriteTopology(config).buildGlobalStateTopology(), - config, - mockConsumer, - new StateDirectory(config, time, true), - 0, - new Metrics(), - new MockTime(), - "clientId", - stateRestoreListener); + globalStreamThread = new GlobalStreamThread( + builder.rewriteTopology(config).buildGlobalStateTopology(), + config, + mockConsumer, + new StateDirectory(config, time, true), + 0, + new Metrics(), + new MockTime(), + "clientId", + stateRestoreListener, + null + ); } @Test @@ -134,15 +137,18 @@ public List partitionsFor(final String topic) { throw new RuntimeException("KABOOM!"); } }; - globalStreamThread = new GlobalStreamThread(builder.buildGlobalStateTopology(), - config, - mockConsumer, - new StateDirectory(config, time, true), - 0, - new Metrics(), - new MockTime(), - "clientId", - stateRestoreListener); + globalStreamThread = new GlobalStreamThread( + builder.buildGlobalStateTopology(), + config, + mockConsumer, + new StateDirectory(config, time, true), + 0, + new Metrics(), + new MockTime(), + "clientId", + stateRestoreListener, + null + ); try { globalStreamThread.start(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockChangelogReader.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockChangelogReader.java index 13309675fd6d..9cc51cedb304 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockChangelogReader.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockChangelogReader.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.List; import org.apache.kafka.common.TopicPartition; import java.util.Collection; @@ -48,10 +49,17 @@ void setRestoredOffsets(final Map restoredOffsets) { } @Override - public void reset() { + public void clear() { registered.clear(); } + @Override + public void remove(final List revokedPartitions) { + for (final TopicPartition partition : revokedPartitions) { + restoredOffsets.remove(partition); + } + } + public boolean wasRegistered(final TopicPartition partition) { return registered.contains(partition); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java index bd3553083ff2..273f4b9e45ad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java @@ -17,11 +17,12 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; public class MockStreamsMetrics extends StreamsMetricsImpl { public MockStreamsMetrics(final Metrics metrics) { - super(metrics, "test"); + super(metrics, "test", StreamsConfig.METRICS_LATEST); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java index cfc814f2e42c..3584f9c9a30d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java @@ -40,12 +40,15 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; public class PartitionGroupTest { private final LogContext logContext = new LogContext(); private final Serializer intSerializer = new IntegerSerializer(); private final Deserializer intDeserializer = new IntegerDeserializer(); private final TimestampExtractor timestampExtractor = new MockTimestampExtractor(); + private final TopicPartition randomPartition = new TopicPartition("random-partition", 0); + private final String errMessage = "Partition " + randomPartition + " not found."; private final String[] topics = {"topic"}; private final TopicPartition partition1 = new TopicPartition(topics[0], 1); private final TopicPartition partition2 = new TopicPartition(topics[0], 2); @@ -86,7 +89,6 @@ private static Sensor getValueSensor(final Metrics metrics, final MetricName met @Test public void testTimeTracking() { assertEquals(0, group.numBuffered()); - // add three 3 records with timestamp 1, 3, 5 to partition-1 final List> list1 = Arrays.asList( new ConsumerRecord<>("topic", 1, 1L, recordKey, recordValue), @@ -119,6 +121,9 @@ record = group.nextRecord(info); // 2:[2, 4, 6] // st: 1 assertEquals(partition1, info.partition()); + assertEquals(3L, group.partitionTimestamp(partition1)); + assertEquals(2L, group.partitionTimestamp(partition2)); + assertEquals(1L, group.streamTime()); verifyTimes(record, 1L, 1L); verifyBuffered(5, 2, 3); assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue()); @@ -129,6 +134,9 @@ record = group.nextRecord(info); // 2:[4, 6] // st: 2 assertEquals(partition2, info.partition()); + assertEquals(3L, group.partitionTimestamp(partition1)); + assertEquals(4L, group.partitionTimestamp(partition2)); + assertEquals(2L, group.streamTime()); verifyTimes(record, 2L, 2L); verifyBuffered(4, 2, 2); assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue()); @@ -143,6 +151,8 @@ record = group.nextRecord(info); // 2:[4, 6] // st: 2 (just adding records shouldn't change it) verifyBuffered(6, 4, 2); + assertEquals(3L, group.partitionTimestamp(partition1)); + assertEquals(4L, group.partitionTimestamp(partition2)); assertEquals(2L, group.streamTime()); assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue()); @@ -152,6 +162,9 @@ record = group.nextRecord(info); // 2:[4, 6] // st: 3 assertEquals(partition1, info.partition()); + assertEquals(5L, group.partitionTimestamp(partition1)); + assertEquals(4L, group.partitionTimestamp(partition2)); + assertEquals(3L, group.streamTime()); verifyTimes(record, 3L, 3L); verifyBuffered(5, 3, 2); assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue()); @@ -162,6 +175,9 @@ record = group.nextRecord(info); // 2:[6] // st: 4 assertEquals(partition2, info.partition()); + assertEquals(5L, group.partitionTimestamp(partition1)); + assertEquals(6L, group.partitionTimestamp(partition2)); + assertEquals(4L, group.streamTime()); verifyTimes(record, 4L, 4L); verifyBuffered(4, 3, 1); assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue()); @@ -172,6 +188,9 @@ record = group.nextRecord(info); // 2:[6] // st: 5 assertEquals(partition1, info.partition()); + assertEquals(5L, group.partitionTimestamp(partition1)); + assertEquals(6L, group.partitionTimestamp(partition2)); + assertEquals(5L, group.streamTime()); verifyTimes(record, 5L, 5L); verifyBuffered(3, 2, 1); assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue()); @@ -182,6 +201,9 @@ record = group.nextRecord(info); // 2:[6] // st: 5 assertEquals(partition1, info.partition()); + assertEquals(5L, group.partitionTimestamp(partition1)); + assertEquals(6L, group.partitionTimestamp(partition2)); + assertEquals(5L, group.streamTime()); verifyTimes(record, 2L, 5L); verifyBuffered(2, 1, 1); assertEquals(3.0, metrics.metric(lastLatenessValue).metricValue()); @@ -192,6 +214,9 @@ record = group.nextRecord(info); // 2:[6] // st: 5 assertEquals(partition1, info.partition()); + assertEquals(5L, group.partitionTimestamp(partition1)); + assertEquals(6L, group.partitionTimestamp(partition2)); + assertEquals(5L, group.streamTime()); verifyTimes(record, 4L, 5L); verifyBuffered(1, 0, 1); assertEquals(1.0, metrics.metric(lastLatenessValue).metricValue()); @@ -202,10 +227,12 @@ record = group.nextRecord(info); // 2:[] // st: 6 assertEquals(partition2, info.partition()); + assertEquals(5L, group.partitionTimestamp(partition1)); + assertEquals(6L, group.partitionTimestamp(partition2)); + assertEquals(6L, group.streamTime()); verifyTimes(record, 6L, 6L); verifyBuffered(0, 0, 0); assertEquals(0.0, metrics.metric(lastLatenessValue).metricValue()); - } @Test @@ -266,4 +293,28 @@ private void verifyBuffered(final int totalBuffered, final int partitionOneBuffe assertEquals(partitionOneBuffered, group.numBuffered(partition1)); assertEquals(partitionTwoBuffered, group.numBuffered(partition2)); } + + @Test + public void shouldSetPartitionTimestampAndStreamTime() { + group.setPartitionTime(partition1, 100L); + assertEquals(100L, group.partitionTimestamp(partition1)); + assertEquals(100L, group.streamTime()); + group.setPartitionTime(partition2, 50L); + assertEquals(50L, group.partitionTimestamp(partition2)); + assertEquals(100L, group.streamTime()); + } + + @Test + public void shouldThrowNullpointerUponSetPartitionTimestampFailure() { + assertThrows(errMessage, NullPointerException.class, () -> { + group.setPartitionTime(randomPartition, 0L); + }); + } + + @Test + public void shouldThrowNullpointerUponGetPartitionTimestampFailure() { + assertThrows(errMessage, NullPointerException.class, () -> { + group.partitionTimestamp(randomPartition); + }); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java index 6dadb49b8ab8..c5f1a76f4130 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java @@ -209,6 +209,33 @@ public void shouldTrackPartitionTimeAsMaxSeenTimestamp() { assertEquals(queue.partitionTime(), 3L); } + @Test + public void shouldSetTimestampAndRespectMaxTimestampPolicy() { + assertTrue(queue.isEmpty()); + assertEquals(0, queue.size()); + assertEquals(RecordQueue.UNKNOWN, queue.headRecordTimestamp()); + queue.setPartitionTime(150L); + + final List> list1 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 200, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 100, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 300, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 400, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + + assertEquals(150L, queue.partitionTime()); + + queue.addRawRecords(list1); + + assertEquals(200L, queue.partitionTime()); + + queue.setPartitionTime(500L); + queue.poll(); + assertEquals(500L, queue.partitionTime()); + + queue.poll(); + assertEquals(500L, queue.partitionTime()); + } + @Test(expected = StreamsException.class) public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() { final byte[] key = Serdes.Long().serializer().serialize("foo", 1L); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 2faa078d6751..6f42fb26526e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -158,7 +160,8 @@ private StreamsConfig createConfig(final File baseDir) throws IOException { private final byte[] recordKey = intSerializer.serialize(null, 1); private final String threadName = "threadName"; - private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), threadName); + private final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(new Metrics(), threadName, StreamsConfig.METRICS_LATEST); @Before public void setup() throws Exception { @@ -554,6 +557,94 @@ private ConsumerRecord makeConsumerRecord(final TopicPartition t ); } + @Test + public void shouldNotGetConsumerCommittedOffsetIfThereAreNoRecordUpdates() throws IOException { + final AtomicInteger committedCallCount = new AtomicInteger(); + + final Consumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST) { + @Override + public synchronized Map committed(final Set partitions) { + committedCallCount.getAndIncrement(); + return super.committed(partitions); + } + }; + + consumer.assign(Collections.singletonList(globalTopicPartition)); + consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(0L)))); + + task = new StandbyTask( + taskId, + ktablePartitions, + ktableTopology, + consumer, + changelogReader, + createConfig(baseDir), + streamsMetrics, + stateDirectory + ); + task.initializeStateStores(); + assertThat(committedCallCount.get(), equalTo(0)); + + task.update(globalTopicPartition, Collections.emptyList()); + // We should not make a consumer.committed() call because there are no new records. + assertThat(committedCallCount.get(), equalTo(0)); + } + + @Test + public void shouldGetConsumerCommittedOffsetsOncePerCommit() throws IOException { + final AtomicInteger committedCallCount = new AtomicInteger(); + + final Consumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST) { + @Override + public synchronized Map committed(final Set partitions) { + committedCallCount.getAndIncrement(); + return super.committed(partitions); + } + }; + + consumer.assign(Collections.singletonList(globalTopicPartition)); + consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(0L)))); + + task = new StandbyTask( + taskId, + ktablePartitions, + ktableTopology, + consumer, + changelogReader, + createConfig(baseDir), + streamsMetrics, + stateDirectory + ); + task.initializeStateStores(); + + task.update( + globalTopicPartition, + Collections.singletonList( + makeConsumerRecord(globalTopicPartition, 1, 1) + ) + ); + assertThat(committedCallCount.get(), equalTo(1)); + + task.update( + globalTopicPartition, + Collections.singletonList( + makeConsumerRecord(globalTopicPartition, 1, 1) + ) + ); + // We should not make another consumer.committed() call until we commit + assertThat(committedCallCount.get(), equalTo(1)); + + task.commit(); + task.update( + globalTopicPartition, + Collections.singletonList( + makeConsumerRecord(globalTopicPartition, 1, 1) + ) + ); + // We committed so we're allowed to make another consumer.committed() call + assertThat(committedCallCount.get(), equalTo(2)); + } + @Test public void shouldInitializeStateStoreWithoutException() throws IOException { final InternalStreamsBuilder builder = new InternalStreamsBuilder(new InternalTopologyBuilder()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index bf59c21f12c1..9ae9d798cb81 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -750,6 +750,64 @@ public void shouldNotThrowTaskMigratedExceptionIfEndOffsetNotExceededDuringResto assertThat(callback.restored.size(), equalTo(10)); } + @Test + public void shouldRestoreUpToOffsetLimit() { + setupConsumer(10, topicPartition); + changelogReader.register(new StateRestorer( + topicPartition, + restoreListener, + 2L, + 5, + true, + "storeName1", + identity())); + expect(active.restoringTaskFor(topicPartition)).andStubReturn(task); + replay(active, task); + changelogReader.restore(active); + + assertThat(callback.restored.size(), equalTo(3)); + assertAllCallbackStatesExecuted(callback, "storeName1"); + assertCorrectOffsetsReportedByListener(callback, 2L, 4L, 3L); + } + + @Test + public void shouldNotRestoreIfCheckpointIsEqualToOffsetLimit() { + setupConsumer(10, topicPartition); + changelogReader.register(new StateRestorer( + topicPartition, + restoreListener, + 5L, + 5, + true, + "storeName1", + identity())); + expect(active.restoringTaskFor(topicPartition)).andStubReturn(task); + replay(active, task); + changelogReader.restore(active); + + assertThat(callback.storeNameCalledStates.size(), equalTo(0)); + assertThat(callback.restored.size(), equalTo(0)); + } + + @Test + public void shouldNotRestoreIfCheckpointIsGreaterThanOffsetLimit() { + setupConsumer(10, topicPartition); + changelogReader.register(new StateRestorer( + topicPartition, + restoreListener, + 10L, + 5, + true, + "storeName1", + identity())); + expect(active.restoringTaskFor(topicPartition)).andStubReturn(task); + replay(active, task); + changelogReader.restore(active); + + assertThat(callback.storeNameCalledStates.size(), equalTo(0)); + assertThat(callback.restored.size(), equalTo(0)); + } + private void setupConsumer(final long messages, final TopicPartition topicPartition) { assignPartition(messages, topicPartition); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index fb196fdcda31..a4f529d8795f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -16,15 +16,19 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.AuthorizationException; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.MetricConfig; @@ -40,6 +44,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; +import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.processor.PunctuationType; @@ -65,7 +70,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.time.Duration; +import java.util.Base64; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -77,6 +84,7 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkProperties; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.nullValue; @@ -98,7 +106,7 @@ public class StreamTaskTest { private final String topic2 = "topic2"; private final TopicPartition partition1 = new TopicPartition(topic1, 1); private final TopicPartition partition2 = new TopicPartition(topic2, 1); - private final Set partitions = Utils.mkSet(partition1, partition2); + private final Set partitions = mkSet(partition1, partition2); private final MockSourceNode source1 = new MockSourceNode<>(new String[]{topic1}, intDeserializer, intDeserializer); private final MockSourceNode source2 = new MockSourceNode<>(new String[]{topic2}, intDeserializer, intDeserializer); @@ -146,6 +154,9 @@ public Map restoredOffsets() { private StreamTask task; private long punctuatedAt; + private static final String APPLICATION_ID = "stream-task-test"; + private static final long DEFAULT_TIMESTAMP = 1000; + private final Punctuator punctuator = new Punctuator() { @Override public void punctuate(final long timestamp) { @@ -185,7 +196,7 @@ static StreamsConfig createConfig(final boolean enableEoS) { throw new RuntimeException(e); } return new StreamsConfig(mkProperties(mkMap( - mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "stream-task-test"), + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"), mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"), mkEntry(StreamsConfig.STATE_DIR_CONFIG, canonicalPath), @@ -642,6 +653,78 @@ public void shouldRespectCommitNeeded() { assertFalse(task.commitNeeded()); } + @Test + public void shouldRestorePartitionTimeAfterRestartWithEosDisabled() { + createTaskWithProcessAndCommit(false); + + assertEquals(DEFAULT_TIMESTAMP, task.decodeTimestamp(consumer.committed(Collections.singleton(partition1)).get(partition1).metadata())); + // reset times here by creating a new task + task = createStatelessTask(createConfig(false)); + + task.initializeTaskTime(); + assertEquals(DEFAULT_TIMESTAMP, task.partitionTime(partition1)); + assertEquals(DEFAULT_TIMESTAMP, task.streamTime()); + } + + @Test + public void shouldRestorePartitionTimeAfterRestartWithEosEnabled() { + createTaskWithProcessAndCommit(true); + + // extract the committed metadata from MockProducer + final List>> metadataList = + producer.consumerGroupOffsetsHistory(); + final String storedMetadata = metadataList.get(0).get(APPLICATION_ID).get(partition1).metadata(); + final long partitionTime = task.decodeTimestamp(storedMetadata); + assertEquals(DEFAULT_TIMESTAMP, partitionTime); + + // since producer and consumer is mocked, we need to "connect" producer and consumer + // so we should manually commit offsets here to simulate this "connection" + final Map offsetMap = new HashMap<>(); + final String encryptedMetadata = task.encodeTimestamp(partitionTime); + offsetMap.put(partition1, new OffsetAndMetadata(partitionTime, encryptedMetadata)); + consumer.commitSync(offsetMap); + + // reset times here by creating a new task + task = createStatelessTask(createConfig(true)); + + task.initializeTaskTime(); + assertEquals(DEFAULT_TIMESTAMP, task.partitionTime(partition1)); + assertEquals(DEFAULT_TIMESTAMP, task.streamTime()); + } + + private void createTaskWithProcessAndCommit(final boolean eosEnabled) { + task = createStatelessTask(createConfig(eosEnabled)); + task.initializeStateStores(); + task.initializeTopology(); + + task.addRecords(partition1, singletonList(getConsumerRecord(partition1, DEFAULT_TIMESTAMP))); + + task.process(); + task.commit(); + } + + @Test + public void shouldEncodeAndDecodeMetadata() { + task = createStatelessTask(createConfig(false)); + assertEquals(DEFAULT_TIMESTAMP, task.decodeTimestamp(task.encodeTimestamp(DEFAULT_TIMESTAMP))); + } + + @Test + public void shouldReturnUnknownTimestampIfUnknownVersion() { + task = createStatelessTask(createConfig(false)); + + final byte[] emptyMessage = {StreamTask.LATEST_MAGIC_BYTE + 1}; + final String encodedString = Base64.getEncoder().encodeToString(emptyMessage); + assertEquals(RecordQueue.UNKNOWN, task.decodeTimestamp(encodedString)); + } + + @Test + public void shouldReturnUnknownTimestampIfEmptyMessage() { + task = createStatelessTask(createConfig(false)); + + assertEquals(RecordQueue.UNKNOWN, task.decodeTimestamp("")); + } + @Test public void shouldRespectCommitRequested() { task = createStatelessTask(createConfig(false)); @@ -1417,7 +1500,7 @@ public void shouldReturnOffsetsForRepartitionTopicsForPurging() { task = new StreamTask( taskId00, - Utils.mkSet(partition1, repartition), + mkSet(partition1, repartition), topology, consumer, changelogReader, @@ -1478,6 +1561,59 @@ public void punctuate(final long timestamp) { assertEquals(1, producer.history().size()); } + @Test(expected = ProcessorStateException.class) + public void shouldThrowProcessorStateExceptionOnInitializeOffsetsWhenAuthorizationException() { + final Consumer consumer = mockConsumerWithCommittedException(new AuthorizationException("message")); + final StreamTask task = createOptimizedStatefulTask(createConfig(false), consumer); + task.initializeStateStores(); + } + + @Test(expected = ProcessorStateException.class) + public void shouldThrowProcessorStateExceptionOnInitializeOffsetsWhenKafkaException() { + final Consumer consumer = mockConsumerWithCommittedException(new KafkaException("message")); + final AbstractTask task = createOptimizedStatefulTask(createConfig(false), consumer); + task.initializeStateStores(); + } + + @Test(expected = WakeupException.class) + public void shouldThrowWakeupExceptionOnInitializeOffsetsWhenWakeupException() { + final Consumer consumer = mockConsumerWithCommittedException(new WakeupException()); + final AbstractTask task = createOptimizedStatefulTask(createConfig(false), consumer); + task.initializeStateStores(); + } + + private Consumer mockConsumerWithCommittedException(final RuntimeException toThrow) { + return new MockConsumer(OffsetResetStrategy.EARLIEST) { + @Override + public Map committed(final Set partitions) { + throw toThrow; + } + }; + } + + private StreamTask createOptimizedStatefulTask(final StreamsConfig config, final Consumer consumer) { + final StateStore stateStore = new MockKeyValueStore(storeName, true); + + final ProcessorTopology topology = ProcessorTopologyFactories.with( + asList(source1), + mkMap(mkEntry(topic1, source1)), + singletonList(stateStore), + Collections.singletonMap(storeName, topic1)); + + return new StreamTask( + taskId00, + mkSet(partition1), + topology, + consumer, + changelogReader, + config, + streamsMetrics, + stateDirectory, + null, + time, + () -> producer = new MockProducer<>(false, bytesSerializer, bytesSerializer)); + } + private StreamTask createStatefulTask(final StreamsConfig config, final boolean logged) { final StateStore stateStore = new MockKeyValueStore(storeName, logged); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 5f7d9dd56daa..ef5b95b979af 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -79,6 +79,7 @@ import java.io.File; import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -222,7 +223,7 @@ public void testStateChangeStartClose() throws Exception { } private Cluster createCluster() { - final Node node = new Node(0, "localhost", 8121); + final Node node = new Node(-1, "localhost", 8121); return new Cluster( "mockClusterId", singletonList(node), @@ -255,7 +256,8 @@ private StreamThread createStreamThread(@SuppressWarnings("SameParameterValue") 0, stateDirectory, new MockStateRestoreListener(), - threadIdx); + threadIdx + ); } @Test @@ -340,7 +342,7 @@ public void shouldNotCommitBeforeTheCommitInterval() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 1); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -383,12 +385,14 @@ public void shouldRespectNumIterationsInMainLoop() { thread.setState(StreamThread.State.STARTING); thread.setState(StreamThread.State.PARTITIONS_REVOKED); + final TaskId task1 = new TaskId(0, t1p1.partition()); final Set assignedPartitions = Collections.singleton(t1p1); thread.taskManager().setAssignmentMetadata( Collections.singletonMap( - new TaskId(0, t1p1.partition()), + task1, assignedPartitions), Collections.emptyMap()); + thread.taskManager().setPartitionsToTaskId(Collections.singletonMap(t1p1, task1)); final MockConsumer mockConsumer = (MockConsumer) thread.consumer; mockConsumer.assign(Collections.singleton(t1p1)); @@ -466,7 +470,7 @@ public void shouldNotCauseExceptionIfNothingCommitted() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 0); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -501,7 +505,7 @@ public void shouldCommitAfterTheCommitInterval() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = mockTaskManagerCommit(consumer, 2, 1); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -583,13 +587,17 @@ public void shouldInjectProducerPerTaskUsingClientSupplierOnCreateIfEosEnable() final Map> activeTasks = new HashMap<>(); final List assignedPartitions = new ArrayList<>(); + final Map partitionsToTaskId = new HashMap<>(); // assign single partition assignedPartitions.add(t1p1); assignedPartitions.add(t1p2); activeTasks.put(task1, Collections.singleton(t1p1)); activeTasks.put(task2, Collections.singleton(t1p2)); + partitionsToTaskId.put(t1p1, task1); + partitionsToTaskId.put(t1p2, task2); + thread.taskManager().setPartitionsToTaskId(partitionsToTaskId); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.emptyMap()); final MockConsumer mockConsumer = (MockConsumer) thread.consumer; @@ -651,7 +659,8 @@ public void shouldShutdownTaskManagerOnClose() { EasyMock.expectLastCall(); EasyMock.replay(taskManager, consumer); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -684,7 +693,8 @@ public void shouldShutdownTaskManagerOnCloseWithoutStart() { EasyMock.expectLastCall(); EasyMock.replay(taskManager, consumer); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -717,6 +727,7 @@ public void shouldNotThrowWithoutPendingShutdownInRunOnce() { private void mockRunOnce(final boolean shutdownOnPoll) { final Collection assignedPartitions = Collections.singletonList(t1p1); + final Map partitionsToTaskId = Collections.singletonMap(t1p1, new TaskId(0, 1)); class MockStreamThreadConsumer extends MockConsumer { private StreamThread streamThread; @@ -731,6 +742,7 @@ public synchronized ConsumerRecords poll(final Duration timeout) { if (shutdownOnPoll) { streamThread.shutdown(); } + streamThread.taskManager().setPartitionsToTaskId(partitionsToTaskId); streamThread.rebalanceListener.onPartitionsAssigned(assignedPartitions); return super.poll(timeout); } @@ -755,8 +767,10 @@ private void setStreamThread(final StreamThread streamThread) { new AssignedStandbyTasks(new LogContext())); taskManager.setConsumer(mockStreamThreadConsumer); taskManager.setAssignmentMetadata(Collections.emptyMap(), Collections.emptyMap()); + taskManager.setPartitionsToTaskId(Collections.emptyMap()); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -790,7 +804,8 @@ public void shouldOnlyShutdownOnce() { EasyMock.expectLastCall(); EasyMock.replay(taskManager, consumer); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -844,15 +859,18 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerWasFencedWh consumer.updatePartitions(topic1, singletonList(new PartitionInfo(topic1, 1, null, null, null))); thread.setState(StreamThread.State.STARTING); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); + final Map partitionsToTaskId = new HashMap<>(); final List assignedPartitions = new ArrayList<>(); // assign single partition assignedPartitions.add(t1p1); activeTasks.put(task1, Collections.singleton(t1p1)); + partitionsToTaskId.put(t1p1, task1); + thread.taskManager().setPartitionsToTaskId(partitionsToTaskId); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.emptyMap()); final MockConsumer mockConsumer = (MockConsumer) thread.consumer; @@ -902,15 +920,18 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedIn internalTopologyBuilder.addSink("out", "output", null, null, null, "name"); thread.setState(StreamThread.State.STARTING); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); + final Map partitionsToTaskId = new HashMap<>(); final List assignedPartitions = new ArrayList<>(); // assign single partition assignedPartitions.add(t1p1); activeTasks.put(task1, Collections.singleton(t1p1)); + partitionsToTaskId.put(t1p1, task1); + thread.taskManager().setPartitionsToTaskId(partitionsToTaskId); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.emptyMap()); final MockConsumer mockConsumer = (MockConsumer) thread.consumer; @@ -923,7 +944,7 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedIn assertThat(thread.tasks().size(), equalTo(1)); clientSupplier.producers.get(0).fenceProducer(); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(assignedPartitions); assertTrue(clientSupplier.producers.get(0).transactionInFlight()); assertFalse(clientSupplier.producers.get(0).transactionCommitted()); assertTrue(clientSupplier.producers.get(0).closed()); @@ -938,15 +959,18 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedIn internalTopologyBuilder.addSink("out", "output", null, null, null, "name"); thread.setState(StreamThread.State.STARTING); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); + final Map partitionsToTaskId = new HashMap<>(); final List assignedPartitions = new ArrayList<>(); // assign single partition assignedPartitions.add(t1p1); activeTasks.put(task1, Collections.singleton(t1p1)); + partitionsToTaskId.put(t1p1, task1); + thread.taskManager().setPartitionsToTaskId(partitionsToTaskId); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.emptyMap()); final MockConsumer mockConsumer = (MockConsumer) thread.consumer; @@ -959,7 +983,7 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedIn assertThat(thread.tasks().size(), equalTo(1)); clientSupplier.producers.get(0).fenceProducerOnClose(); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(assignedPartitions); assertFalse(clientSupplier.producers.get(0).transactionInFlight()); assertTrue(clientSupplier.producers.get(0).transactionCommitted()); @@ -994,15 +1018,18 @@ public void shouldReturnActiveTaskMetadataWhileRunningState() { final StreamThread thread = createStreamThread(clientId, config, false); thread.setState(StreamThread.State.STARTING); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); final Map> activeTasks = new HashMap<>(); final List assignedPartitions = new ArrayList<>(); + final Map partitionsToTaskId = new HashMap<>(); // assign single partition assignedPartitions.add(t1p1); activeTasks.put(task1, Collections.singleton(t1p1)); + partitionsToTaskId.put(t1p1, task1); + thread.taskManager().setPartitionsToTaskId(partitionsToTaskId); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.emptyMap()); final MockConsumer mockConsumer = (MockConsumer) thread.consumer; @@ -1012,10 +1039,19 @@ public void shouldReturnActiveTaskMetadataWhileRunningState() { thread.runOnce(); - final ThreadMetadata threadMetadata = thread.threadMetadata(); - assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState()); - assertTrue(threadMetadata.activeTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1)))); - assertTrue(threadMetadata.standbyTasks().isEmpty()); + final ThreadMetadata metadata = thread.threadMetadata(); + assertEquals(StreamThread.State.RUNNING.name(), metadata.threadState()); + assertTrue(metadata.activeTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1)))); + assertTrue(metadata.standbyTasks().isEmpty()); + + assertTrue("#threadState() was: " + metadata.threadState() + "; expected either RUNNING, STARTING, PARTITIONS_REVOKED, PARTITIONS_ASSIGNED, or CREATED", + Arrays.asList("RUNNING", "STARTING", "PARTITIONS_REVOKED", "PARTITIONS_ASSIGNED", "CREATED").contains(metadata.threadState())); + final String threadName = metadata.threadName(); + assertTrue(threadName.startsWith("clientId-StreamThread-")); + assertEquals(threadName + "-consumer", metadata.consumerClientId()); + assertEquals(threadName + "-restore-consumer", metadata.restoreConsumerClientId()); + assertEquals(Collections.singleton(threadName + "-producer"), metadata.producerClientIds()); + assertEquals("clientId-admin", metadata.adminClientId()); } @Test @@ -1043,13 +1079,16 @@ public void shouldReturnStandbyTaskMetadataWhileRunningState() { restoreConsumer.updateBeginningOffsets(offsets); thread.setState(StreamThread.State.STARTING); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); final Map> standbyTasks = new HashMap<>(); + final Map partitionsToTaskId = new HashMap<>(); // assign single partition standbyTasks.put(task1, Collections.singleton(t1p1)); + partitionsToTaskId.put(t1p1, task1); + thread.taskManager().setPartitionsToTaskId(partitionsToTaskId); thread.taskManager().setAssignmentMetadata(Collections.emptyMap(), standbyTasks); thread.rebalanceListener.onPartitionsAssigned(Collections.emptyList()); @@ -1120,14 +1159,17 @@ public void shouldUpdateStandbyTask() throws Exception { } thread.setState(StreamThread.State.STARTING); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); final Map> standbyTasks = new HashMap<>(); + final Map partitionsToTaskId = new HashMap<>(); // assign single partition standbyTasks.put(task1, Collections.singleton(t1p1)); standbyTasks.put(task3, Collections.singleton(t2p1)); + partitionsToTaskId.put(t1p1, task1); + thread.taskManager().setPartitionsToTaskId(partitionsToTaskId); thread.taskManager().setAssignmentMetadata(Collections.emptyMap(), standbyTasks); thread.rebalanceListener.onPartitionsAssigned(Collections.emptyList()); @@ -1184,7 +1226,8 @@ private void setupInternalTopologyWithoutState() { private StandbyTask createStandbyTask() { final LogContext logContext = new LogContext("test"); final Logger log = logContext.logger(StreamThreadTest.class); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread.StandbyTaskCreator standbyTaskCreator = new StreamThread.StandbyTaskCreator( internalTopologyBuilder, config, @@ -1224,15 +1267,18 @@ public void close() {} final StreamThread thread = createStreamThread(clientId, config, false); thread.setState(StreamThread.State.STARTING); - thread.rebalanceListener.onPartitionsRevoked(null); + thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); final List assignedPartitions = new ArrayList<>(); final Map> activeTasks = new HashMap<>(); + final Map partitionsToTaskId = new HashMap<>(); // assign single partition assignedPartitions.add(t1p1); activeTasks.put(task1, Collections.singleton(t1p1)); + partitionsToTaskId.put(t1p1, task1); + thread.taskManager().setPartitionsToTaskId(partitionsToTaskId); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.emptyMap()); clientSupplier.consumer.assign(assignedPartitions); @@ -1351,7 +1397,9 @@ public void shouldRecoverFromInvalidOffsetExceptionOnRestoreAndFinishRestore() t final Set topicPartitionSet = Collections.singleton(topicPartition); final Map> activeTasks = new HashMap<>(); - activeTasks.put(new TaskId(0, 0), topicPartitionSet); + final TaskId task0 = new TaskId(0, 0); + activeTasks.put(task0, topicPartitionSet); + thread.taskManager().setPartitionsToTaskId(Collections.singletonMap(topicPartition, task0)); thread.taskManager().setAssignmentMetadata(activeTasks, Collections.emptyMap()); mockConsumer.updatePartitions( @@ -1457,11 +1505,11 @@ public void shouldRecordSkippedMetricForDeserializationException() { thread.setState(StreamThread.State.STARTING); thread.setState(StreamThread.State.PARTITIONS_REVOKED); + final TaskId task1 = new TaskId(0, t1p1.partition()); final Set assignedPartitions = Collections.singleton(t1p1); + thread.taskManager().setPartitionsToTaskId(Collections.singletonMap(t1p1, task1)); thread.taskManager().setAssignmentMetadata( - Collections.singletonMap( - new TaskId(0, t1p1.partition()), - assignedPartitions), + Collections.singletonMap(task1, assignedPartitions), Collections.emptyMap()); final MockConsumer mockConsumer = (MockConsumer) thread.consumer; @@ -1528,12 +1576,14 @@ public void shouldReportSkippedRecordsForInvalidTimestamps() { thread.setState(StreamThread.State.STARTING); thread.setState(StreamThread.State.PARTITIONS_REVOKED); + final TaskId task1 = new TaskId(0, t1p1.partition()); final Set assignedPartitions = Collections.singleton(t1p1); thread.taskManager().setAssignmentMetadata( Collections.singletonMap( - new TaskId(0, t1p1.partition()), + task1, assignedPartitions), Collections.emptyMap()); + thread.taskManager().setPartitionsToTaskId(Collections.singletonMap(t1p1, task1)); final MockConsumer mockConsumer = (MockConsumer) thread.consumer; mockConsumer.assign(Collections.singleton(t1p1)); @@ -1621,7 +1671,8 @@ public void producerMetricsVerificationWithoutEOS() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 0); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -1660,7 +1711,7 @@ public void adminClientMetricsVerification() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -1683,7 +1734,7 @@ public void adminClientMetricsVerification() { null, new MockTime()); - EasyMock.expect(taskManager.getAdminClient()).andReturn(adminClient); + EasyMock.expect(taskManager.adminClient()).andReturn(adminClient); EasyMock.expectLastCall(); EasyMock.replay(taskManager, consumer); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index 2886f51fb5d6..2bc465b50099 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -152,7 +152,7 @@ private void createMockTaskManager(final Set prevTasks, taskManager = EasyMock.createNiceMock(TaskManager.class); EasyMock.expect(taskManager.adminClient()).andReturn(null).anyTimes(); EasyMock.expect(taskManager.builder()).andReturn(builder).anyTimes(); - EasyMock.expect(taskManager.prevActiveTaskIds()).andReturn(prevTasks).anyTimes(); + EasyMock.expect(taskManager.previousRunningTaskIds()).andReturn(prevTasks).anyTimes(); EasyMock.expect(taskManager.cachedTasksIds()).andReturn(cachedTasks).anyTimes(); EasyMock.expect(taskManager.processId()).andReturn(processId).anyTimes(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 77e0254b9f05..020ee363c05f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -17,6 +17,8 @@ package org.apache.kafka.streams.processor.internals; +import java.util.ArrayList; +import java.util.List; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.DeleteRecordsResult; import org.apache.kafka.clients.admin.DeletedRecords; @@ -67,6 +69,7 @@ public class TaskManagerTest { private final TopicPartition t1p0 = new TopicPartition("t1", 0); private final Set taskId0Partitions = Utils.mkSet(t1p0); private final Map> taskId0Assignment = Collections.singletonMap(taskId0, taskId0Partitions); + private final Map taskId0PartitionToTaskId = Collections.singletonMap(t1p0, taskId0); @Mock(type = MockType.STRICT) private InternalTopologyBuilder.SubscriptionUpdates subscriptionUpdates; @@ -113,6 +116,10 @@ public class TaskManagerTest { private final TaskId task03 = new TaskId(0, 3); private final TaskId task11 = new TaskId(1, 1); + private final Set revokedTasks = new HashSet<>(); + private final List revokedPartitions = new ArrayList<>(); + private final List revokedChangelogs = Collections.emptyList(); + @Rule public final TemporaryFolder testFolder = new TemporaryFolder(); @@ -129,6 +136,7 @@ public void setUp() { active, standby); taskManager.setConsumer(consumer); + revokedChangelogs.clear(); } private void replay() { @@ -235,81 +243,77 @@ public void shouldReturnCachedTaskIdsFromDirectory() throws IOException { } @Test - public void shouldCloseActiveUnAssignedSuspendedTasksWhenCreatingNewTasks() { + public void shouldCloseActiveUnAssignedSuspendedTasksWhenClosingRevokedTasks() { mockSingleActiveTask(); - active.closeNonAssignedSuspendedTasks(taskId0Assignment); - expectLastCall(); + EasyMock.expect(active.closeNotAssignedSuspendedTasks(taskId0Assignment.keySet())).andReturn(null).once(); + expect(restoreConsumer.assignment()).andReturn(Collections.emptySet()); + replay(); taskManager.setAssignmentMetadata(taskId0Assignment, Collections.>emptyMap()); - taskManager.createTasks(taskId0Partitions); + taskManager.setAssignmentMetadata(Collections.>emptyMap(), Collections.>emptyMap()); + + taskManager.closeRevokedSuspendedTasks(); verify(active); } @Test - public void shouldCloseStandbyUnAssignedSuspendedTasksWhenCreatingNewTasks() { + public void shouldCloseStandbyUnassignedTasksWhenCreatingNewTasks() { mockSingleActiveTask(); - standby.closeNonAssignedSuspendedTasks(taskId0Assignment); - expectLastCall(); + EasyMock.expect(standby.closeRevokedStandbyTasks(taskId0Assignment)).andReturn(Collections.emptyList()).once(); replay(); taskManager.setAssignmentMetadata(taskId0Assignment, Collections.>emptyMap()); + taskManager.setPartitionsToTaskId(taskId0PartitionToTaskId); taskManager.createTasks(taskId0Partitions); verify(active); } @Test - public void shouldAddNonResumedActiveTasks() { + public void shouldAddNonResumedSuspendedTasks() { mockSingleActiveTask(); expect(active.maybeResumeSuspendedTask(taskId0, taskId0Partitions)).andReturn(false); active.addNewTask(EasyMock.same(streamTask)); replay(); + // Need to call this twice so task manager doesn't consider all partitions "new" + taskManager.setAssignmentMetadata(taskId0Assignment, Collections.>emptyMap()); taskManager.setAssignmentMetadata(taskId0Assignment, Collections.>emptyMap()); + taskManager.setPartitionsToTaskId(taskId0PartitionToTaskId); taskManager.createTasks(taskId0Partitions); verify(activeTaskCreator, active); } @Test - public void shouldNotAddResumedActiveTasks() { - checkOrder(active, true); - expect(active.maybeResumeSuspendedTask(taskId0, taskId0Partitions)).andReturn(true); + public void shouldAddNewActiveTasks() { + mockSingleActiveTask(); + active.addNewTask(EasyMock.same(streamTask)); replay(); taskManager.setAssignmentMetadata(taskId0Assignment, Collections.>emptyMap()); + taskManager.setPartitionsToTaskId(taskId0PartitionToTaskId); taskManager.createTasks(taskId0Partitions); - // should be no calls to activeTaskCreator and no calls to active.addNewTasks(..) - verify(active, activeTaskCreator); - } - - @Test - public void shouldAddNonResumedStandbyTasks() { - mockStandbyTaskExpectations(); - expect(standby.maybeResumeSuspendedTask(taskId0, taskId0Partitions)).andReturn(false); - standby.addNewTask(EasyMock.same(standbyTask)); - replay(); - - taskManager.setAssignmentMetadata(Collections.>emptyMap(), taskId0Assignment); - taskManager.createTasks(taskId0Partitions); - - verify(standbyTaskCreator, active); + verify(activeTaskCreator, active); } @Test - public void shouldNotAddResumedStandbyTasks() { + public void shouldNotAddResumedActiveTasks() { checkOrder(active, true); - expect(standby.maybeResumeSuspendedTask(taskId0, taskId0Partitions)).andReturn(true); + expect(active.maybeResumeSuspendedTask(taskId0, taskId0Partitions)).andReturn(true); replay(); - taskManager.setAssignmentMetadata(Collections.>emptyMap(), taskId0Assignment); + // Need to call this twice so task manager doesn't consider all partitions "new" + taskManager.setAssignmentMetadata(taskId0Assignment, Collections.>emptyMap()); + taskManager.setAssignmentMetadata(taskId0Assignment, Collections.>emptyMap()); + taskManager.setPartitionsToTaskId(taskId0PartitionToTaskId); taskManager.createTasks(taskId0Partitions); - // should be no calls to standbyTaskCreator and no calls to standby.addNewTasks(..) - verify(standby, standbyTaskCreator); + // should be no calls to activeTaskCreator and no calls to active.addNewTasks(..) + verify(active, activeTaskCreator); } @Test @@ -320,48 +324,46 @@ public void shouldPauseActivePartitions() { replay(); taskManager.setAssignmentMetadata(taskId0Assignment, Collections.>emptyMap()); + taskManager.setPartitionsToTaskId(taskId0PartitionToTaskId); taskManager.createTasks(taskId0Partitions); verify(consumer); } @Test public void shouldSuspendActiveTasks() { - expect(active.suspend()).andReturn(null); + expect(active.suspendOrCloseTasks(revokedTasks, revokedChangelogs)).andReturn(null); + expect(restoreConsumer.assignment()).andReturn(Collections.emptySet()); replay(); - taskManager.suspendTasksAndState(); + taskManager.suspendActiveTasksAndState(revokedPartitions); verify(active); } @Test - public void shouldSuspendStandbyTasks() { - expect(standby.suspend()).andReturn(null); - replay(); - - taskManager.suspendTasksAndState(); - verify(standby); - } - - @Test + @SuppressWarnings("unchecked") public void shouldUnassignChangelogPartitionsOnSuspend() { - restoreConsumer.unsubscribe(); + expect(active.suspendOrCloseTasks(revokedTasks, new ArrayList<>())) + .andAnswer(() -> { + ((List) EasyMock.getCurrentArguments()[1]).add(t1p0); + return null; + }); + expect(restoreConsumer.assignment()).andReturn(Collections.singleton(t1p0)); + + restoreConsumer.assign(Collections.emptySet()); expectLastCall(); replay(); - taskManager.suspendTasksAndState(); + taskManager.suspendActiveTasksAndState(Collections.emptySet()); verify(restoreConsumer); } @Test public void shouldThrowStreamsExceptionAtEndIfExceptionDuringSuspend() { - expect(active.suspend()).andReturn(new RuntimeException("")); - expect(standby.suspend()).andReturn(new RuntimeException("")); - expectLastCall(); - restoreConsumer.unsubscribe(); + expect(active.suspendOrCloseTasks(revokedTasks, revokedChangelogs)).andReturn(new RuntimeException("")); replay(); try { - taskManager.suspendTasksAndState(); + taskManager.suspendActiveTasksAndState(revokedPartitions); fail("Should have thrown streams exception"); } catch (final StreamsException e) { // expected @@ -401,6 +403,8 @@ public void shouldUnassignChangelogPartitionsOnShutdown() { @Test public void shouldInitializeNewActiveTasks() { + EasyMock.expect(restoreConsumer.assignment()).andReturn(Collections.emptySet()).once(); + EasyMock.expect(changeLogReader.restore(active)).andReturn(taskId0Partitions).once(); active.updateRestored(EasyMock.>anyObject()); expectLastCall(); replay(); @@ -411,6 +415,8 @@ public void shouldInitializeNewActiveTasks() { @Test public void shouldInitializeNewStandbyTasks() { + EasyMock.expect(restoreConsumer.assignment()).andReturn(Collections.emptySet()).once(); + EasyMock.expect(changeLogReader.restore(active)).andReturn(taskId0Partitions).once(); active.updateRestored(EasyMock.>anyObject()); expectLastCall(); replay(); @@ -421,6 +427,7 @@ public void shouldInitializeNewStandbyTasks() { @Test public void shouldRestoreStateFromChangeLogReader() { + EasyMock.expect(restoreConsumer.assignment()).andReturn(taskId0Partitions).once(); expect(changeLogReader.restore(active)).andReturn(taskId0Partitions); active.updateRestored(taskId0Partitions); expectLastCall(); @@ -432,6 +439,7 @@ public void shouldRestoreStateFromChangeLogReader() { @Test public void shouldResumeRestoredPartitions() { + EasyMock.expect(restoreConsumer.assignment()).andReturn(taskId0Partitions).once(); expect(changeLogReader.restore(active)).andReturn(taskId0Partitions); expect(active.allTasksRunning()).andReturn(true); expect(consumer.assignment()).andReturn(taskId0Partitions); @@ -475,6 +483,7 @@ public void shouldReturnFalseWhenOnlyActiveTasksAreRunning() { @Test public void shouldReturnFalseWhenThereAreStillNonRunningTasks() { expect(active.allTasksRunning()).andReturn(false); + EasyMock.expect(changeLogReader.restore(active)).andReturn(Collections.emptySet()).once(); replay(); assertFalse(taskManager.updateNewAndRestoringTasks()); @@ -623,10 +632,12 @@ public void shouldPunctuateActiveTasks() { @Test public void shouldNotResumeConsumptionUntilAllStoresRestored() { + EasyMock.expect(changeLogReader.restore(active)).andReturn(Collections.emptySet()).once(); expect(active.allTasksRunning()).andReturn(false); + final Consumer consumer = EasyMock.createStrictMock(Consumer.class); taskManager.setConsumer(consumer); - EasyMock.replay(active, consumer); + EasyMock.replay(active, consumer, changeLogReader); // shouldn't invoke `resume` method in consumer taskManager.updateNewAndRestoringTasks(); @@ -662,6 +673,9 @@ private void mockAssignStandbyPartitions(final long offset) { expectLastCall(); EasyMock.replay(task); + + EasyMock.expect(restoreConsumer.assignment()).andReturn(taskId0Partitions).once(); + EasyMock.expect(changeLogReader.restore(active)).andReturn(taskId0Partitions).once(); } private void mockStandbyTaskExpectations() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java index 4fd6f88a95df..b58911976616 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java @@ -23,6 +23,8 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.Sensor.RecordingLevel; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.junit.Test; @@ -50,6 +52,8 @@ public class StreamsMetricsImplTest extends EasyMockSupport { private final static String SENSOR_PREFIX_DELIMITER = "."; private final static String SENSOR_NAME_DELIMITER = ".s."; private final static String INTERNAL_PREFIX = "internal"; + private final static String THREAD_NAME = "test-thread"; + private final static String VERSION = StreamsConfig.METRICS_LATEST; private final Metrics metrics = new Metrics(); private final Sensor sensor = metrics.sensor("dummy"); @@ -58,22 +62,23 @@ public class StreamsMetricsImplTest extends EasyMockSupport { private final Map tags = mkMap(mkEntry("tag", "value")); private final String description1 = "description number one"; private final String description2 = "description number two"; + private final String description3 = "description number three"; private final MockTime time = new MockTime(0); + private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_NAME, VERSION); @Test public void shouldGetThreadLevelSensor() { final Metrics metrics = mock(Metrics.class); - final String threadName = "thread1"; + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_NAME, VERSION); final String sensorName = "sensor1"; final String expectedFullSensorName = - INTERNAL_PREFIX + SENSOR_PREFIX_DELIMITER + threadName + SENSOR_NAME_DELIMITER + sensorName; + INTERNAL_PREFIX + SENSOR_PREFIX_DELIMITER + THREAD_NAME + SENSOR_NAME_DELIMITER + sensorName; final RecordingLevel recordingLevel = RecordingLevel.DEBUG; final Sensor[] parents = {}; EasyMock.expect(metrics.sensor(expectedFullSensorName, recordingLevel, parents)).andReturn(null); replayAll(); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, threadName); final Sensor sensor = streamsMetrics.threadLevelSensor(sensorName, recordingLevel); verifyAll(); @@ -83,12 +88,11 @@ public void shouldGetThreadLevelSensor() { @Test(expected = NullPointerException.class) public void testNullMetrics() { - new StreamsMetricsImpl(null, ""); + new StreamsMetricsImpl(null, "", VERSION); } @Test(expected = NullPointerException.class) public void testRemoveNullSensor() { - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), ""); streamsMetrics.removeSensor(null); } @@ -98,7 +102,6 @@ public void testRemoveSensor() { final String scope = "scope"; final String entity = "entity"; final String operation = "put"; - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), ""); final Sensor sensor1 = streamsMetrics.addSensor(sensorName, Sensor.RecordingLevel.DEBUG); streamsMetrics.removeSensor(sensor1); @@ -116,9 +119,9 @@ public void testRemoveSensor() { } @Test - public void testMutiLevelSensorRemoval() { + public void testMultiLevelSensorRemoval() { final Metrics registry = new Metrics(); - final StreamsMetricsImpl metrics = new StreamsMetricsImpl(registry, ""); + final StreamsMetricsImpl metrics = new StreamsMetricsImpl(registry, THREAD_NAME, VERSION); for (final MetricName defaultMetric : registry.metrics().keySet()) { registry.removeMetric(defaultMetric); } @@ -169,7 +172,6 @@ public void testMutiLevelSensorRemoval() { @Test public void testLatencyMetrics() { - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), ""); final int defaultMetrics = streamsMetrics.metrics().size(); final String scope = "scope"; @@ -189,7 +191,6 @@ public void testLatencyMetrics() { @Test public void testThroughputMetrics() { - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), ""); final int defaultMetrics = streamsMetrics.metrics().size(); final String scope = "scope"; @@ -211,7 +212,7 @@ public void testTotalMetricDoesntDecrease() { final MockTime time = new MockTime(1); final MetricConfig config = new MetricConfig().timeWindow(1, TimeUnit.MILLISECONDS); final Metrics metrics = new Metrics(config, time); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, ""); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "", VERSION); final String scope = "scope"; final String entity = "entity"; @@ -245,8 +246,6 @@ public void testTotalMetricDoesntDecrease() { @Test public void shouldGetStoreLevelTagMap() { - final String threadName = "test-thread"; - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, threadName); final String taskName = "test-task"; final String storeType = "remote-window"; final String storeName = "window-keeper"; @@ -254,11 +253,40 @@ public void shouldGetStoreLevelTagMap() { final Map tagMap = streamsMetrics.storeLevelTagMap(taskName, storeType, storeName); assertThat(tagMap.size(), equalTo(3)); - assertThat(tagMap.get(StreamsMetricsImpl.THREAD_ID_TAG), equalTo(threadName)); + assertThat(tagMap.get(StreamsMetricsImpl.THREAD_ID_TAG_0100_TO_23), equalTo(THREAD_NAME)); assertThat(tagMap.get(StreamsMetricsImpl.TASK_ID_TAG), equalTo(taskName)); assertThat(tagMap.get(storeType + "-" + StreamsMetricsImpl.STORE_ID_TAG), equalTo(storeName)); } + @Test + public void shouldGetCacheLevelTagMapForBuiltInMetricsLatestVersion() { + shouldGetCacheLevelTagMap(StreamsConfig.METRICS_LATEST); + } + + @Test + public void shouldGetCacheLevelTagMapForBuiltInMetricsVersion0100To23() { + shouldGetCacheLevelTagMap(StreamsConfig.METRICS_0100_TO_23); + } + + private void shouldGetCacheLevelTagMap(final String builtInMetricsVersion) { + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, THREAD_NAME, builtInMetricsVersion); + final String taskName = "taskName"; + final String storeName = "storeName"; + + final Map tagMap = streamsMetrics.cacheLevelTagMap(taskName, storeName); + + assertThat(tagMap.size(), equalTo(3)); + assertThat( + tagMap.get( + builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? StreamsMetricsImpl.THREAD_ID_TAG + : StreamsMetricsImpl.THREAD_ID_TAG_0100_TO_23), + equalTo(Thread.currentThread().getName()) + ); + assertThat(tagMap.get(StreamsMetricsImpl.TASK_ID_TAG), equalTo(taskName)); + assertThat(tagMap.get(StreamsMetricsImpl.RECORD_CACHE_ID_TAG), equalTo(storeName)); + } + @Test public void shouldAddAmountRateAndSum() { StreamsMetricsImpl @@ -327,6 +355,36 @@ public void shouldAddAvgAndTotalMetricsToSensor() { assertThat(metrics.metrics().size(), equalTo(2 + 1)); // one metric is added automatically in the constructor of Metrics } + @Test + public void shouldAddAvgAndMinAndMaxMetricsToSensor() { + StreamsMetricsImpl + .addAvgAndMinAndMaxToSensor(sensor, group, tags, metricNamePrefix, description1, description2, description3); + + final double valueToRecord1 = 18.0; + final double valueToRecord2 = 42.0; + final double expectedAvgMetricValue = (valueToRecord1 + valueToRecord2) / 2; + verifyMetric(metricNamePrefix + "-avg", description1, valueToRecord1, valueToRecord2, expectedAvgMetricValue); + verifyMetric(metricNamePrefix + "-min", description2, valueToRecord1, valueToRecord2, valueToRecord1); + verifyMetric(metricNamePrefix + "-max", description3, valueToRecord1, valueToRecord2, valueToRecord2); + assertThat(metrics.metrics().size(), equalTo(3 + 1)); // one metric is added automatically in the constructor of Metrics + } + + @Test + public void shouldReturnMetricsVersionCurrent() { + assertThat( + new StreamsMetricsImpl(metrics, THREAD_NAME, StreamsConfig.METRICS_LATEST).version(), + equalTo(Version.LATEST) + ); + } + + @Test + public void shouldReturnMetricsVersionFrom100To23() { + assertThat( + new StreamsMetricsImpl(metrics, THREAD_NAME, StreamsConfig.METRICS_0100_TO_23).version(), + equalTo(Version.FROM_100_TO_23) + ); + } + private void verifyMetric(final String name, final String description, final double valueToRecord1, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java index 9c76e14d253b..1ab8684e9052 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; @@ -90,7 +91,9 @@ public void before() { final ProcessorContextImpl mockContext = mock(ProcessorContextImpl.class); expect(mockContext.applicationId()).andReturn("appId").anyTimes(); - expect(mockContext.metrics()).andReturn(new StreamsMetricsImpl(new Metrics(), "threadName")).anyTimes(); + expect(mockContext.metrics()) + .andReturn(new StreamsMetricsImpl(new Metrics(), "threadName", StreamsConfig.METRICS_LATEST)) + .anyTimes(); expect(mockContext.taskId()).andReturn(new TaskId(0, 0)).anyTimes(); expect(mockContext.recordCollector()).andReturn(null).anyTimes(); replay(mockContext); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java index e3378faf885f..53096d2bf13c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java @@ -59,7 +59,8 @@ public class MeteredTimestampedWindowStoreTest { @Before public void setUp() { - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test"); + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, "test", StreamsConfig.METRICS_LATEST); context = new InternalMockProcessorContext( TestUtils.tempDirectory(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index bd5bce1632f1..e62ab3a8a424 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -75,7 +75,8 @@ public class MeteredWindowStoreTest { @Before public void setUp() { - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test"); + final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, "test", StreamsConfig.METRICS_LATEST); context = new InternalMockProcessorContext( TestUtils.tempDirectory(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/Murmur3Test.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/Murmur3Test.java new file mode 100644 index 000000000000..d0759bfba14c --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/Murmur3Test.java @@ -0,0 +1,70 @@ +/* + * 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.streams.state.internals; + +import static org.junit.Assert.*; + +import org.junit.Test; + +import java.util.Map; + +/** + * This class was taken from Hive org.apache.hive.common.util; + * https://github.com/apache/hive/blob/master/storage-api/src/test/org/apache/hive/common/util/TestMurmur3.java + * Commit: dffa3a16588bc8e95b9d0ab5af295a74e06ef702 + * + * + * Tests for Murmur3 variants. + */ +public class Murmur3Test { + + @Test + public void testMurmur3_32() { + Map cases = new java.util.HashMap<>(); + cases.put("21".getBytes(), 896581614); + cases.put("foobar".getBytes(), -328928243); + cases.put("a-little-bit-long-string".getBytes(), -1479816207); + cases.put("a-little-bit-longer-string".getBytes(), -153232333); + cases.put("lkjh234lh9fiuh90y23oiuhsafujhadof229phr9h19h89h8".getBytes(), 13417721); + cases.put(new byte[]{'a', 'b', 'c'}, 461137560); + + int seed = 123; + for (Map.Entry c : cases.entrySet()) { + byte[] b = (byte[]) c.getKey(); + assertEquals(c.getValue(), Murmur3.hash32(b, b.length, seed)); + } + } + + @Test + public void testMurmur3_128() { + Map cases = new java.util.HashMap<>(); + cases.put("21".getBytes(), new long[]{5857341059704281894L, -5288187638297930763L}); + cases.put("foobar".getBytes(), new long[]{-351361463397418609L, 8959716011862540668L}); + cases.put("a-little-bit-long-string".getBytes(), new long[]{8836256500583638442L, -198172363548498523L}); + cases.put("a-little-bit-longer-string".getBytes(), new long[]{1838346159335108511L, 8794688210320490705L}); + cases.put("lkjh234lh9fiuh90y23oiuhsafujhadof229phr9h19h89h8".getBytes(), new long[]{-4024021876037397259L, -1482317706335141238L}); + cases.put(new byte[]{'a', 'b', 'c'}, new long[]{1489494923063836066L, -5440978547625122829L}); + + int seed = 123; + + for (Map.Entry c : cases.entrySet()) { + byte[] b = (byte[]) c.getKey(); + long[] result = Murmur3.hash128(b, 0, b.length, seed); + assertArrayEquals((long[]) c.getValue(), result); + } + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java index 6c82209c33ac..ea52b4e82d86 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; @@ -29,20 +28,15 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; -import static org.apache.kafka.test.StreamsTestUtils.getMetricByNameFilterByTags; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; public class NamedCacheTest { @@ -61,7 +55,7 @@ public void setUp() { } @Test - public void shouldKeepTrackOfMostRecentlyAndLeastRecentlyUsed() throws IOException { + public void shouldKeepTrackOfMostRecentlyAndLeastRecentlyUsed() { final List> toInsert = Arrays.asList( new KeyValue<>("K1", "V1"), new KeyValue<>("K2", "V2"), @@ -83,31 +77,6 @@ public void shouldKeepTrackOfMostRecentlyAndLeastRecentlyUsed() throws IOExcepti } } - @Test - public void testMetrics() { - final Map metricTags = new LinkedHashMap<>(); - metricTags.put("record-cache-id", underlyingStoreName); - metricTags.put("task-id", taskIDString); - metricTags.put("client-id", "test"); - - getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-avg", "stream-record-cache-metrics", metricTags); - getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-min", "stream-record-cache-metrics", metricTags); - getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-max", "stream-record-cache-metrics", metricTags); - - // test "all" - metricTags.put("record-cache-id", "all"); - getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-avg", "stream-record-cache-metrics", metricTags); - getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-min", "stream-record-cache-metrics", metricTags); - getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-max", "stream-record-cache-metrics", metricTags); - - final JmxReporter reporter = new JmxReporter("kafka.streams"); - innerMetrics.addReporter(reporter); - assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-record-cache-metrics,client-id=test,task-id=%s,record-cache-id=%s", - taskIDString, underlyingStoreName))); - assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-record-cache-metrics,client-id=test,task-id=%s,record-cache-id=%s", - taskIDString, "all"))); - } - @Test public void shouldKeepTrackOfSize() { final LRUCacheEntry value = new LRUCacheEntry(new byte[]{0}); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java index 70c44a5d5588..e3b7274df1d4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.Sensor.RecordingLevel; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; @@ -26,19 +28,21 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.ProcessorStateException; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreListener; -import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.RocksDBConfigSetter; +import org.apache.kafka.streams.state.internals.metrics.RocksDBMetrics; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder; +import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecordingTrigger; import org.apache.kafka.test.InternalMockProcessorContext; import org.apache.kafka.test.StreamsTestUtils; import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.BloomFilter; import org.rocksdb.Filter; @@ -57,15 +61,10 @@ import java.util.Set; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.streams.StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG; -import static org.apache.kafka.streams.StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG; -import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.mock; -import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.reset; -import static org.easymock.EasyMock.verify; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -73,7 +72,11 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.powermock.api.easymock.PowerMock.replay; +import static org.powermock.api.easymock.PowerMock.verify; +@RunWith(PowerMockRunner.class) +@PrepareForTest({RocksDBMetrics.class, Sensor.class}) public class RocksDBStoreTest { private static boolean enableBloomFilters = false; final static String DB_NAME = "db-name"; @@ -83,6 +86,8 @@ public class RocksDBStoreTest { private final Serializer stringSerializer = new StringSerializer(); private final Deserializer stringDeserializer = new StringDeserializer(); + private final RocksDBMetricsRecorder metricsRecorder = mock(RocksDBMetricsRecorder.class); + InternalMockProcessorContext context; RocksDBStore rocksDBStore; @@ -96,29 +101,79 @@ public void setUp() { Serdes.String(), Serdes.String(), new StreamsConfig(props)); + context.metrics().setRocksDBMetricsRecordingTrigger(new RocksDBMetricsRecordingTrigger()); } RocksDBStore getRocksDBStore() { return new RocksDBStore(DB_NAME, METRICS_SCOPE); } + private RocksDBStore getRocksDBStoreWithRocksDBMetricsRecorder() { + return new RocksDBStore(DB_NAME, METRICS_SCOPE, metricsRecorder); + } + + private InternalMockProcessorContext getProcessorContext(final Properties streamsProps) { + return new InternalMockProcessorContext( + TestUtils.tempDirectory(), + new StreamsConfig(streamsProps) + ); + } + + private InternalMockProcessorContext getProcessorContext( + final RecordingLevel recordingLevel, + final Class rocksDBConfigSetterClass) { + + final Properties streamsProps = StreamsTestUtils.getStreamsConfig(); + streamsProps.setProperty(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, recordingLevel.name()); + streamsProps.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, rocksDBConfigSetterClass); + return getProcessorContext(streamsProps); + } + + private InternalMockProcessorContext getProcessorContext(final RecordingLevel recordingLevel) { + final Properties streamsProps = StreamsTestUtils.getStreamsConfig(); + streamsProps.setProperty(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, recordingLevel.name()); + return getProcessorContext(streamsProps); + } + @After public void tearDown() { rocksDBStore.close(); } @Test - public void shouldRemoveStatisticsFromInjectedMetricsRecorderOnClose() { - final RocksDBMetricsRecorder metricsRecorder = mock(RocksDBMetricsRecorder.class); - final RocksDBStore store = new RocksDBStore(DB_NAME, METRICS_SCOPE, metricsRecorder); - final ProcessorContext mockContext = mock(ProcessorContext.class); - expect(mockContext.appConfigs()).andReturn(mkMap(mkEntry(METRICS_RECORDING_LEVEL_CONFIG, "DEBUG"))); - final String directoryPath = TestUtils.tempDirectory().getAbsolutePath(); - final File directory = new File(directoryPath); - expect(mockContext.stateDir()).andReturn(directory); - expect(mockContext.metrics()).andReturn(mock(StreamsMetricsImpl.class)); - expect(mockContext.taskId()).andReturn(new TaskId(0, 0)); - replay(mockContext); + public void shouldAddStatisticsToInjectedMetricsRecorderWhenRecordingLevelIsDebug() { + final RocksDBStore store = getRocksDBStoreWithRocksDBMetricsRecorder(); + final InternalMockProcessorContext mockContext = getProcessorContext(RecordingLevel.DEBUG); + reset(metricsRecorder); + metricsRecorder.addStatistics( + eq(DB_NAME), + anyObject(Statistics.class), + eq(mockContext.metrics()), + eq(mockContext.taskId()) + ); + replay(metricsRecorder); + + store.openDB(mockContext); + + verify(metricsRecorder); + } + + @Test + public void shouldNotAddStatisticsToInjectedMetricsRecorderWhenRecordingLevelIsInfo() { + final RocksDBStore store = getRocksDBStoreWithRocksDBMetricsRecorder(); + final InternalMockProcessorContext mockContext = getProcessorContext(RecordingLevel.INFO); + reset(metricsRecorder); + replay(metricsRecorder); + + store.openDB(mockContext); + + verify(metricsRecorder); + } + + @Test + public void shouldRemoveStatisticsFromInjectedMetricsRecorderOnCloseWhenRecordingLevelIsDebug() { + final RocksDBStore store = getRocksDBStoreWithRocksDBMetricsRecorder(); + final InternalMockProcessorContext mockContext = getProcessorContext(RecordingLevel.DEBUG); store.openDB(mockContext); reset(metricsRecorder); metricsRecorder.removeStatistics(DB_NAME); @@ -131,24 +186,19 @@ public void shouldRemoveStatisticsFromInjectedMetricsRecorderOnClose() { @Test public void shouldNotRemoveStatisticsFromInjectedMetricsRecorderOnCloseWhenRecordingLevelIsInfo() { - final RocksDBMetricsRecorder metricsRecorder = mock(RocksDBMetricsRecorder.class); - replay(metricsRecorder); - final RocksDBStore store = new RocksDBStore(DB_NAME, METRICS_SCOPE, metricsRecorder); - final ProcessorContext mockContext = mock(ProcessorContext.class); - expect(mockContext.appConfigs()).andReturn(mkMap(mkEntry(METRICS_RECORDING_LEVEL_CONFIG, "INFO"))); - final String directoryPath = TestUtils.tempDirectory().getAbsolutePath(); - final File directory = new File(directoryPath); - expect(mockContext.stateDir()).andReturn(directory); - replay(mockContext); + final RocksDBStore store = getRocksDBStoreWithRocksDBMetricsRecorder(); + final InternalMockProcessorContext mockContext = getProcessorContext(RecordingLevel.INFO); store.openDB(mockContext); + reset(metricsRecorder); + replay(metricsRecorder); store.close(); verify(metricsRecorder); } - public static class TestRocksDBConfigSetter implements RocksDBConfigSetter { - public TestRocksDBConfigSetter(){} + public static class RocksDBConfigSetterWithUserProvidedStatistics implements RocksDBConfigSetter { + public RocksDBConfigSetterWithUserProvidedStatistics(){} public void setConfig(final String storeName, final Options options, final Map configs) { options.setStatistics(new Statistics()); @@ -160,19 +210,24 @@ public void close(final String storeName, final Options options) { } @Test - public void shouldNotRemoveStatisticsFromInjectedMetricsRecorderOnCloseWhenUserProvidsStatistics() { - final RocksDBMetricsRecorder metricsRecorder = mock(RocksDBMetricsRecorder.class); + public void shouldNotAddStatisticsToInjectedMetricsRecorderWhenUserProvidesStatistics() { + final RocksDBStore store = getRocksDBStoreWithRocksDBMetricsRecorder(); + final InternalMockProcessorContext mockContext = + getProcessorContext(RecordingLevel.DEBUG, RocksDBConfigSetterWithUserProvidedStatistics.class); replay(metricsRecorder); - final RocksDBStore store = new RocksDBStore(DB_NAME, METRICS_SCOPE, metricsRecorder); - final ProcessorContext mockContext = mock(ProcessorContext.class); - expect(mockContext.appConfigs()).andReturn(mkMap( - mkEntry(METRICS_RECORDING_LEVEL_CONFIG, "DEBUG"), - mkEntry(ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, TestRocksDBConfigSetter.class))); - final String directoryPath = TestUtils.tempDirectory().getAbsolutePath(); - final File directory = new File(directoryPath); - expect(mockContext.stateDir()).andReturn(directory); - replay(mockContext); + store.openDB(mockContext); + verify(metricsRecorder); + } + + @Test + public void shouldNotRemoveStatisticsFromInjectedMetricsRecorderOnCloseWhenUserProvidesStatistics() { + final RocksDBStore store = getRocksDBStoreWithRocksDBMetricsRecorder(); + final InternalMockProcessorContext mockContext = + getProcessorContext(RecordingLevel.DEBUG, RocksDBConfigSetterWithUserProvidedStatistics.class); + store.openDB(mockContext); + reset(metricsRecorder); + replay(metricsRecorder); store.close(); @@ -526,6 +581,7 @@ public void shouldHandleToggleOfEnablingBloomFilters() { Serdes.String(), Serdes.String(), new StreamsConfig(props)); + context.metrics().setRocksDBMetricsRecordingTrigger(new RocksDBMetricsRecordingTrigger()); enableBloomFilters = false; rocksDBStore.init(context, rocksDBStore); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index 85ec3379b2a9..2e305dd406b2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -41,10 +41,11 @@ public class SegmentIteratorTest { + private final RocksDBMetricsRecorder rocksDBMetricsRecorder = new RocksDBMetricsRecorder("metrics-scope", "store-name"); private final KeyValueSegment segmentOne = - new KeyValueSegment("one", "one", 0, new RocksDBMetricsRecorder("metrics-scope", "store-name")); + new KeyValueSegment("one", "one", 0, rocksDBMetricsRecorder); private final KeyValueSegment segmentTwo = - new KeyValueSegment("two", "window", 1, new RocksDBMetricsRecorder("metrics-scope", "store-name")); + new KeyValueSegment("two", "window", 1, rocksDBMetricsRecorder); private final HasNextCondition hasNextCondition = Iterator::hasNext; private SegmentIterator iterator = null; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java index f48c31c1d91c..a2e4557c27ea 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java @@ -317,10 +317,7 @@ private StreamTask createStreamsTask(final StreamsConfig streamsConfig, stateDirectory, null, new MockTime(), - () -> clientSupplier.getProducer(new HashMap<>())) { - @Override - protected void updateOffsetLimits() {} - }; + () -> clientSupplier.getProducer(new HashMap<>())); } private void mockThread(final boolean initialized) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetricsTest.java new file mode 100644 index 000000000000..46ea302738be --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetricsTest.java @@ -0,0 +1,117 @@ +/* + * 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.streams.state.internals.metrics; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.Sensor.RecordingLevel; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Map; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.mock; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.powermock.api.easymock.PowerMock.createMock; +import static org.powermock.api.easymock.PowerMock.mockStatic; +import static org.powermock.api.easymock.PowerMock.replay; +import static org.powermock.api.easymock.PowerMock.verify; + + +@RunWith(PowerMockRunner.class) +@PrepareForTest({StreamsMetricsImpl.class, Sensor.class}) +public class NamedCacheMetricsTest { + + private static final String TASK_NAME = "taskName"; + private static final String STORE_NAME = "storeName"; + private static final String HIT_RATIO_AVG_DESCRIPTION = "The average cache hit ratio"; + private static final String HIT_RATIO_MIN_DESCRIPTION = "The minimum cache hit ratio"; + private static final String HIT_RATIO_MAX_DESCRIPTION = "The maximum cache hit ratio"; + + private final StreamsMetricsImpl streamsMetrics = createMock(StreamsMetricsImpl.class); + private final Sensor expectedSensor = mock(Sensor.class); + private final Map tagMap = mkMap(mkEntry("key", "value")); + + @Test + public void shouldGetHitRatioSensorWithBuiltInMetricsVersionCurrent() { + final String hitRatio = "hit-ratio"; + mockStatic(StreamsMetricsImpl.class); + setUpStreamsMetrics(Version.LATEST, hitRatio); + replay(streamsMetrics); + replay(StreamsMetricsImpl.class); + + final Sensor sensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, TASK_NAME, STORE_NAME); + + verifyResult(sensor); + } + + @Test + public void shouldGetHitRatioSensorWithBuiltInMetricsVersionBefore24() { + final Map parentTagMap = mkMap(mkEntry("key", "all")); + final String hitRatio = "hitRatio"; + final RecordingLevel recordingLevel = RecordingLevel.DEBUG; + mockStatic(StreamsMetricsImpl.class); + final Sensor parentSensor = mock(Sensor.class); + expect(streamsMetrics.taskLevelSensor(TASK_NAME, hitRatio, recordingLevel)).andReturn(parentSensor); + expect(streamsMetrics.cacheLevelTagMap(TASK_NAME, StreamsMetricsImpl.ROLLUP_VALUE)).andReturn(parentTagMap); + StreamsMetricsImpl.addAvgAndMinAndMaxToSensor( + parentSensor, + StreamsMetricsImpl.CACHE_LEVEL_GROUP, + parentTagMap, + hitRatio, + HIT_RATIO_AVG_DESCRIPTION, + HIT_RATIO_MIN_DESCRIPTION, + HIT_RATIO_MAX_DESCRIPTION); + setUpStreamsMetrics(Version.FROM_100_TO_23, hitRatio, parentSensor); + replay(streamsMetrics); + replay(StreamsMetricsImpl.class); + + final Sensor sensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, TASK_NAME, STORE_NAME); + + verifyResult(sensor); + } + + private void setUpStreamsMetrics(final Version builtInMetricsVersion, + final String hitRatio, + final Sensor... parents) { + expect(streamsMetrics.version()).andReturn(builtInMetricsVersion); + expect(streamsMetrics.cacheLevelSensor(TASK_NAME, STORE_NAME, hitRatio, RecordingLevel.DEBUG, parents)) + .andReturn(expectedSensor); + expect(streamsMetrics.cacheLevelTagMap(TASK_NAME, STORE_NAME)).andReturn(tagMap); + StreamsMetricsImpl.addAvgAndMinAndMaxToSensor( + expectedSensor, + StreamsMetricsImpl.CACHE_LEVEL_GROUP, + tagMap, + hitRatio, + HIT_RATIO_AVG_DESCRIPTION, + HIT_RATIO_MIN_DESCRIPTION, + HIT_RATIO_MAX_DESCRIPTION); + } + + private void verifyResult(final Sensor sensor) { + verify(streamsMetrics); + verify(StreamsMetricsImpl.class); + assertThat(sensor, is(expectedSensor)); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java index a5bd1da8db29..9bef2cc3b6f6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java @@ -20,16 +20,22 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetrics.RocksDBMetricContext; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; import org.rocksdb.Statistics; import org.rocksdb.StatsLevel; +import org.rocksdb.TickerType; +import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.mock; +import static org.easymock.EasyMock.niceMock; +import static org.easymock.EasyMock.resetToNice; +import static org.junit.Assert.assertThrows; import static org.powermock.api.easymock.PowerMock.reset; import static org.powermock.api.easymock.PowerMock.createMock; import static org.powermock.api.easymock.PowerMock.mockStatic; @@ -40,105 +46,312 @@ @RunWith(PowerMockRunner.class) @PrepareForTest({RocksDBMetrics.class, Sensor.class}) public class RocksDBMetricsRecorderTest { - private final static String METRICS_SCOPE = "metrics-scope"; - private final static String STORE_NAME = "store name"; - private final static String SEGMENT_STORE_NAME_1 = "segment store name 1"; - private final static String SEGMENT_STORE_NAME_2 = "segment name 2"; + private final static String STORE_NAME = "store-name"; + private final static String SEGMENT_STORE_NAME_1 = "segment-store-name-1"; + private final static String SEGMENT_STORE_NAME_2 = "segment-store-name-2"; private final Statistics statisticsToAdd1 = mock(Statistics.class); private final Statistics statisticsToAdd2 = mock(Statistics.class); - private final Sensor sensor = createMock(Sensor.class); - private final StreamsMetricsImpl streamsMetrics = mock(StreamsMetricsImpl.class); - private final TaskId taskId = new TaskId(0, 0); + + private final Sensor bytesWrittenToDatabaseSensor = createMock(Sensor.class); + private final Sensor bytesReadFromDatabaseSensor = createMock(Sensor.class); + private final Sensor memtableBytesFlushedSensor = createMock(Sensor.class); + private final Sensor memtableHitRatioSensor = createMock(Sensor.class); + private final Sensor writeStallDurationSensor = createMock(Sensor.class); + private final Sensor blockCacheDataHitRatioSensor = createMock(Sensor.class); + private final Sensor blockCacheIndexHitRatioSensor = createMock(Sensor.class); + private final Sensor blockCacheFilterHitRatioSensor = createMock(Sensor.class); + private final Sensor bytesReadDuringCompactionSensor = createMock(Sensor.class); + private final Sensor bytesWrittenDuringCompactionSensor = createMock(Sensor.class); + private final Sensor numberOfOpenFilesSensor = createMock(Sensor.class); + private final Sensor numberOfFileErrorsSensor = createMock(Sensor.class); + + private final StreamsMetricsImpl streamsMetrics = niceMock(StreamsMetricsImpl.class); + private final RocksDBMetricsRecordingTrigger recordingTrigger = mock(RocksDBMetricsRecordingTrigger.class); + private final TaskId taskId1 = new TaskId(0, 0); + private final TaskId taskId2 = new TaskId(0, 2); private final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME); + @Before + public void setUp() { + expect(streamsMetrics.rocksDBMetricsRecordingTrigger()).andStubReturn(recordingTrigger); + replay(streamsMetrics); + } + @Test - public void shouldSetStatsLevelToExceptDetailedTimers() { + public void shouldSetStatsLevelToExceptDetailedTimersWhenStatisticsIsAdded() { mockStaticNice(RocksDBMetrics.class); replay(RocksDBMetrics.class); statisticsToAdd1.setStatsLevel(StatsLevel.EXCEPT_DETAILED_TIMERS); replay(statisticsToAdd1); - recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); verify(statisticsToAdd1); } @Test - public void shouldInitMetricsOnlyWhenFirstStatisticsIsAdded() { - replayMetricsInitialization(); - statisticsToAdd1.setStatsLevel(StatsLevel.EXCEPT_DETAILED_TIMERS); - replay(statisticsToAdd1); - recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId); + public void shouldThrowIfTaskIdOfStatisticsToAddDiffersFromInitialisedOne() { + mockStaticNice(RocksDBMetrics.class); + replay(RocksDBMetrics.class); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + assertThrows( + IllegalStateException.class, + () -> recorder.addStatistics(SEGMENT_STORE_NAME_2, statisticsToAdd2, streamsMetrics, taskId2) + ); + } + + @Test + public void shouldThrowIfStatisticsToAddHasBeenAlreadyAdded() { + mockStaticNice(RocksDBMetrics.class); + replay(RocksDBMetrics.class); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + + assertThrows( + IllegalStateException.class, + () -> recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1) + ); + } + + @Test + public void shouldInitMetricsAndAddItselfToRecordingTriggerOnlyWhenFirstStatisticsIsAdded() { + setUpMetricsMock(); + recordingTrigger.addMetricsRecorder(recorder); + replay(recordingTrigger); + + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + + verify(recordingTrigger); verify(RocksDBMetrics.class); mockStatic(RocksDBMetrics.class); replay(RocksDBMetrics.class); - recorder.addStatistics(SEGMENT_STORE_NAME_2, statisticsToAdd2, streamsMetrics, taskId); + reset(recordingTrigger); + replay(recordingTrigger); + + recorder.addStatistics(SEGMENT_STORE_NAME_2, statisticsToAdd2, streamsMetrics, taskId1); + + verify(recordingTrigger); verify(RocksDBMetrics.class); } @Test - public void shouldCloseStatisticsWhenRecorderIsClosed() { + public void shouldAddItselfToRecordingTriggerWhenEmptyButInitialised() { + mockStaticNice(RocksDBMetrics.class); + replay(RocksDBMetrics.class); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + recorder.removeStatistics(SEGMENT_STORE_NAME_1); + reset(recordingTrigger); + recordingTrigger.addMetricsRecorder(recorder); + replay(recordingTrigger); + + recorder.addStatistics(SEGMENT_STORE_NAME_2, statisticsToAdd2, streamsMetrics, taskId1); + + verify(recordingTrigger); + } + + @Test + public void shouldNotAddItselfToRecordingTriggerWhenNotEmpty() { + mockStaticNice(RocksDBMetrics.class); + replay(RocksDBMetrics.class); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + reset(recordingTrigger); + replay(recordingTrigger); + + recorder.addStatistics(SEGMENT_STORE_NAME_2, statisticsToAdd2, streamsMetrics, taskId1); + + verify(recordingTrigger); + } + + @Test + public void shouldCloseStatisticsWhenStatisticsIsRemoved() { mockStaticNice(RocksDBMetrics.class); replay(RocksDBMetrics.class); - recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId); - recorder.addStatistics(SEGMENT_STORE_NAME_2, statisticsToAdd2, streamsMetrics, taskId); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); reset(statisticsToAdd1); - reset(statisticsToAdd2); statisticsToAdd1.close(); - statisticsToAdd2.close(); replay(statisticsToAdd1); - replay(statisticsToAdd2); - recorder.close(); + recorder.removeStatistics(SEGMENT_STORE_NAME_1); verify(statisticsToAdd1); - verify(statisticsToAdd2); } @Test - public void shouldCloseStatisticsWhenStatisticsIsRemoved() { + public void shouldRemoveItselfFromRecordingTriggerWhenLastStatisticsIsRemoved() { + mockStaticNice(RocksDBMetrics.class); + replay(RocksDBMetrics.class); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + recorder.addStatistics(SEGMENT_STORE_NAME_2, statisticsToAdd2, streamsMetrics, taskId1); + reset(recordingTrigger); + replay(recordingTrigger); + + recorder.removeStatistics(SEGMENT_STORE_NAME_1); + + verify(recordingTrigger); + + reset(recordingTrigger); + recordingTrigger.removeMetricsRecorder(recorder); + replay(recordingTrigger); + + recorder.removeStatistics(SEGMENT_STORE_NAME_2); + + verify(recordingTrigger); + } + + @Test + public void shouldThrowIfStatisticsToRemoveNotFound() { mockStaticNice(RocksDBMetrics.class); replay(RocksDBMetrics.class); - recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + assertThrows( + IllegalStateException.class, + () -> recorder.removeStatistics(SEGMENT_STORE_NAME_2) + ); + } + + @Test + public void shouldRecordMetrics() { + setUpMetricsMock(); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + recorder.addStatistics(SEGMENT_STORE_NAME_2, statisticsToAdd2, streamsMetrics, taskId1); reset(statisticsToAdd1); - statisticsToAdd1.close(); + reset(statisticsToAdd2); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BYTES_WRITTEN)).andReturn(1L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BYTES_WRITTEN)).andReturn(2L); + bytesWrittenToDatabaseSensor.record(1 + 2); + replay(bytesWrittenToDatabaseSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BYTES_READ)).andReturn(2L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BYTES_READ)).andReturn(3L); + bytesReadFromDatabaseSensor.record(2 + 3); + replay(bytesReadFromDatabaseSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.FLUSH_WRITE_BYTES)).andReturn(3L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.FLUSH_WRITE_BYTES)).andReturn(4L); + memtableBytesFlushedSensor.record(3 + 4); + replay(memtableBytesFlushedSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.MEMTABLE_HIT)).andReturn(1L); + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.MEMTABLE_MISS)).andReturn(2L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.MEMTABLE_HIT)).andReturn(3L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.MEMTABLE_MISS)).andReturn(4L); + memtableHitRatioSensor.record((double) 4 / (4 + 6)); + replay(memtableHitRatioSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.STALL_MICROS)).andReturn(4L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.STALL_MICROS)).andReturn(5L); + writeStallDurationSensor.record(4 + 5); + replay(writeStallDurationSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_HIT)).andReturn(5L); + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_MISS)).andReturn(4L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_HIT)).andReturn(3L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_MISS)).andReturn(2L); + blockCacheDataHitRatioSensor.record((double) 8 / (8 + 6)); + replay(blockCacheDataHitRatioSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_HIT)).andReturn(4L); + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_MISS)).andReturn(2L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_HIT)).andReturn(2L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_MISS)).andReturn(4L); + blockCacheIndexHitRatioSensor.record((double) 6 / (6 + 6)); + replay(blockCacheIndexHitRatioSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_HIT)).andReturn(2L); + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_MISS)).andReturn(4L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_HIT)).andReturn(3L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_MISS)).andReturn(5L); + blockCacheFilterHitRatioSensor.record((double) 5 / (5 + 9)); + replay(blockCacheFilterHitRatioSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.COMPACT_WRITE_BYTES)).andReturn(2L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.COMPACT_WRITE_BYTES)).andReturn(4L); + bytesWrittenDuringCompactionSensor.record(2 + 4); + replay(bytesWrittenDuringCompactionSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.COMPACT_READ_BYTES)).andReturn(5L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.COMPACT_READ_BYTES)).andReturn(6L); + bytesReadDuringCompactionSensor.record(5 + 6); + replay(bytesReadDuringCompactionSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_OPENS)).andReturn(5L); + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_CLOSES)).andReturn(3L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_OPENS)).andReturn(7L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_CLOSES)).andReturn(4L); + numberOfOpenFilesSensor.record((5 + 7) - (3 + 4)); + replay(numberOfOpenFilesSensor); + + expect(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_ERRORS)).andReturn(34L); + expect(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_ERRORS)).andReturn(11L); + numberOfFileErrorsSensor.record(11 + 34); + replay(numberOfFileErrorsSensor); + replay(statisticsToAdd1); + replay(statisticsToAdd2); - recorder.removeStatistics(SEGMENT_STORE_NAME_1); + recorder.record(); verify(statisticsToAdd1); + verify(statisticsToAdd2); + verify(bytesWrittenToDatabaseSensor); + } + + @Test + public void shouldCorrectlyHandleHitRatioRecordingsWithZeroHitsAndMisses() { + setUpMetricsMock(); + recorder.addStatistics(SEGMENT_STORE_NAME_1, statisticsToAdd1, streamsMetrics, taskId1); + resetToNice(statisticsToAdd1); + expect(statisticsToAdd1.getTickerCount(anyObject())).andReturn(0L).anyTimes(); + replay(statisticsToAdd1); + memtableHitRatioSensor.record(0); + blockCacheDataHitRatioSensor.record(0); + blockCacheIndexHitRatioSensor.record(0); + blockCacheFilterHitRatioSensor.record(0); + replay(memtableHitRatioSensor); + replay(blockCacheDataHitRatioSensor); + replay(blockCacheIndexHitRatioSensor); + replay(blockCacheFilterHitRatioSensor); + + recorder.record(); + + verify(memtableHitRatioSensor); + verify(blockCacheDataHitRatioSensor); + verify(blockCacheIndexHitRatioSensor); + verify(blockCacheFilterHitRatioSensor); } - private void replayMetricsInitialization() { + private void setUpMetricsMock() { mockStatic(RocksDBMetrics.class); final RocksDBMetricContext metricsContext = - new RocksDBMetricContext(taskId.toString(), METRICS_SCOPE, STORE_NAME); - expect(RocksDBMetrics.bytesWrittenToDatabaseSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.bytesReadFromDatabaseSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.memtableBytesFlushedSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.memtableHitRatioSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.memtableAvgFlushTimeSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.memtableMinFlushTimeSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.memtableMaxFlushTimeSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.writeStallDurationSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.blockCacheDataHitRatioSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.blockCacheIndexHitRatioSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.blockCacheFilterHitRatioSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect( - RocksDBMetrics.bytesReadDuringCompactionSensor(eq(streamsMetrics), eq(metricsContext)) - ).andReturn(sensor); - expect( - RocksDBMetrics.bytesWrittenDuringCompactionSensor(eq(streamsMetrics), eq(metricsContext)) - ).andReturn(sensor); - expect(RocksDBMetrics.compactionTimeMinSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.compactionTimeMaxSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.compactionTimeAvgSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.numberOfOpenFilesSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); - expect(RocksDBMetrics.numberOfFileErrorsSensor(eq(streamsMetrics), eq(metricsContext))).andReturn(sensor); + new RocksDBMetricContext(taskId1.toString(), METRICS_SCOPE, STORE_NAME); + expect(RocksDBMetrics.bytesWrittenToDatabaseSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(bytesWrittenToDatabaseSensor); + expect(RocksDBMetrics.bytesReadFromDatabaseSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(bytesReadFromDatabaseSensor); + expect(RocksDBMetrics.memtableBytesFlushedSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(memtableBytesFlushedSensor); + expect(RocksDBMetrics.memtableHitRatioSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(memtableHitRatioSensor); + expect(RocksDBMetrics.writeStallDurationSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(writeStallDurationSensor); + expect(RocksDBMetrics.blockCacheDataHitRatioSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(blockCacheDataHitRatioSensor); + expect(RocksDBMetrics.blockCacheIndexHitRatioSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(blockCacheIndexHitRatioSensor); + expect(RocksDBMetrics.blockCacheFilterHitRatioSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(blockCacheFilterHitRatioSensor); + expect(RocksDBMetrics.bytesWrittenDuringCompactionSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(bytesWrittenDuringCompactionSensor); + expect(RocksDBMetrics.bytesReadDuringCompactionSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(bytesReadDuringCompactionSensor); + expect(RocksDBMetrics.numberOfOpenFilesSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(numberOfOpenFilesSensor); + expect(RocksDBMetrics.numberOfFileErrorsSensor(eq(streamsMetrics), eq(metricsContext))) + .andReturn(numberOfFileErrorsSensor); replay(RocksDBMetrics.class); } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTriggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTriggerTest.java new file mode 100644 index 000000000000..c341055f3976 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTriggerTest.java @@ -0,0 +1,87 @@ +/* + * 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.streams.state.internals.metrics; + +import org.apache.kafka.streams.processor.TaskId; +import org.junit.Before; +import org.junit.Test; + +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.niceMock; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.resetToDefault; +import static org.easymock.EasyMock.verify; +import static org.junit.Assert.assertThrows; + +public class RocksDBMetricsRecordingTriggerTest { + + private final static String STORE_NAME1 = "store-name1"; + private final static String STORE_NAME2 = "store-name2"; + private final static TaskId TASK_ID1 = new TaskId(1, 2); + private final static TaskId TASK_ID2 = new TaskId(2, 4); + private final RocksDBMetricsRecorder recorder1 = niceMock(RocksDBMetricsRecorder.class); + private final RocksDBMetricsRecorder recorder2 = niceMock(RocksDBMetricsRecorder.class); + + private final RocksDBMetricsRecordingTrigger recordingTrigger = new RocksDBMetricsRecordingTrigger(); + + @Before + public void setUp() { + expect(recorder1.storeName()).andStubReturn(STORE_NAME1); + expect(recorder1.taskId()).andStubReturn(TASK_ID1); + replay(recorder1); + expect(recorder2.storeName()).andStubReturn(STORE_NAME2); + expect(recorder2.taskId()).andStubReturn(TASK_ID2); + replay(recorder2); + } + + @Test + public void shouldTriggerAddedMetricsRecorders() { + recordingTrigger.addMetricsRecorder(recorder1); + recordingTrigger.addMetricsRecorder(recorder2); + + resetToDefault(recorder1); + recorder1.record(); + replay(recorder1); + resetToDefault(recorder2); + recorder2.record(); + replay(recorder2); + + recordingTrigger.run(); + + verify(recorder1); + verify(recorder2); + } + + @Test + public void shouldThrowIfRecorderToAddHasBeenAlreadyAdded() { + recordingTrigger.addMetricsRecorder(recorder1); + + assertThrows( + IllegalStateException.class, + () -> recordingTrigger.addMetricsRecorder(recorder1) + ); + } + + @Test + public void shouldThrowIfRecorderToRemoveCouldNotBeFound() { + recordingTrigger.addMetricsRecorder(recorder1); + assertThrows( + IllegalStateException.class, + () -> recordingTrigger.removeMetricsRecorder(recorder2) + ); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsTest.java index 52632d5360bd..aae7afeeb7e2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsTest.java @@ -204,17 +204,14 @@ public void shouldGetCompactionTimeMaxSensor() { public void shouldGetNumberOfOpenFilesSensor() { final String metricNamePrefix = "number-open-files"; final String description = "Number of currently open files"; - verifyValueSensor(metricNamePrefix, description, RocksDBMetrics::numberOfOpenFilesSensor); + verifySumSensor(metricNamePrefix, false, description, RocksDBMetrics::numberOfOpenFilesSensor); } @Test public void shouldGetNumberOfFilesErrors() { final String metricNamePrefix = "number-file-errors"; final String description = "Total number of file errors occurred"; - setupStreamsMetricsMock(metricNamePrefix); - StreamsMetricsImpl.addSumMetricToSensor(sensor, STATE_LEVEL_GROUP, tags, metricNamePrefix, description); - - replayCallAndVerify(RocksDBMetrics::numberOfFileErrorsSensor); + verifySumSensor(metricNamePrefix, true, description, RocksDBMetrics::numberOfFileErrorsSensor); } private void verifyRateAndTotalSensor(final String metricNamePrefix, @@ -252,6 +249,21 @@ private void verifyValueSensor(final String metricNamePrefix, replayCallAndVerify(sensorCreator); } + private void verifySumSensor(final String metricNamePrefix, + final boolean withSuffix, + final String description, + final SensorCreator sensorCreator) { + setupStreamsMetricsMock(metricNamePrefix); + if (withSuffix) { + StreamsMetricsImpl.addSumMetricToSensor(sensor, STATE_LEVEL_GROUP, tags, metricNamePrefix, description); + } else { + StreamsMetricsImpl + .addSumMetricToSensor(sensor, STATE_LEVEL_GROUP, tags, metricNamePrefix, withSuffix, description); + } + + replayCallAndVerify(sensorCreator); + } + private void setupStreamsMetricsMock(final String metricNamePrefix) { mockStatic(StreamsMetricsImpl.class); expect(streamsMetrics.storeLevelSensor( diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java index a4735869c515..de5311cef92d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -124,7 +124,7 @@ public ByteBuffer subscriptionUserData(final Set topics) { // 3. Task ids of valid local states on the client's state directory. final TaskManager taskManager = taskManger(); - final Set previousActiveTasks = taskManager.prevActiveTaskIds(); + final Set previousActiveTasks = taskManager.previousRunningTaskIds(); final Set standbyTasks = taskManager.cachedTasksIds(); standbyTasks.removeAll(previousActiveTasks); final FutureSubscriptionInfo data = new FutureSubscriptionInfo( @@ -176,12 +176,15 @@ public void onAssignment(final ConsumerPartitionAssignor.Assignment assignment, final Map topicToPartitionInfo = new HashMap<>(); final Map> partitionsByHost; - processVersionTwoAssignment("test ", info, partitions, activeTasks, topicToPartitionInfo); + final Map partitionsToTaskId = new HashMap<>(); + + processVersionTwoAssignment("test ", info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId); partitionsByHost = info.partitionsByHost(); final TaskManager taskManager = taskManger(); taskManager.setClusterMetadata(Cluster.empty().withPartitions(topicToPartitionInfo)); taskManager.setPartitionsByHostState(partitionsByHost); + taskManager.setPartitionsToTaskId(partitionsToTaskId); taskManager.setAssignmentMetadata(activeTasks, info.standbyTasks()); taskManager.updateSubscriptionsFromAssignment(partitions); } diff --git a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java index 4b9267959ac8..f5cb014550a9 100644 --- a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java @@ -41,6 +41,7 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.internals.ThreadCache; +import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecordingTrigger; import java.io.File; import java.time.Duration; @@ -69,7 +70,7 @@ public InternalMockProcessorContext() { this(null, null, null, - new StreamsMetricsImpl(new Metrics(), "mock"), + new StreamsMetricsImpl(new Metrics(), "mock", StreamsConfig.METRICS_LATEST), new StreamsConfig(StreamsTestUtils.getStreamsConfig()), null, null @@ -78,14 +79,30 @@ public InternalMockProcessorContext() { public InternalMockProcessorContext(final File stateDir, final StreamsConfig config) { - this(stateDir, null, null, new StreamsMetricsImpl(new Metrics(), "mock"), config, null, null); + this( + stateDir, + null, + null, + new StreamsMetricsImpl(new Metrics(), "mock", StreamsConfig.METRICS_LATEST), + config, + null, + null + ); } public InternalMockProcessorContext(final File stateDir, final Serde keySerde, final Serde valSerde, final StreamsConfig config) { - this(stateDir, keySerde, valSerde, new StreamsMetricsImpl(new Metrics(), "mock"), config, null, null); + this( + stateDir, + keySerde, + valSerde, + new StreamsMetricsImpl(new Metrics(), "mock", StreamsConfig.METRICS_LATEST), + config, + null, + null + ); } public InternalMockProcessorContext(final StateSerdes serdes, @@ -100,7 +117,7 @@ public InternalMockProcessorContext(final StateSerdes serdes, null, serdes.keySerde(), serdes.valueSerde(), - new StreamsMetricsImpl(metrics, "mock"), + new StreamsMetricsImpl(metrics, "mock", StreamsConfig.METRICS_LATEST), new StreamsConfig(StreamsTestUtils.getStreamsConfig()), () -> collector, null @@ -115,7 +132,7 @@ public InternalMockProcessorContext(final File stateDir, this(stateDir, keySerde, valSerde, - new StreamsMetricsImpl(new Metrics(), "mock"), + new StreamsMetricsImpl(new Metrics(), "mock", StreamsConfig.METRICS_LATEST), new StreamsConfig(StreamsTestUtils.getStreamsConfig()), () -> collector, cache @@ -139,6 +156,7 @@ public InternalMockProcessorContext(final File stateDir, this.keySerde = keySerde; this.valSerde = valSerde; this.recordCollectorSupplier = collectorSupplier; + this.metrics().setRocksDBMetricsRecordingTrigger(new RocksDBMetricsRecordingTrigger()); } @Override diff --git a/streams/src/test/java/org/apache/kafka/test/MockClientSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockClientSupplier.java index c199ac7a66e6..746dbd101a1a 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockClientSupplier.java +++ b/streams/src/test/java/org/apache/kafka/test/MockClientSupplier.java @@ -58,7 +58,7 @@ public void setClusterForAdminClient(final Cluster cluster) { @Override public Admin getAdmin(final Map config) { - return new MockAdminClient(cluster.nodes(), cluster.nodeById(0)); + return new MockAdminClient(cluster.nodes(), cluster.nodeById(-1)); } @Override diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index d0ca6eae253d..a0ca6b2cad2d 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -74,6 +74,7 @@ import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.internals.ThreadCache; +import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecordingTrigger; import org.apache.kafka.streams.test.ConsumerRecordFactory; import org.apache.kafka.streams.test.OutputVerifier; import org.slf4j.Logger; @@ -280,7 +281,12 @@ public List partitionsFor(final String topic) { metrics = new Metrics(metricConfig, mockWallClockTime); final String threadName = "topology-test-driver-virtual-thread"; - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, threadName); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( + metrics, + threadName, + StreamsConfig.METRICS_LATEST + ); + streamsMetrics.setRocksDBMetricsRecordingTrigger(new RocksDBMetricsRecordingTrigger()); final Sensor skippedRecordsSensor = streamsMetrics.threadLevelSensor("skipped-records", Sensor.RecordingLevel.INFO); final String threadLevelGroup = "stream-metrics"; skippedRecordsSensor.add(new MetricName("skipped-records-rate", diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java index c478b6ba2bf0..041bf0d65d8b 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java @@ -214,7 +214,7 @@ public MockProcessorContext(final Properties config, final TaskId taskId, final final MetricConfig metricConfig = new MetricConfig(); metricConfig.recordLevel(Sensor.RecordingLevel.DEBUG); final String threadName = "mock-processor-context-virtual-thread"; - this.metrics = new StreamsMetricsImpl(new Metrics(metricConfig), threadName); + this.metrics = new StreamsMetricsImpl(new Metrics(metricConfig), threadName, StreamsConfig.METRICS_LATEST); ThreadMetrics.skipRecordSensor(metrics); } diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 83330679edf6..ea90533b03ac 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -65,6 +65,8 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service): DATA_LOG_DIR_2 = "%s-2" % (DATA_LOG_DIR_PREFIX) CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "kafka.properties") # Kafka Authorizer + ACL_AUTHORIZER = "kafka.security.authorizer.AclAuthorizer" + # Old Kafka Authorizer. This is deprecated but still supported. SIMPLE_AUTHORIZER = "kafka.security.auth.SimpleAclAuthorizer" HEAP_DUMP_FILE = os.path.join(PERSISTENT_ROOT, "kafka_heap_dump.bin") INTERBROKER_LISTENER_NAME = 'INTERNAL' diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py index 329d9f9404df..ada6ca27b487 100644 --- a/tests/kafkatest/tests/connect/connect_distributed_test.py +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py @@ -24,7 +24,7 @@ from kafkatest.services.connect import ConnectDistributedService, VerifiableSource, VerifiableSink, ConnectRestError, MockSink, MockSource from kafkatest.services.console_consumer import ConsoleConsumer from kafkatest.services.security.security_config import SecurityConfig -from kafkatest.version import DEV_BRANCH, LATEST_0_11_0, LATEST_0_10_2, LATEST_0_10_1, LATEST_0_10_0, LATEST_0_9, LATEST_0_8_2, KafkaVersion +from kafkatest.version import DEV_BRANCH, LATEST_2_3, LATEST_2_2, LATEST_2_1, LATEST_2_0, LATEST_1_1, LATEST_1_0, LATEST_0_11_0, LATEST_0_10_2, LATEST_0_10_1, LATEST_0_10_0, LATEST_0_9, LATEST_0_8_2, KafkaVersion from collections import Counter, namedtuple import itertools @@ -528,11 +528,23 @@ def test_transformations(self, connect_protocol): @cluster(num_nodes=5) @parametrize(broker_version=str(DEV_BRANCH), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') + @parametrize(broker_version=str(LATEST_2_3), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') + @parametrize(broker_version=str(LATEST_2_2), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') + @parametrize(broker_version=str(LATEST_2_1), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') + @parametrize(broker_version=str(LATEST_2_0), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') + @parametrize(broker_version=str(LATEST_1_1), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') + @parametrize(broker_version=str(LATEST_1_0), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') @parametrize(broker_version=str(LATEST_0_11_0), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') @parametrize(broker_version=str(LATEST_0_10_2), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') @parametrize(broker_version=str(LATEST_0_10_1), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') @parametrize(broker_version=str(LATEST_0_10_0), auto_create_topics=True, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='compatible') @parametrize(broker_version=str(DEV_BRANCH), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') + @parametrize(broker_version=str(LATEST_2_3), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') + @parametrize(broker_version=str(LATEST_2_2), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') + @parametrize(broker_version=str(LATEST_2_1), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') + @parametrize(broker_version=str(LATEST_2_0), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') + @parametrize(broker_version=str(LATEST_1_1), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') + @parametrize(broker_version=str(LATEST_1_0), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') @parametrize(broker_version=str(LATEST_0_11_0), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') @parametrize(broker_version=str(LATEST_0_10_2), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') @parametrize(broker_version=str(LATEST_0_10_1), auto_create_topics=False, security_protocol=SecurityConfig.PLAINTEXT, connect_protocol='eager') diff --git a/tests/kafkatest/tests/core/security_rolling_upgrade_test.py b/tests/kafkatest/tests/core/security_rolling_upgrade_test.py index a64363c266a7..b4c92fe5f508 100644 --- a/tests/kafkatest/tests/core/security_rolling_upgrade_test.py +++ b/tests/kafkatest/tests/core/security_rolling_upgrade_test.py @@ -73,8 +73,8 @@ def roll_in_secured_settings(self, client_protocol, broker_protocol): self.kafka.close_port(SecurityConfig.PLAINTEXT) self.set_authorizer_and_bounce(client_protocol, broker_protocol) - def set_authorizer_and_bounce(self, client_protocol, broker_protocol): - self.kafka.authorizer_class_name = KafkaService.SIMPLE_AUTHORIZER + def set_authorizer_and_bounce(self, client_protocol, broker_protocol, authorizer_class_name = KafkaService.ACL_AUTHORIZER): + self.kafka.authorizer_class_name = authorizer_class_name self.acls.set_acls(client_protocol, self.kafka, self.topic, self.group) self.acls.set_acls(broker_protocol, self.kafka, self.topic, self.group) self.bounce() @@ -95,8 +95,8 @@ def roll_in_sasl_mechanism(self, security_protocol, new_sasl_mechanism): self.kafka.interbroker_sasl_mechanism = new_sasl_mechanism self.bounce() - # Bounce again with ACLs for new mechanism - self.set_authorizer_and_bounce(security_protocol, security_protocol) + # Bounce again with ACLs for new mechanism. Use old SimpleAclAuthorizer here to ensure that is also tested. + self.set_authorizer_and_bounce(security_protocol, security_protocol, KafkaService.SIMPLE_AUTHORIZER) def add_separate_broker_listener(self, broker_security_protocol, broker_sasl_mechanism): self.kafka.setup_interbroker_listener(broker_security_protocol, True) diff --git a/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py b/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py index 235f2fabecf9..d44bd4fd8f62 100644 --- a/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py +++ b/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py @@ -102,7 +102,7 @@ def test_zk_security_upgrade(self, security_protocol): # set acls if self.is_secure: - self.kafka.authorizer_class_name = KafkaService.SIMPLE_AUTHORIZER + self.kafka.authorizer_class_name = KafkaService.ACL_AUTHORIZER self.acls.set_acls(security_protocol, self.kafka, self.topic, self.group) if self.no_sasl: diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java b/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java index a0ac1f188f21..cfbac1a4e4c4 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java +++ b/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java @@ -195,13 +195,13 @@ private static Map consumerPositions(KafkaCon } private static void resetToLastCommittedPositions(KafkaConsumer consumer) { - for (TopicPartition topicPartition : consumer.assignment()) { - OffsetAndMetadata offsetAndMetadata = consumer.committed(topicPartition); + final Map committed = consumer.committed(consumer.assignment()); + committed.forEach((tp, offsetAndMetadata) -> { if (offsetAndMetadata != null) - consumer.seek(topicPartition, offsetAndMetadata.offset()); + consumer.seek(tp, offsetAndMetadata.offset()); else - consumer.seekToBeginning(singleton(topicPartition)); - } + consumer.seekToBeginning(singleton(tp)); + }); } private static long messagesRemaining(KafkaConsumer consumer, TopicPartition partition) {

    file.delete.delay.ms